Skip to content

[SPARK-16473][MLLIB] Fix BisectingKMeans Algorithm failing in edge case #16355

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

Closed
wants to merge 7 commits into from

Conversation

imatiach-msft
Copy link
Contributor

[SPARK-16473][MLLIB] Fix BisectingKMeans Algorithm failing in edge case where no children exist in updateAssignments

What changes were proposed in this pull request?

Fix a bug in which BisectingKMeans fails with error:
java.util.NoSuchElementException: key not found: 166
at scala.collection.MapLike$class.default(MapLike.scala:228)
at scala.collection.AbstractMap.default(Map.scala:58)
at scala.collection.MapLike$class.apply(MapLike.scala:141)
at scala.collection.AbstractMap.apply(Map.scala:58)
at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1$$anonfun$2.apply$mcDJ$sp(BisectingKMeans.scala:338)
at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1$$anonfun$2.apply(BisectingKMeans.scala:337)
at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1$$anonfun$2.apply(BisectingKMeans.scala:337)
at scala.collection.TraversableOnce$$anonfun$minBy$1.apply(TraversableOnce.scala:231)
at scala.collection.LinearSeqOptimized$class.foldLeft(LinearSeqOptimized.scala:111)
at scala.collection.immutable.List.foldLeft(List.scala:84)
at scala.collection.LinearSeqOptimized$class.reduceLeft(LinearSeqOptimized.scala:125)
at scala.collection.immutable.List.reduceLeft(List.scala:84)
at scala.collection.TraversableOnce$class.minBy(TraversableOnce.scala:231)
at scala.collection.AbstractTraversable.minBy(Traversable.scala:105)
at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1.apply(BisectingKMeans.scala:337)
at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1.apply(BisectingKMeans.scala:334)
at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
at scala.collection.Iterator$$anon$14.hasNext(Iterator.scala:389)

How was this patch tested?

The dataset was run against the code change to verify that the code works. I will try to add unit tests to the code.

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Please review http://spark.apache.org/contributing.html before opening a pull request.

…se where no children exist in updateAssignments
@wangmiao1981
Copy link
Contributor

Jenkins, test this please.

@wangmiao1981
Copy link
Contributor

@imatiach-msft Can you add a test case?

@alokob
Copy link

alokob commented Dec 21, 2016

@imatiach-msft , thanks for creating pull request and committing change which I have shared , I will try to share some sample dataset for this issue.

