Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2942#discussion_r19454416
  
    --- Diff: 
mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala ---
    @@ -0,0 +1,246 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.mllib.clustering
    +
    +import breeze.linalg.{Vector => BV}
    +
    +import scala.reflect.ClassTag
    +import scala.util.Random._
    +
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.Logging
    +import org.apache.spark.mllib.linalg.{Vectors, Vector}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.SparkContext._
    +import org.apache.spark.streaming.dstream.DStream
    +import org.apache.spark.streaming.StreamingContext._
    +
    +/**
    + * :: DeveloperApi ::
    + *
    + * StreamingKMeansModel extends MLlib's KMeansModel for streaming
    + * algorithms, so it can keep track of the number of points assigned
    + * to each cluster, and also update the model by doing a single iteration
    + * of the standard KMeans algorithm.
    + *
    + * The update algorithm uses the "mini-batch" KMeans rule,
    + * generalized to incorporate forgetfullness (i.e. decay).
    + * The basic update rule (for each cluster) is:
    + *
    + * c_t+1 = [(c_t * n_t) + (x_t * m_t)] / [n_t + m_t]
    + * n_t+t = n_t + m_t
    + *
    + * Where c_t is the previously estimated centroid for that cluster,
    + * n_t is the number of points assigned to it thus far, x_t is the centroid
    + * estimated on the current batch, and m_t is the number of points assigned
    + * to that centroid in the current batch.
    + *
    + * This update rule is modified with a decay factor 'a' that scales
    + * the contribution of the clusters as estimated thus far.
    + * If a=1, all batches are weighted equally. If a=0, new centroids
    + * are determined entirely by recent data. Lower values correspond to
    + * more forgetting.
    + *
    + * Decay can optionally be specified as a decay fraction 'q',
    + * which corresponds to the fraction of batches (or points)
    + * after which the past will be reduced to a contribution of 0.5.
    + * This decay fraction can be specified in units of 'points' or 'batches'.
    + * if 'batches', behavior will be independent of the number of points per 
batch;
    + * if 'points', the expected number of points per batch must be specified.
    + *
    + * Use a builder pattern to construct a streaming KMeans analysis
    + * in an application, like:
    + *
    + *  val model = new StreamingKMeans()
    + *    .setDecayFactor(0.5)
    + *    .setK(3)
    + *    .setRandomCenters(5)
    + *    .trainOn(DStream)
    + *
    + */
    +@DeveloperApi
    +class StreamingKMeansModel(
    +    override val clusterCenters: Array[Vector],
    +    val clusterCounts: Array[Long]) extends KMeansModel(clusterCenters) 
with Logging {
    +
    +  // do a sequential KMeans update on a batch of data
    +  def update(data: RDD[Vector], a: Double, units: String): 
StreamingKMeansModel = {
    +
    +    val centers = clusterCenters
    +    val counts = clusterCounts
    +
    +    // find nearest cluster to each point
    +    val closest = data.map(point => (this.predict(point), (point.toBreeze, 
1.toLong)))
    +
    +    // get sums and counts for updating each cluster
    +    type WeightedPoint = (BV[Double], Long)
    +    def mergeContribs(p1: WeightedPoint, p2: WeightedPoint): WeightedPoint 
= {
    +      (p1._1 += p2._1, p1._2 + p2._2)
    +    }
    +    val pointStats: Array[(Int, (BV[Double], Long))] =
    +      closest.reduceByKey{mergeContribs}.collectAsMap().toArray
    +
    +    // implement update rule
    +    for (newP <- pointStats) {
    +      // store old count and centroid
    +      val oldCount = counts(newP._1)
    +      val oldCentroid = centers(newP._1).toBreeze
    +      // get new count and centroid
    +      val newCount = newP._2._2
    +      val newCentroid = newP._2._1 / newCount.toDouble
    +      // compute the normalized scale factor that controls forgetting
    +      val decayFactor = units match {
    +        case "batches" =>  newCount / (a * oldCount + newCount)
    +        case "points" => newCount / (math.pow(a, newCount) * oldCount + 
newCount)
    +      }
    +      // perform the update
    +      val updatedCentroid = oldCentroid + (newCentroid - oldCentroid) * 
decayFactor
    +      // store the new counts and centers
    +      counts(newP._1) = oldCount + newCount
    +      centers(newP._1) = Vectors.fromBreeze(updatedCentroid)
    +
    +      // display the updated cluster centers
    +      val display = centers(newP._1).size match {
    +        case x if x > 100 => 
centers(newP._1).toArray.take(100).mkString("[", ",", "...")
    +        case _ => centers(newP._1).toArray.mkString("[", ",", "]")
    +      }
    +      logInfo("Cluster %d updated: %s ".format (newP._1, display))
    +    }
    +    new StreamingKMeansModel(centers, counts)
    +  }
    +
    +}
    +
    +@DeveloperApi
    +class StreamingKMeans(
    +     var k: Int,
    +     var a: Double,
    +     var units: String) extends Logging {
    +
    +  protected var model: StreamingKMeansModel = new 
StreamingKMeansModel(null, null)
    +
    +  def this() = this(2, 1.0, "batches")
    +
    +  /** Set the number of clusters. */
    +  def setK(k: Int): this.type = {
    +    this.k = k
    +    this
    +  }
    +
    +  /** Set the decay factor directly (for forgetful algorithms). */
    +  def setDecayFactor(a: Double): this.type = {
    +    this.a = a
    +    this
    +  }
    +
    +  /** Set the decay units for forgetful algorithms ("batches" or 
"points"). */
    +  def setUnits(units: String): this.type = {
    +    if (units != "batches" && units != "points") {
    +      throw new IllegalArgumentException("Invalid units for decay: " + 
units)
    +    }
    +    this.units = units
    +    this
    +  }
    +
    +  /** Set decay fraction in units of batches. */
    +  def setDecayFractionBatches(q: Double): this.type = {
    +    this.a = math.log(1 - q) / math.log(0.5)
    +    this.units = "batches"
    +    this
    +  }
    +
    +  /** Set decay fraction in units of points. Must specify expected number 
of points per batch. */
    +  def setDecayFractionPoints(q: Double, m: Double): this.type = {
    +    this.a = math.pow(math.log(1 - q) / math.log(0.5), 1/m)
    +    this.units = "points"
    +    this
    +  }
    +
    +  /** Specify initial explicitly directly. */
    +  def setInitialCenters(initialCenters: Array[Vector]): this.type = {
    +    val clusterCounts = Array.fill(this.k)(0).map(_.toLong)
    --- End diff --
    
    Actually since the initial value is 0L, you can just do
    ```scala
    new Array[Long](this.k)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to