diff options
Diffstat (limited to 'python/pyspark/ml/clustering.py')
-rw-r--r-- | python/pyspark/ml/clustering.py | 22 |
1 files changed, 12 insertions, 10 deletions
diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index ac7183d2ef..a457904e78 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -73,7 +73,7 @@ class GaussianMixture(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIte ... (Vectors.dense([0.75, 0.935]),), ... (Vectors.dense([-0.83, -0.68]),), ... (Vectors.dense([-0.91, -0.76]),)] - >>> df = sqlContext.createDataFrame(data, ["features"]) + >>> df = spark.createDataFrame(data, ["features"]) >>> gm = GaussianMixture(k=3, tol=0.0001, ... maxIter=10, seed=10) >>> model = gm.fit(df) @@ -197,7 +197,7 @@ class KMeans(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIter, HasTol >>> from pyspark.ml.linalg import Vectors >>> data = [(Vectors.dense([0.0, 0.0]),), (Vectors.dense([1.0, 1.0]),), ... (Vectors.dense([9.0, 8.0]),), (Vectors.dense([8.0, 9.0]),)] - >>> df = sqlContext.createDataFrame(data, ["features"]) + >>> df = spark.createDataFrame(data, ["features"]) >>> kmeans = KMeans(k=2, seed=1) >>> model = kmeans.fit(df) >>> centers = model.clusterCenters() @@ -350,7 +350,7 @@ class BisectingKMeans(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIte >>> from pyspark.ml.linalg import Vectors >>> data = [(Vectors.dense([0.0, 0.0]),), (Vectors.dense([1.0, 1.0]),), ... (Vectors.dense([9.0, 8.0]),), (Vectors.dense([8.0, 9.0]),)] - >>> df = sqlContext.createDataFrame(data, ["features"]) + >>> df = spark.createDataFrame(data, ["features"]) >>> bkm = BisectingKMeans(k=2, minDivisibleClusterSize=1.0) >>> model = bkm.fit(df) >>> centers = model.clusterCenters() @@ -627,7 +627,7 @@ class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInter >>> from pyspark.ml.linalg import Vectors, SparseVector >>> from pyspark.ml.clustering import LDA - >>> df = sqlContext.createDataFrame([[1, Vectors.dense([0.0, 1.0])], + >>> df = spark.createDataFrame([[1, Vectors.dense([0.0, 1.0])], ... [2, SparseVector(2, {0: 1.0})],], ["id", "features"]) >>> lda = LDA(k=2, seed=1, optimizer="em") >>> model = lda.fit(df) @@ -933,21 +933,23 @@ class LDA(JavaEstimator, HasFeaturesCol, HasMaxIter, HasSeed, HasCheckpointInter if __name__ == "__main__": import doctest import pyspark.ml.clustering - from pyspark.context import SparkContext - from pyspark.sql import SQLContext + from pyspark.sql import SparkSession globs = pyspark.ml.clustering.__dict__.copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: - sc = SparkContext("local[2]", "ml.clustering tests") - sqlContext = SQLContext(sc) + spark = SparkSession.builder\ + .master("local[2]")\ + .appName("ml.clustering 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: |