@@ -339,10 +339,14 @@ private object BisectingKMeans extends Serializable {
assignments.map { case (index, v) =>
if (divisibleIndices.contains(index)) {
val children = Seq(leftChildIndex(index), rightChildIndex(index))
val selected = children.minBy { child =>
KMeans.fastSquaredDistance(newClusterCenters(child), v)
if (children.length > 0) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: you could write children.nonEmpty, but, when would it not have length 2? it's initialized in the line above.

Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I can see following possibility to get java.util.NoSuchElementException in the original code.

leftChildIndex(index) , this methods returns index2 and rightChildIndex(index) returns index2+1. If the Map object newClusterCenters does not have entries belonging to leftChildIndex(index) or rightChildIndex(index) .
Here we need to have this check newClusterCenters.contains(child) , to avoid this issue.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I added a filter on whether new cluster centers contains the children. Would you be able to send me a sample dataset so I can validate the fix? Thank you!

@imatiach-msft
Copy link
Contributor Author

Good point. It looks like we should be checking if the map contains the child or not. However, I'm not sure if that is the correct solution either. I need a repro dataset from the bug reporter.

@srowen
Copy link
Member

srowen commented Dec 21, 2016

That makes more sense as a fix, yes. Sounds like there is still a to-do to verify the fix. If it's possible to write a simple unit test to cover it, all the better.

@imatiach-msft
Copy link
Contributor Author

Yep, there is still a TODO to verify the fix. I'm waiting for the dataset from Alok to reproduce the issue:
https://issues.apache.org/jira/browse/SPARK-16473

@alokob
Copy link

alokob commented Dec 23, 2016

You can get sample vectors at this location https://github.com/alokob/SparkClusteringDataSet/SampleVectors.txt.

Also while executing bisecting K-Means , we have set following configuration settings ,

BisectingKMeans.setK(100);
BisectingKMeans.setMinDivisibleClusterSize(4);
BisectingKMeans.setMaxIter(4);

Please let me know if any additional details are needed.

@alokob
Copy link

alokob commented Dec 26, 2016

@imatiach-msft Did you find the dataset suitable. Is anything else needed from my side?

@imatiach-msft
Copy link
Contributor Author

Hi Alok!
Sorry I was away for holiday break. I will try to reproduce the failure.
Thank you, Ilya

@alokob
Copy link

alokob commented Dec 28, 2016

Thats ok , enjoy Xmas.
Please keep me posted if you find that issue is not resolved.

@imatiach-msft
Copy link
Contributor Author

I have very good news :). I was not only able to repro the issue with your dataset, but I was also able to verify that with the suggested fix the algorithm does not fail (adding the val newClusterChildren = children.filter(newClusterCenters.contains(_)) fixed the issue).
The error I saw was:
Job aborted due to stage failure: Task 0 in stage 53.0 failed 1 times, most recent failure: Lost task 0.0 in stage 53.0 (TID 105, localhost, executor driver): java.util.NoSuchElementException: key not found: 162

I need to figure out how to actually add the test case to spark though - my understanding is that checking in dataset files is not allowed? My test code (not cleaned up yet) was:

import org.apache.spark.mllib.linalg.{
Vector => OldVector,
Vectors => OldVectors,
DenseVector => OldDenseVector,
SparseVector => OldSparseVector}

import org.apache.spark.ml.linalg.{
Vector => NewVector,
Vectors => NewVectors,
DenseVector => NewDenseVector,
SparseVector => NewSparseVector
}

@transient var loadedDataset: Dataset[_] = _

override def beforeAll(): Unit = {
super.beforeAll()
dataset = KMeansSuite.generateKMeansData(spark, 50, 3, k)
loadedDataset = spark.createDataFrame(sc.textFile("/home/ilya/SampleVectors.txt")
.map(line => Row({ OldVectors.parse(line.substring(1, line.length - 1)) match {
case sv: OldSparseVector => NewVectors.sparse(sv.size, sv.indices, sv.values)
case dv: OldDenseVector => NewVectors.dense(dv.values)
} })),
StructType(Seq(StructField("features", SQLDataTypes.VectorType))))
}
.....
test("SPARK-16473: Verify Bisecting K-Means does not fail in edge case where no children exist") {
val bkm = new BisectingKMeans().setK(100).setMinDivisibleClusterSize(4).setMaxIter(4)

assert(bkm.getK === 100)
assert(bkm.getFeaturesCol === "features")
assert(bkm.getPredictionCol === "prediction")
assert(bkm.getMaxIter === 4)
assert(bkm.getMinDivisibleClusterSize === 4)
val model = bkm.fit(loadedDataset)
assert(model.hasSummary)

}

@jkbradley
Copy link
Member

ok to test

@alokob
Copy link

alokob commented Dec 28, 2016

Nice to know that , codefix I suggested is working. Its really nice to contribute in spark.

@SparkQA
Copy link

SparkQA commented Dec 28, 2016

Test build #70679 has finished for PR 16355 at commit 55ab179.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

…c sparse data which can generate the exception user encountered.
@imatiach-msft
Copy link
Contributor Author

I've updated with a new commit. I was able to reproduce the issue by generating a synthetic sparse dataset similar to the one Alok sent me, in accordance with the test-style of spark test methods. Hence, I was able to verify the fix. Please review the new code changes and let me know who else needs to review in order for the changes to be committed.

@imatiach-msft
Copy link
Contributor Author

Jenkins, retest this please

@SparkQA
Copy link

SparkQA commented Dec 28, 2016

Test build #70682 has finished for PR 16355 at commit 75192a7.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@imatiach-msft
Copy link
Contributor Author

Jenkins, retest this please

@SparkQA
Copy link

SparkQA commented Dec 29, 2016

Test build #70688 has finished for PR 16355 at commit 14c7ce3.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@imatiach-msft
Copy link
Contributor Author

@jkbradley @srowen any comments on the changes? Thank you!

@jkbradley
Copy link
Member

@yu-iskw Pinging on this since you wrote bisecting k-means originally. Do you have time to take a look? Thanks!

@imatiach-msft
Copy link
Contributor Author

imatiach-msft commented Dec 29, 2016

the only problem I see is that with this code we generate k-1 clusters instead of k, but it states in the algorithm documentation that it is not guaranteed to generate k clusters, it could be fewer if the leaf clusters are not divisible (see spark/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala):

Iteratively it finds divisible clusters on the bottom level and bisects each of them using
k-means, until there are k leaf clusters in total or no leaf clusters are divisible.

It seems in the dataset Alok gave, one of the clusters which was assumed to be divisible and was divided ended up generating two clusters, one which contained all the points and the other none, which is what created the error (his cluster 162, child of 81, was empty, but cluster 163 was non-empty after reassignment).

My intuition is that it should be possible for the algorithm to account for failed splits and then try to split in a different leaf node, but that change seems non-trivial to me.

Copy link
Member

@jkbradley jkbradley left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the updates! I agree we shouldn't try to retry failed splits in this PR.

override def beforeAll(): Unit = {
super.beforeAll()
dataset = KMeansSuite.generateKMeansData(spark, 50, 3, k)
sparseDataset = KMeansSuite.generateSparseData(spark, 100, 1000, k, 42)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does the test really need to be this large? It takes ~1 sec which is long-ish for a unit test.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I can only repro the issue with very sparse data where number of columns is around 1k. I was able to reduce the number of rows to be only 10 however. I hope that is a small enough dataset.

val bkm = new BisectingKMeans().setK(k).setMinDivisibleClusterSize(4).setMaxIter(4)

assert(bkm.getK === k)
assert(bkm.getFeaturesCol === "features")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There's no need to test featuresCol, predictionCol, and other things which aren't relevant to this unit test. I'd simplify it.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done, removed

assert(bkm.getPredictionCol === "prediction")
assert(bkm.getMaxIter === 4)
assert(bkm.getMinDivisibleClusterSize === 4)
// Verify fit does not fail on very sparse data
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's not clear to me that this unit test actually tests the issue fixed in this PR. Is there a good way to see why it would? If not, then it would be great to write a tiny dataset by hand which would trigger the failure.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I added this check to verify:
// Verify we hit the edge case
assert(numClusters < k && numClusters > 1)
the issue only occurs for very sparse data, but it occurs very consistently (almost all very sparse data that I generate can trigger the error)

@@ -160,6 +162,17 @@ object KMeansSuite {
spark.createDataFrame(rdd)
}

def generateSparseData(spark: SparkSession, rows: Int, dim: Int, k: Int, seed: Int): DataFrame = {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

k is never used

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

modified the method to use it

Copy link
Member

@jkbradley jkbradley left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just 2 small comments left. Thanks!

"one cluster is empty after split") {
val bkm = new BisectingKMeans().setK(k).setMinDivisibleClusterSize(4).setMaxIter(4)

assert(bkm.getK === k)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Don't bother testing getK, getMaxIter, or getMinDivisibleClusterSize
The setters should be tested elsewhere, so you may assume they work here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done, removed

val nnz = random.nextInt(dim)
val rdd = sc.parallelize(1 to rows)
.map(i => Vectors.sparse(dim, random.shuffle(0 to dim - 1).slice(0, nnz).sorted.toArray,
Array.fill(nnz)(random.nextInt(k).toDouble)))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't understand this use of k. The feature value can be any random number. I'd remove k.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done, removed k

@imatiach-msft
Copy link
Contributor Author

@jkbradley thanks, I've updated the code based on your latest comments - I removed k and the verification for the setters.

@SparkQA
Copy link

SparkQA commented Jan 13, 2017

Test build #71289 has finished for PR 16355 at commit 138ab34.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@imatiach-msft
Copy link
Contributor Author

ping @jkbradley would you be able to take another look at the bisecting K-Means fix?

@jkbradley
Copy link
Member

test this please

@@ -51,6 +54,18 @@ class BisectingKMeansSuite
assert(copiedModel.hasSummary)
}

test("SPARK-16473: Verify Bisecting K-Means does not fail in edge case where" +
"one cluster is empty after split") {
val bkm = new BisectingKMeans().setK(k).setMinDivisibleClusterSize(4).setMaxIter(4)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please set the seed too since this test seems at risk of flakiness

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done, added seed

@jkbradley
Copy link
Member

I was about to say this is ready, but I do think we should add the seed. Other than that, this should be ready!

@imatiach-msft
Copy link
Contributor Author

@jkbradley done, added seed. Thanks!

@SparkQA
Copy link

SparkQA commented Jan 17, 2017

Test build #71533 has finished for PR 16355 at commit 138ab34.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 18, 2017

Test build #71538 has finished for PR 16355 at commit ba00ad0.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jan 18, 2017

Test build #3538 has finished for PR 16355 at commit ba00ad0.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@imatiach-msft
Copy link
Contributor Author

ping @jkbradley would you be able to take another look at the bisecting kmeans model? I've updated with the random seed as requested.

@imatiach-msft
Copy link
Contributor Author

ping @jkbradley would you be able to take another look at the bisecting kmeans model? I've updated with the random seed as requested, and the build succeeded. Thank you!

@imatiach-msft
Copy link
Contributor Author

ping @jkbradley would you be able to take another look at the bisecting kmeans model? Thanks!

@imatiach-msft
Copy link
Contributor Author

ping @jkbradley would you be able to take another look at the bisecting kmeans model? Thanks!

@jkbradley
Copy link
Member

LGTM
Thanks!
Will merge after fresh tests

@SparkQA
Copy link

SparkQA commented Jan 23, 2017

Test build #3548 has finished for PR 16355 at commit ba00ad0.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@jkbradley
Copy link
Member

Merging with master. Will try to backport to branch-2.1 as well.
Thanks!

@asfgit asfgit closed this in 5b258b8 Jan 23, 2017
@jkbradley
Copy link
Member

I was able to check out this commit and test it with branch-2.1, but now I can't get the merge script to merge it for branch-2.1. @srowen would you mind trying? Thanks!

@vanzin
Copy link
Contributor

vanzin commented Jan 23, 2017

but now I can't get the merge script to merge it for branch-2.1

I just had some issues with that too. But manually merging (git cherry-pick + git push) seems to still work, so maybe try that.

@srowen
Copy link
Member

srowen commented Jan 24, 2017

It's an apache-github sync issue:

https://github.com/apache/spark/commits/branch-2.1
is missing the latest commit from
https://git-wip-us.apache.org/repos/asf?p=spark.git;a=shortlog;h=refs/heads/branch-2.1

I'll cherry-pick onto apache/branch-2.1 and push as that might also kick the sync to try again.

asfgit pushed a commit that referenced this pull request Jan 24, 2017
[SPARK-16473][MLLIB] Fix BisectingKMeans Algorithm failing in edge case where no children exist in updateAssignments

## What changes were proposed in this pull request?

Fix a bug in which BisectingKMeans fails with error:
java.util.NoSuchElementException: key not found: 166
        at scala.collection.MapLike$class.default(MapLike.scala:228)
        at scala.collection.AbstractMap.default(Map.scala:58)
        at scala.collection.MapLike$class.apply(MapLike.scala:141)
        at scala.collection.AbstractMap.apply(Map.scala:58)
        at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1$$anonfun$2.apply$mcDJ$sp(BisectingKMeans.scala:338)
        at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1$$anonfun$2.apply(BisectingKMeans.scala:337)
        at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1$$anonfun$2.apply(BisectingKMeans.scala:337)
        at scala.collection.TraversableOnce$$anonfun$minBy$1.apply(TraversableOnce.scala:231)
        at scala.collection.LinearSeqOptimized$class.foldLeft(LinearSeqOptimized.scala:111)
        at scala.collection.immutable.List.foldLeft(List.scala:84)
        at scala.collection.LinearSeqOptimized$class.reduceLeft(LinearSeqOptimized.scala:125)
        at scala.collection.immutable.List.reduceLeft(List.scala:84)
        at scala.collection.TraversableOnce$class.minBy(TraversableOnce.scala:231)
        at scala.collection.AbstractTraversable.minBy(Traversable.scala:105)
        at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1.apply(BisectingKMeans.scala:337)
        at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1.apply(BisectingKMeans.scala:334)
        at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
        at scala.collection.Iterator$$anon$14.hasNext(Iterator.scala:389)

## How was this patch tested?

The dataset was run against the code change to verify that the code works.  I will try to add unit tests to the code.

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Ilya Matiach <ilmat@microsoft.com>

Closes #16355 from imatiach-msft/ilmat/fix-kmeans.
@srowen
Copy link
Member

srowen commented Jan 24, 2017

Done, and it synced now. Merged to master/2.1

@jkbradley
Copy link
Member

Oh OK! Thanks @srowen

uzadude pushed a commit to uzadude/spark that referenced this pull request Jan 27, 2017
[SPARK-16473][MLLIB] Fix BisectingKMeans Algorithm failing in edge case where no children exist in updateAssignments

## What changes were proposed in this pull request?

Fix a bug in which BisectingKMeans fails with error:
java.util.NoSuchElementException: key not found: 166
        at scala.collection.MapLike$class.default(MapLike.scala:228)
        at scala.collection.AbstractMap.default(Map.scala:58)
        at scala.collection.MapLike$class.apply(MapLike.scala:141)
        at scala.collection.AbstractMap.apply(Map.scala:58)
        at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1$$anonfun$2.apply$mcDJ$sp(BisectingKMeans.scala:338)
        at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1$$anonfun$2.apply(BisectingKMeans.scala:337)
        at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1$$anonfun$2.apply(BisectingKMeans.scala:337)
        at scala.collection.TraversableOnce$$anonfun$minBy$1.apply(TraversableOnce.scala:231)
        at scala.collection.LinearSeqOptimized$class.foldLeft(LinearSeqOptimized.scala:111)
        at scala.collection.immutable.List.foldLeft(List.scala:84)
        at scala.collection.LinearSeqOptimized$class.reduceLeft(LinearSeqOptimized.scala:125)
        at scala.collection.immutable.List.reduceLeft(List.scala:84)
        at scala.collection.TraversableOnce$class.minBy(TraversableOnce.scala:231)
        at scala.collection.AbstractTraversable.minBy(Traversable.scala:105)
        at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1.apply(BisectingKMeans.scala:337)
        at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1.apply(BisectingKMeans.scala:334)
        at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
        at scala.collection.Iterator$$anon$14.hasNext(Iterator.scala:389)

## How was this patch tested?

The dataset was run against the code change to verify that the code works.  I will try to add unit tests to the code.

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Ilya Matiach <ilmat@microsoft.com>

Closes apache#16355 from imatiach-msft/ilmat/fix-kmeans.
cmonkey pushed a commit to cmonkey/spark that referenced this pull request Feb 15, 2017
[SPARK-16473][MLLIB] Fix BisectingKMeans Algorithm failing in edge case where no children exist in updateAssignments

## What changes were proposed in this pull request?

Fix a bug in which BisectingKMeans fails with error:
java.util.NoSuchElementException: key not found: 166
        at scala.collection.MapLike$class.default(MapLike.scala:228)
        at scala.collection.AbstractMap.default(Map.scala:58)
        at scala.collection.MapLike$class.apply(MapLike.scala:141)
        at scala.collection.AbstractMap.apply(Map.scala:58)
        at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1$$anonfun$2.apply$mcDJ$sp(BisectingKMeans.scala:338)
        at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1$$anonfun$2.apply(BisectingKMeans.scala:337)
        at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1$$anonfun$2.apply(BisectingKMeans.scala:337)
        at scala.collection.TraversableOnce$$anonfun$minBy$1.apply(TraversableOnce.scala:231)
        at scala.collection.LinearSeqOptimized$class.foldLeft(LinearSeqOptimized.scala:111)
        at scala.collection.immutable.List.foldLeft(List.scala:84)
        at scala.collection.LinearSeqOptimized$class.reduceLeft(LinearSeqOptimized.scala:125)
        at scala.collection.immutable.List.reduceLeft(List.scala:84)
        at scala.collection.TraversableOnce$class.minBy(TraversableOnce.scala:231)
        at scala.collection.AbstractTraversable.minBy(Traversable.scala:105)
        at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1.apply(BisectingKMeans.scala:337)
        at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1.apply(BisectingKMeans.scala:334)
        at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
        at scala.collection.Iterator$$anon$14.hasNext(Iterator.scala:389)

## How was this patch tested?

The dataset was run against the code change to verify that the code works.  I will try to add unit tests to the code.

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Ilya Matiach <ilmat@microsoft.com>

Closes apache#16355 from imatiach-msft/ilmat/fix-kmeans.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

9 participants