aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/ml/regression.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/regression.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/regression.py')
-rw-r--r--python/pyspark/ml/regression.py46
1 files changed, 24 insertions, 22 deletions
diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py
index e21dd83923..8f58594a66 100644
--- a/python/pyspark/ml/regression.py
+++ b/python/pyspark/ml/regression.py
@@ -55,19 +55,19 @@ class LinearRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPrediction
- L2 + L1 (elastic net)
>>> from pyspark.ml.linalg import Vectors
- >>> df = sqlContext.createDataFrame([
+ >>> df = spark.createDataFrame([
... (1.0, 2.0, Vectors.dense(1.0)),
... (0.0, 2.0, Vectors.sparse(1, [], []))], ["label", "weight", "features"])
>>> lr = LinearRegression(maxIter=5, regParam=0.0, solver="normal", weightCol="weight")
>>> model = lr.fit(df)
- >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"])
+ >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"])
>>> abs(model.transform(test0).head().prediction - (-1.0)) < 0.001
True
>>> abs(model.coefficients[0] - 1.0) < 0.001
True
>>> abs(model.intercept - 0.0) < 0.001
True
- >>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"])
+ >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"])
>>> abs(model.transform(test1).head().prediction - 1.0) < 0.001
True
>>> lr.setParams("vector")
@@ -413,12 +413,12 @@ class IsotonicRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti
Only univariate (single feature) algorithm supported.
>>> from pyspark.ml.linalg import Vectors
- >>> df = sqlContext.createDataFrame([
+ >>> df = spark.createDataFrame([
... (1.0, Vectors.dense(1.0)),
... (0.0, Vectors.sparse(1, [], []))], ["label", "features"])
>>> ir = IsotonicRegression()
>>> model = ir.fit(df)
- >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"])
+ >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"])
>>> model.transform(test0).head().prediction
0.0
>>> model.boundaries
@@ -643,7 +643,7 @@ class DecisionTreeRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi
It supports both continuous and categorical features.
>>> from pyspark.ml.linalg import Vectors
- >>> df = sqlContext.createDataFrame([
+ >>> df = spark.createDataFrame([
... (1.0, Vectors.dense(1.0)),
... (0.0, Vectors.sparse(1, [], []))], ["label", "features"])
>>> dt = DecisionTreeRegressor(maxDepth=2, varianceCol="variance")
@@ -654,10 +654,10 @@ class DecisionTreeRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi
3
>>> model.featureImportances
SparseVector(1, {0: 1.0})
- >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"])
+ >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"])
>>> model.transform(test0).head().prediction
0.0
- >>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"])
+ >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"])
>>> model.transform(test1).head().prediction
1.0
>>> dtr_path = temp_path + "/dtr"
@@ -809,7 +809,7 @@ class RandomForestRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi
>>> from numpy import allclose
>>> from pyspark.ml.linalg import Vectors
- >>> df = sqlContext.createDataFrame([
+ >>> df = spark.createDataFrame([
... (1.0, Vectors.dense(1.0)),
... (0.0, Vectors.sparse(1, [], []))], ["label", "features"])
>>> rf = RandomForestRegressor(numTrees=2, maxDepth=2, seed=42)
@@ -818,10 +818,10 @@ class RandomForestRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi
SparseVector(1, {0: 1.0})
>>> allclose(model.treeWeights, [1.0, 1.0])
True
- >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"])
+ >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"])
>>> model.transform(test0).head().prediction
0.0
- >>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"])
+ >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"])
>>> model.transform(test1).head().prediction
0.5
>>> rfr_path = temp_path + "/rfr"
@@ -921,7 +921,7 @@ class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol,
>>> from numpy import allclose
>>> from pyspark.ml.linalg import Vectors
- >>> df = sqlContext.createDataFrame([
+ >>> df = spark.createDataFrame([
... (1.0, Vectors.dense(1.0)),
... (0.0, Vectors.sparse(1, [], []))], ["label", "features"])
>>> gbt = GBTRegressor(maxIter=5, maxDepth=2, seed=42)
@@ -932,10 +932,10 @@ class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol,
SparseVector(1, {0: 1.0})
>>> allclose(model.treeWeights, [1.0, 0.1, 0.1, 0.1, 0.1])
True
- >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"])
+ >>> test0 = spark.createDataFrame([(Vectors.dense(-1.0),)], ["features"])
>>> model.transform(test0).head().prediction
0.0
- >>> test1 = sqlContext.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"])
+ >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"])
>>> model.transform(test1).head().prediction
1.0
>>> gbtr_path = temp_path + "gbtr"
@@ -1056,7 +1056,7 @@ class AFTSurvivalRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi
.. seealso:: `AFT Model <https://en.wikipedia.org/wiki/Accelerated_failure_time_model>`_
>>> from pyspark.ml.linalg import Vectors
- >>> df = sqlContext.createDataFrame([
+ >>> df = spark.createDataFrame([
... (1.0, Vectors.dense(1.0), 1.0),
... (0.0, Vectors.sparse(1, [], []), 0.0)], ["label", "features", "censor"])
>>> aftsr = AFTSurvivalRegression()
@@ -1257,7 +1257,7 @@ class GeneralizedLinearRegression(JavaEstimator, HasLabelCol, HasFeaturesCol, Ha
.. seealso:: `GLM <https://en.wikipedia.org/wiki/Generalized_linear_model>`_
>>> from pyspark.ml.linalg import Vectors
- >>> df = sqlContext.createDataFrame([
+ >>> df = spark.createDataFrame([
... (1.0, Vectors.dense(0.0, 0.0)),
... (1.0, Vectors.dense(1.0, 2.0)),
... (2.0, Vectors.dense(0.0, 0.0)),
@@ -1603,21 +1603,23 @@ class GeneralizedLinearRegressionTrainingSummary(GeneralizedLinearRegressionSumm
if __name__ == "__main__":
import doctest
import pyspark.ml.regression
- from pyspark.context import SparkContext
- from pyspark.sql import SQLContext
+ from pyspark.sql import SparkSession
globs = pyspark.ml.regression.__dict__.copy()
# The small batch size here ensures that we see multiple batches,
# even in these small test examples:
- sc = SparkContext("local[2]", "ml.regression tests")
- sqlContext = SQLContext(sc)
+ spark = SparkSession.builder\
+ .master("local[2]")\
+ .appName("ml.regression tests")\
+ .getOrCreate()
+ sc = spark.sparkContext
globs['sc'] = sc
- globs['sqlContext'] = sqlContext
+ globs['spark'] = spark
import tempfile
temp_path = tempfile.mkdtemp()
globs['temp_path'] = temp_path
try:
(failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS)
- sc.stop()
+ spark.stop()
finally:
from shutil import rmtree
try: