-
Notifications
You must be signed in to change notification settings - Fork 28.5k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Streaming KMeans [MLLIB][SPARK-3254]
This adds a Streaming KMeans algorithm to MLlib. It uses an update rule that generalizes the mini-batch KMeans update to incorporate a decay factor, which allows past data to be forgotten. The decay factor can be specified explicitly, or via a more intuitive "fractional decay" setting, in units of either data points or batches. The PR includes: - StreamingKMeans algorithm with decay factor settings - Usage example - Additions to documentation clustering page - Unit tests of basic behavior and decay behaviors tdas mengxr rezazadeh Author: freeman <[email protected]> Author: Jeremy Freeman <[email protected]> Author: Xiangrui Meng <[email protected]> Closes #2942 from freeman-lab/streaming-kmeans and squashes the following commits: b2e5b4a [freeman] Fixes to docs / examples 078617c [Jeremy Freeman] Merge pull request #1 from mengxr/SPARK-3254 2e682c0 [Xiangrui Meng] take discount on previous weights; use BLAS; detect dying clusters 0411bf5 [freeman] Change decay parameterization 9f7aea9 [freeman] Style fixes 374a706 [freeman] Formatting ad9bdc2 [freeman] Use labeled points and predictOnValues in examples 77dbd3f [freeman] Make initialization check an assertion 9cfc301 [freeman] Make random seed an argument 44050a9 [freeman] Simpler constructor c7050d5 [freeman] Fix spacing 2899623 [freeman] Use pattern matching for clarity a4a316b [freeman] Use collect 1472ec5 [freeman] Doc formatting ea22ec8 [freeman] Fix imports 2086bdc [freeman] Log cluster center updates ea9877c [freeman] More documentation 9facbe3 [freeman] Bug fix 5db7074 [freeman] Example usage for StreamingKMeans f33684b [freeman] Add explanation and example to docs b5b5f8d [freeman] Add better documentation a0fd790 [freeman] Merge remote-tracking branch 'upstream/master' into streaming-kmeans 9fd9c15 [freeman] Merge remote-tracking branch 'upstream/master' into streaming-kmeans b93350f [freeman] Streaming KMeans with decay
- Loading branch information
1 parent
8602195
commit 98c556e
Showing
4 changed files
with
597 additions
and
1 deletion.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
77 changes: 77 additions & 0 deletions
77
examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeans.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,77 @@ | ||
/* | ||
* 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.examples.mllib | ||
|
||
import org.apache.spark.mllib.linalg.Vectors | ||
import org.apache.spark.mllib.regression.LabeledPoint | ||
import org.apache.spark.mllib.clustering.StreamingKMeans | ||
import org.apache.spark.SparkConf | ||
import org.apache.spark.streaming.{Seconds, StreamingContext} | ||
|
||
/** | ||
* Estimate clusters on one stream of data and make predictions | ||
* on another stream, where the data streams arrive as text files | ||
* into two different directories. | ||
* | ||
* The rows of the training text files must be vector data in the form | ||
* `[x1,x2,x3,...,xn]` | ||
* Where n is the number of dimensions. | ||
* | ||
* The rows of the test text files must be labeled data in the form | ||
* `(y,[x1,x2,x3,...,xn])` | ||
* Where y is some identifier. n must be the same for train and test. | ||
* | ||
* Usage: StreamingKmeans <trainingDir> <testDir> <batchDuration> <numClusters> <numDimensions> | ||
* | ||
* To run on your local machine using the two directories `trainingDir` and `testDir`, | ||
* with updates every 5 seconds, 2 dimensions per data point, and 3 clusters, call: | ||
* $ bin/run-example \ | ||
* org.apache.spark.examples.mllib.StreamingKMeans trainingDir testDir 5 3 2 | ||
* | ||
* As you add text files to `trainingDir` the clusters will continuously update. | ||
* Anytime you add text files to `testDir`, you'll see predicted labels using the current model. | ||
* | ||
*/ | ||
object StreamingKMeans { | ||
|
||
def main(args: Array[String]) { | ||
if (args.length != 5) { | ||
System.err.println( | ||
"Usage: StreamingKMeans " + | ||
"<trainingDir> <testDir> <batchDuration> <numClusters> <numDimensions>") | ||
System.exit(1) | ||
} | ||
|
||
val conf = new SparkConf().setMaster("local").setAppName("StreamingLinearRegression") | ||
val ssc = new StreamingContext(conf, Seconds(args(2).toLong)) | ||
|
||
val trainingData = ssc.textFileStream(args(0)).map(Vectors.parse) | ||
val testData = ssc.textFileStream(args(1)).map(LabeledPoint.parse) | ||
|
||
val model = new StreamingKMeans() | ||
.setK(args(3).toInt) | ||
.setDecayFactor(1.0) | ||
.setRandomCenters(args(4).toInt, 0.0) | ||
|
||
model.trainOn(trainingData) | ||
model.predictOnValues(testData.map(lp => (lp.label, lp.features))).print() | ||
|
||
ssc.start() | ||
ssc.awaitTermination() | ||
} | ||
} |
Oops, something went wrong.