aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/ml/tuning.py
diff options
context:
space:
mode:
authorWeichenXu <WeichenXu123@outlook.com>2016-05-23 18:14:48 -0700
committerAndrew Or <andrew@databricks.com>2016-05-23 18:14:48 -0700
commita15ca5533db91fefaf3248255a59c4d94eeda1a9 (patch)
tree80867e08b17b01d96611a9a695cbb1f01c0198f6 /python/pyspark/ml/tuning.py
parent5afd927a47aa7ede3039234f2f7262e2247aa2ae (diff)
downloadspark-a15ca5533db91fefaf3248255a59c4d94eeda1a9.tar.gz
spark-a15ca5533db91fefaf3248255a59c4d94eeda1a9.tar.bz2
spark-a15ca5533db91fefaf3248255a59c4d94eeda1a9.zip
[SPARK-15464][ML][MLLIB][SQL][TESTS] Replace SQLContext and SparkContext with SparkSession using builder pattern in python test code
## What changes were proposed in this pull request? Replace SQLContext and SparkContext with SparkSession using builder pattern in python test code. ## How was this patch tested? Existing test. Author: WeichenXu <WeichenXu123@outlook.com> Closes #13242 from WeichenXu123/python_doctest_update_sparksession.
Diffstat (limited to 'python/pyspark/ml/tuning.py')
-rw-r--r--python/pyspark/ml/tuning.py18
1 files changed, 10 insertions, 8 deletions
diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py
index 4f7a6b0f7b..fe87b6cdb9 100644
--- a/python/pyspark/ml/tuning.py
+++ b/python/pyspark/ml/tuning.py
@@ -152,7 +152,7 @@ class CrossValidator(Estimator, ValidatorParams):
>>> from pyspark.ml.classification import LogisticRegression
>>> from pyspark.ml.evaluation import BinaryClassificationEvaluator
>>> from pyspark.ml.linalg import Vectors
- >>> dataset = sqlContext.createDataFrame(
+ >>> dataset = spark.createDataFrame(
... [(Vectors.dense([0.0]), 0.0),
... (Vectors.dense([0.4]), 1.0),
... (Vectors.dense([0.5]), 0.0),
@@ -311,7 +311,7 @@ class TrainValidationSplit(Estimator, ValidatorParams):
>>> from pyspark.ml.classification import LogisticRegression
>>> from pyspark.ml.evaluation import BinaryClassificationEvaluator
>>> from pyspark.ml.linalg import Vectors
- >>> dataset = sqlContext.createDataFrame(
+ >>> dataset = spark.createDataFrame(
... [(Vectors.dense([0.0]), 0.0),
... (Vectors.dense([0.4]), 1.0),
... (Vectors.dense([0.5]), 0.0),
@@ -456,17 +456,19 @@ class TrainValidationSplitModel(Model, ValidatorParams):
if __name__ == "__main__":
import doctest
- from pyspark.context import SparkContext
- from pyspark.sql import SQLContext
+ from pyspark.sql import SparkSession
globs = globals().copy()
# The small batch size here ensures that we see multiple batches,
# even in these small test examples:
- sc = SparkContext("local[2]", "ml.tuning tests")
- sqlContext = SQLContext(sc)
+ spark = SparkSession.builder\
+ .master("local[2]")\
+ .appName("ml.tuning tests")\
+ .getOrCreate()
+ sc = spark.sparkContext
globs['sc'] = sc
- globs['sqlContext'] = sqlContext
+ globs['spark'] = spark
(failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS)
- sc.stop()
+ spark.stop()
if failure_count:
exit(-1)