Skip to content

Commit

Permalink
[SPARK-4118] [MLLIB] [PYSPARK] Python bindings for StreamingKMeans
Browse files Browse the repository at this point in the history
Python bindings for StreamingKMeans

Will change status to MRG once docs, tests and examples are updated.

Author: MechCoder <[email protected]>

Closes #6499 from MechCoder/spark-4118 and squashes the following commits:

7722d16 [MechCoder] minor style fixes
51052d3 [MechCoder] Doc fixes
2061a76 [MechCoder] Add tests for simultaneous training and prediction Minor style fixes
81482fd [MechCoder] minor
5d9fe61 [MechCoder] predictOn should take into account the latest model
8ab9e89 [MechCoder] Fix Python3 error
a9817df [MechCoder] Better tests and minor fixes
c80e451 [MechCoder] Add ignore_unicode_prefix
ee8ce16 [MechCoder] Update tests, doc and examples
4b1481f [MechCoder] Some changes and tests
d8b066a [MechCoder] [SPARK-4118] [MLlib] [PySpark] Python bindings for StreamingKMeans
  • Loading branch information
MechCoder authored and mengxr committed Jun 19, 2015
1 parent e41e2fd commit 54976e5
Show file tree
Hide file tree
Showing 4 changed files with 411 additions and 9 deletions.
48 changes: 44 additions & 4 deletions docs/mllib-clustering.md
Original file line number Diff line number Diff line change
Expand Up @@ -592,15 +592,55 @@ model.predictOnValues(testData.map(lp => (lp.label, lp.features))).print()
ssc.start()
ssc.awaitTermination()

{% endhighlight %}
</div>

<div data-lang="python" markdown="1">
First we import the neccessary classes.

{% highlight python %}
from pyspark.mllib.linalg import Vectors
from pyspark.mllib.regression import LabeledPoint
from pyspark.mllib.clustering import StreamingKMeans
{% endhighlight %}

Then we make an input stream of vectors for training, as well as a stream of labeled data
points for testing. We assume a StreamingContext `ssc` has been created, see
[Spark Streaming Programming Guide](streaming-programming-guide.html#initializing) for more info.

{% highlight python %}
def parse(lp):
label = float(lp[lp.find('(') + 1: lp.find(',')])
vec = Vectors.dense(lp[lp.find('[') + 1: lp.find(']')].split(','))
return LabeledPoint(label, vec)

trainingData = ssc.textFileStream("/training/data/dir").map(Vectors.parse)
testData = ssc.textFileStream("/testing/data/dir").map(parse)
{% endhighlight %}

We create a model with random clusters and specify the number of clusters to find

{% highlight python %}
model = StreamingKMeans(k=2, decayFactor=1.0).setRandomCenters(3, 1.0, 0)
{% endhighlight %}

Now register the streams for training and testing and start the job, printing
the predicted cluster assignments on new data points as they arrive.

{% highlight python %}
model.trainOn(trainingData)
print(model.predictOnValues(testData.map(lambda lp: (lp.label, lp.features))))

ssc.start()
ssc.awaitTermination()
{% endhighlight %}
</div>

</div>

As you add new text files with data the cluster centers will update. Each training
point should be formatted as `[x1, x2, x3]`, and each test data point
should be formatted as `(y, [x1, x2, x3])`, where `y` is some useful label or identifier
(e.g. a true category assignment). Anytime a text file is placed in `/training/data/dir`
the model will update. Anytime a text file is placed in `/testing/data/dir`
you will see predictions. With new data, the cluster centers will change!

</div>

</div>
Original file line number Diff line number Diff line change
Expand Up @@ -964,6 +964,21 @@ private[python] class PythonMLLibAPI extends Serializable {
points.asScala.toArray)
}

/**
* Java stub for the update method of StreamingKMeansModel.
*/
def updateStreamingKMeansModel(
clusterCenters: JList[Vector],
clusterWeights: JList[Double],
data: JavaRDD[Vector],
decayFactor: Double,
timeUnit: String): JList[Object] = {
val model = new StreamingKMeansModel(
clusterCenters.asScala.toArray, clusterWeights.asScala.toArray)
.update(data, decayFactor, timeUnit)
List[AnyRef](model.clusterCenters, Vectors.dense(model.clusterWeights)).asJava
}

}

