-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-3261] [MLLIB] KMeans clusterer can return duplicate cluster centers #15450
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 5 commits
012979e
85c9857
ebebcb9
793e4d5
d1004d9
79c84ad
f870fe9
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,6 +29,8 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { | |
|
|
||
| import org.apache.spark.mllib.clustering.KMeans.{K_MEANS_PARALLEL, RANDOM} | ||
|
|
||
| private val seed = 42 | ||
|
|
||
| test("single cluster") { | ||
| val data = sc.parallelize(Array( | ||
| Vectors.dense(1.0, 2.0, 6.0), | ||
|
|
@@ -64,30 +66,55 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { | |
| assert(model.clusterCenters.head ~== center absTol 1E-5) | ||
| } | ||
|
|
||
| test("no distinct points") { | ||
| test("fewer distinct points than clusters") { | ||
| val data = sc.parallelize( | ||
| Array( | ||
| Vectors.dense(1.0, 2.0, 3.0), | ||
| Vectors.dense(1.0, 2.0, 3.0), | ||
| Vectors.dense(1.0, 2.0, 3.0)), | ||
| 2) | ||
| val center = Vectors.dense(1.0, 2.0, 3.0) | ||
|
|
||
| // Make sure code runs. | ||
| var model = KMeans.train(data, k = 2, maxIterations = 1) | ||
| assert(model.clusterCenters.size === 2) | ||
| } | ||
| var model = KMeans.train(data, k = 2, maxIterations = 1, initializationMode = "random") | ||
| assert(model.clusterCenters.length === 1) | ||
|
|
||
| test("more clusters than points") { | ||
| val data = sc.parallelize( | ||
| Array( | ||
| Vectors.dense(1.0, 2.0, 3.0), | ||
| Vectors.dense(1.0, 3.0, 4.0)), | ||
| 2) | ||
| model = KMeans.train(data, k = 2, maxIterations = 1, initializationMode = "k-means||") | ||
| assert(model.clusterCenters.length === 1) | ||
| } | ||
|
|
||
| // Make sure code runs. | ||
| var model = KMeans.train(data, k = 3, maxIterations = 1) | ||
| assert(model.clusterCenters.size === 3) | ||
| test("unique cluster centers") { | ||
| val rng = new Random(seed) | ||
| val numDistinctPoints = 10 | ||
| val points = (0 until numDistinctPoints).map(i => Vectors.dense(Array.fill(3)(rng.nextDouble))) | ||
| val data = sc.parallelize(points.flatMap(Array.fill(1 + rng.nextInt(3))(_)), 2) | ||
| val normedData = data.map(new VectorWithNorm(_)) | ||
|
|
||
| // less centers than k | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can we also test the "random" method here? I was going to suggest putting the test cases inside |
||
| val km = new KMeans().setK(50) | ||
| .setMaxIterations(5) | ||
| .setInitializationMode("k-means||") | ||
| .setInitializationSteps(10) | ||
| .setSeed(seed) | ||
| val initialCenters = km.initKMeansParallel(normedData).map(_.vector) | ||
| assert(initialCenters.length === initialCenters.distinct.length) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. also check |
||
| assert(initialCenters.length <= numDistinctPoints) | ||
|
|
||
| val model = km.run(data) | ||
| val finalCenters = model.clusterCenters | ||
| assert(finalCenters.length === finalCenters.distinct.length) | ||
|
|
||
| // run local k-means | ||
| val km2 = new KMeans().setK(10) | ||
| .setMaxIterations(5) | ||
| .setInitializationMode("k-means||") | ||
| .setInitializationSteps(10) | ||
| .setSeed(seed) | ||
| val initialCenters2 = km2.initKMeansParallel(normedData).map(_.vector) | ||
| assert(initialCenters2.length === initialCenters2.distinct.length) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. also check
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This condition failed, though it should be OK. The problem was that the data setup maps each of 10 distinct points to 0-3 copies, meaning that there may be less than 10 distinct points in the end. I just make that 1-3 copies and it works. Fixed the doc problem too, thanks. |
||
| assert(initialCenters2.length === 10) | ||
|
||
|
|
||
| val model2 = km2.run(data) | ||
| val finalCenters2 = model2.clusterCenters | ||
| assert(finalCenters2.length === finalCenters2.distinct.length) | ||
| } | ||
|
|
||
| test("deterministic initialization") { | ||
|
|
@@ -98,11 +125,11 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { | |
| for (initMode <- Seq(RANDOM, K_MEANS_PARALLEL)) { | ||
| // Create three deterministic models and compare cluster means | ||
| val model1 = KMeans.train(rdd, k = 10, maxIterations = 2, runs = 1, | ||
| initializationMode = initMode, seed = 42) | ||
| initializationMode = initMode, seed = seed) | ||
| val centers1 = model1.clusterCenters | ||
|
|
||
| val model2 = KMeans.train(rdd, k = 10, maxIterations = 2, runs = 1, | ||
| initializationMode = initMode, seed = 42) | ||
| initializationMode = initMode, seed = seed) | ||
| val centers2 = model2.clusterCenters | ||
|
|
||
| centers1.zip(centers2).foreach { case (c1, c2) => | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: indentation