|
25 | 25 |
|
26 | 26 | from numpy import array, random, tile |
27 | 27 |
|
| 28 | +from collections import namedtuple |
| 29 | + |
28 | 30 | from pyspark import SparkContext |
29 | 31 | from pyspark.rdd import RDD, ignore_unicode_prefix |
30 | | -from pyspark.mllib.common import callMLlibFunc, callJavaFunc, _py2java, _java2py |
| 32 | +from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, callJavaFunc, _py2java, _java2py |
31 | 33 | from pyspark.mllib.linalg import SparseVector, _convert_to_vector, DenseVector |
32 | 34 | from pyspark.mllib.stat.distribution import MultivariateGaussian |
33 | | -from pyspark.mllib.util import Saveable, Loader, inherit_doc |
| 35 | +from pyspark.mllib.util import Saveable, Loader, inherit_doc, JavaLoader, JavaSaveable |
34 | 36 | from pyspark.streaming import DStream |
35 | 37 |
|
36 | 38 | __all__ = ['KMeansModel', 'KMeans', 'GaussianMixtureModel', 'GaussianMixture', |
| 39 | + 'PowerIterationClusteringModel', 'PowerIterationClustering', |
37 | 40 | 'StreamingKMeans', 'StreamingKMeansModel'] |
38 | 41 |
|
39 | 42 |
|
@@ -272,6 +275,94 @@ def train(cls, rdd, k, convergenceTol=1e-3, maxIterations=100, seed=None, initia |
272 | 275 | return GaussianMixtureModel(weight, mvg_obj) |
273 | 276 |
|
274 | 277 |
|
| 278 | +class PowerIterationClusteringModel(JavaModelWrapper, JavaSaveable, JavaLoader): |
| 279 | + |
| 280 | + """ |
| 281 | + .. note:: Experimental |
| 282 | +
|
| 283 | + Model produced by [[PowerIterationClustering]]. |
| 284 | +
|
| 285 | + >>> data = [(0, 1, 1.0), (0, 2, 1.0), (1, 3, 1.0), (2, 3, 1.0), |
| 286 | + ... (0, 3, 1.0), (1, 2, 1.0), (0, 4, 0.1)] |
| 287 | + >>> rdd = sc.parallelize(data, 2) |
| 288 | + >>> model = PowerIterationClustering.train(rdd, 2, 100) |
| 289 | + >>> model.k |
| 290 | + 2 |
| 291 | + >>> sorted(model.assignments().collect()) |
| 292 | + [Assignment(id=0, cluster=1), Assignment(id=1, cluster=0), ... |
| 293 | + >>> import os, tempfile |
| 294 | + >>> path = tempfile.mkdtemp() |
| 295 | + >>> model.save(sc, path) |
| 296 | + >>> sameModel = PowerIterationClusteringModel.load(sc, path) |
| 297 | + >>> sameModel.k |
| 298 | + 2 |
| 299 | + >>> sorted(sameModel.assignments().collect()) |
| 300 | + [Assignment(id=0, cluster=1), Assignment(id=1, cluster=0), ... |
| 301 | + >>> from shutil import rmtree |
| 302 | + >>> try: |
| 303 | + ... rmtree(path) |
| 304 | + ... except OSError: |
| 305 | + ... pass |
| 306 | + """ |
| 307 | + |
| 308 | + @property |
| 309 | + def k(self): |
| 310 | + """ |
| 311 | + Returns the number of clusters. |
| 312 | + """ |
| 313 | + return self.call("k") |
| 314 | + |
| 315 | + def assignments(self): |
| 316 | + """ |
| 317 | + Returns the cluster assignments of this model. |
| 318 | + """ |
| 319 | + return self.call("getAssignments").map( |
| 320 | + lambda x: (PowerIterationClustering.Assignment(*x))) |
| 321 | + |
| 322 | + @classmethod |
| 323 | + def load(cls, sc, path): |
| 324 | + model = cls._load_java(sc, path) |
| 325 | + wrapper = sc._jvm.PowerIterationClusteringModelWrapper(model) |
| 326 | + return PowerIterationClusteringModel(wrapper) |
| 327 | + |
| 328 | + |
| 329 | +class PowerIterationClustering(object): |
| 330 | + """ |
| 331 | + .. note:: Experimental |
| 332 | +
|
| 333 | + Power Iteration Clustering (PIC), a scalable graph clustering algorithm |
| 334 | + developed by [[http://www.icml2010.org/papers/387.pdf Lin and Cohen]]. |
| 335 | + From the abstract: PIC finds a very low-dimensional embedding of a |
| 336 | + dataset using truncated power iteration on a normalized pair-wise |
| 337 | + similarity matrix of the data. |
| 338 | + """ |
| 339 | + |
| 340 | + @classmethod |
| 341 | + def train(cls, rdd, k, maxIterations=100, initMode="random"): |
| 342 | + """ |
| 343 | + :param rdd: an RDD of (i, j, s,,ij,,) tuples representing the |
| 344 | + affinity matrix, which is the matrix A in the PIC paper. |
| 345 | + The similarity s,,ij,, must be nonnegative. |
| 346 | + This is a symmetric matrix and hence s,,ij,, = s,,ji,,. |
| 347 | + For any (i, j) with nonzero similarity, there should be |
| 348 | + either (i, j, s,,ij,,) or (j, i, s,,ji,,) in the input. |
| 349 | + Tuples with i = j are ignored, because we assume |
| 350 | + s,,ij,, = 0.0. |
| 351 | + :param k: Number of clusters. |
| 352 | + :param maxIterations: Maximum number of iterations of the |
| 353 | + PIC algorithm. |
| 354 | + :param initMode: Initialization mode. |
| 355 | + """ |
| 356 | + model = callMLlibFunc("trainPowerIterationClusteringModel", |
| 357 | + rdd.map(_convert_to_vector), int(k), int(maxIterations), initMode) |
| 358 | + return PowerIterationClusteringModel(model) |
| 359 | + |
| 360 | + class Assignment(namedtuple("Assignment", ["id", "cluster"])): |
| 361 | + """ |
| 362 | + Represents an (id, cluster) tuple. |
| 363 | + """ |
| 364 | + |
| 365 | + |
275 | 366 | class StreamingKMeansModel(KMeansModel): |
276 | 367 | """ |
277 | 368 | .. note:: Experimental |
@@ -466,7 +557,8 @@ def predictOnValues(self, dstream): |
466 | 557 |
|
467 | 558 | def _test(): |
468 | 559 | import doctest |
469 | | - globs = globals().copy() |
| 560 | + import pyspark.mllib.clustering |
| 561 | + globs = pyspark.mllib.clustering.__dict__.copy() |
470 | 562 | globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) |
471 | 563 | (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) |
472 | 564 | globs['sc'].stop() |
|
0 commit comments