/**
Expand Down
207 changes: 203 additions & 4 deletions python/pyspark/mllib/clustering.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,16 +21,20 @@
if sys.version > '3':
xrange = range

from numpy import array
from math import exp, log

from numpy import array, random, tile

from pyspark import RDD
from pyspark import SparkContext
from pyspark.rdd import RDD, ignore_unicode_prefix
from pyspark.mllib.common import callMLlibFunc, callJavaFunc, _py2java, _java2py
from pyspark.mllib.linalg import SparseVector, _convert_to_vector
from pyspark.mllib.linalg import SparseVector, _convert_to_vector, DenseVector
from pyspark.mllib.stat.distribution import MultivariateGaussian
from pyspark.mllib.util import Saveable, Loader, inherit_doc
from pyspark.streaming import DStream

__all__ = ['KMeansModel', 'KMeans', 'GaussianMixtureModel', 'GaussianMixture']
__all__ = ['KMeansModel', 'KMeans', 'GaussianMixtureModel', 'GaussianMixture',
'StreamingKMeans', 'StreamingKMeansModel']


@inherit_doc
Expand Down Expand Up @@ -98,6 +102,9 @@ def predict(self, x):
"""Find the cluster to which x belongs in this model."""
best = 0
best_distance = float("inf")
if isinstance(x, RDD):
return x.map(self.predict)

x = _convert_to_vector(x)
for i in xrange(len(self.centers)):
distance = x.squared_distance(self.centers[i])
Expand Down Expand Up @@ -264,6 +271,198 @@ def train(cls, rdd, k, convergenceTol=1e-3, maxIterations=100, seed=None, initia
return GaussianMixtureModel(weight, mvg_obj)


class StreamingKMeansModel(KMeansModel):
"""
.. note:: Experimental
Clustering model which can perform an online update of the centroids.
The update formula for each centroid is given by
* c_t+1 = ((c_t * n_t * a) + (x_t * m_t)) / (n_t + m_t)
* n_t+1 = n_t * a + m_t
where
* c_t: Centroid at the n_th iteration.
* n_t: Number of samples (or) weights associated with the centroid
at the n_th iteration.
* x_t: Centroid of the new data closest to c_t.
* m_t: Number of samples (or) weights of the new data closest to c_t
* c_t+1: New centroid.
* n_t+1: New number of weights.
* a: Decay Factor, which gives the forgetfulness.
Note that if a is set to 1, it is the weighted mean of the previous
and new data. If it set to zero, the old centroids are completely
forgotten.
:param clusterCenters: Initial cluster centers.
:param clusterWeights: List of weights assigned to each cluster.
>>> initCenters = [[0.0, 0.0], [1.0, 1.0]]
>>> initWeights = [1.0, 1.0]
>>> stkm = StreamingKMeansModel(initCenters, initWeights)
>>> data = sc.parallelize([[-0.1, -0.1], [0.1, 0.1],
... [0.9, 0.9], [1.1, 1.1]])
>>> stkm = stkm.update(data, 1.0, u"batches")
>>> stkm.centers
array([[ 0., 0.],
[ 1., 1.]])
>>> stkm.predict([-0.1, -0.1])
0
>>> stkm.predict([0.9, 0.9])
1
>>> stkm.clusterWeights
[3.0, 3.0]
>>> decayFactor = 0.0
>>> data = sc.parallelize([DenseVector([1.5, 1.5]), DenseVector([0.2, 0.2])])
>>> stkm = stkm.update(data, 0.0, u"batches")
>>> stkm.centers
array([[ 0.2, 0.2],
[ 1.5, 1.5]])
>>> stkm.clusterWeights
[1.0, 1.0]
>>> stkm.predict([0.2, 0.2])
0
>>> stkm.predict([1.5, 1.5])
1
"""
def __init__(self, clusterCenters, clusterWeights):
super(StreamingKMeansModel, self).__init__(centers=clusterCenters)
self._clusterWeights = list(clusterWeights)

@property
def clusterWeights(self):
"""Return the cluster weights."""
return self._clusterWeights

@ignore_unicode_prefix
def update(self, data, decayFactor, timeUnit):
"""Update the centroids, according to data
:param data: Should be a RDD that represents the new data.
:param decayFactor: forgetfulness of the previous centroids.
:param timeUnit: Can be "batches" or "points". If points, then the
decay factor is raised to the power of number of new
points and if batches, it is used as it is.
"""
if not isinstance(data, RDD):
raise TypeError("Data should be of an RDD, got %s." % type(data))
data = data.map(_convert_to_vector)
decayFactor = float(decayFactor)
if timeUnit not in ["batches", "points"]:
raise ValueError(
"timeUnit should be 'batches' or 'points', got %s." % timeUnit)
vectorCenters = [_convert_to_vector(center) for center in self.centers]
updatedModel = callMLlibFunc(
"updateStreamingKMeansModel", vectorCenters, self._clusterWeights,
data, decayFactor, timeUnit)
self.centers = array(updatedModel[0])
self._clusterWeights = list(updatedModel[1])
return self


class StreamingKMeans(object):
"""
.. note:: Experimental
Provides methods to set k, decayFactor, timeUnit to configure the
KMeans algorithm for fitting and predicting on incoming dstreams.
More details on how the centroids are updated are provided under the
docs of StreamingKMeansModel.
:param k: int, number of clusters
:param decayFactor: float, forgetfulness of the previous centroids.
:param timeUnit: can be "batches" or "points". If points, then the
decayfactor is raised to the power of no. of new points.
"""
def __init__(self, k=2, decayFactor=1.0, timeUnit="batches"):
self._k = k
self._decayFactor = decayFactor
if timeUnit not in ["batches", "points"]:
raise ValueError(
"timeUnit should be 'batches' or 'points', got %s." % timeUnit)
self._timeUnit = timeUnit
self._model = None

def latestModel(self):
"""Return the latest model"""
return self._model

def _validate(self, dstream):
if self._model is None:
raise ValueError(
"Initial centers should be set either by setInitialCenters "
"or setRandomCenters.")
if not isinstance(dstream, DStream):
raise TypeError(
"Expected dstream to be of type DStream, "
"got type %s" % type(dstream))

def setK(self, k):
"""Set number of clusters."""
self._k = k
return self

def setDecayFactor(self, decayFactor):
"""Set decay factor."""
self._decayFactor = decayFactor
return self

def setHalfLife(self, halfLife, timeUnit):
"""
Set number of batches after which the centroids of that
particular batch has half the weightage.
"""
self._timeUnit = timeUnit
self._decayFactor = exp(log(0.5) / halfLife)
return self

def setInitialCenters(self, centers, weights):
"""
Set initial centers. Should be set before calling trainOn.
"""
self._model = StreamingKMeansModel(centers, weights)
return self

def setRandomCenters(self, dim, weight, seed):
"""
Set the initial centres to be random samples from
a gaussian population with constant weights.
"""
rng = random.RandomState(seed)
clusterCenters = rng.randn(self._k, dim)
clusterWeights = tile(weight, self._k)
self._model = StreamingKMeansModel(clusterCenters, clusterWeights)
return self

def trainOn(self, dstream):
"""Train the model on the incoming dstream."""
self._validate(dstream)

def update(rdd):
self._model.update(rdd, self._decayFactor, self._timeUnit)

dstream.foreachRDD(update)

def predictOn(self, dstream):
"""
Make predictions on a dstream.
Returns a transformed dstream object
"""
self._validate(dstream)
return dstream.map(lambda x: self._model.predict(x))

def predictOnValues(self, dstream):
"""
Make predictions on a keyed dstream.
Returns a transformed dstream object.
"""
self._validate(dstream)
return dstream.mapValues(lambda x: self._model.predict(x))


def _test():
import doctest
globs = globals().copy()
Expand Down
Loading

0 comments on commit 54976e5

Please sign in to comment.