Skip to content

[SPARK-3218, SPARK-3219, SPARK-3261, SPARK-3424] [RESUBMIT] MLLIB K-Means Clusterer #2634

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
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -94,11 +94,7 @@ object DenseKMeans {
case Parallel => KMeans.K_MEANS_PARALLEL
}

val model = new KMeans()
.setInitializationMode(initMode)
.setK(params.k)
.setMaxIterations(params.numIterations)
.run(examples)
val model = KMeans.train(examples, params.k, params.numIterations, 1, initMode)

val cost = model.computeCost(examples)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -246,14 +246,7 @@ class PythonMLLibAPI extends Serializable {
maxIterations: Int,
runs: Int,
initializationMode: String): KMeansModel = {
val kMeansAlg = new KMeans()
.setK(k)
.setMaxIterations(maxIterations)
.setRuns(runs)
.setInitializationMode(initializationMode)
// Disable the uncached input warning because 'data' is a deliberately uncached MappedRDD.
.disableUncachedWarning()
return kMeansAlg.run(data.rdd)
return KMeans.train(data,k, maxIterations,runs,initializationMode)
}

/**
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
/*
* 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 org.apache.spark.SparkContext._
import org.apache.spark.api.java.JavaRDD
import org.apache.spark.mllib.base.{ PointOps, FP }
import org.apache.spark.mllib.linalg.Vector
import org.apache.spark.rdd.RDD

/**
* A clustering model for K-means. Each point belongs to the cluster with the closest center.
*/
private[mllib] class GeneralizedKMeansModel[P <: FP, C <: FP](
val pointOps: PointOps[P, C],
Copy link
Contributor

Choose a reason for hiding this comment

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

4-space indentation

val centers: Array[C])
extends Serializable {
Copy link
Contributor

Choose a reason for hiding this comment

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

merge this line to 31


val k: Int = clusterCenters.length

def clusterCenters: Array[Vector] = centers.map { c => pointOps.centerToVector(c) }

/** Returns the cluster index that a given point belongs to. */
def predict(point: Vector): Int =
pointOps.findClosest(centers, pointOps.vectorToPoint(point))._1

/** Maps given points to their cluster indices. */
def predict(points: RDD[Vector]): RDD[Int] =
points.map(p => pointOps.findClosest(centers, pointOps.vectorToPoint(p))._1)

/** Maps given points to their cluster indices. */
def predict(points: JavaRDD[Vector]): JavaRDD[java.lang.Integer] =
predict(points.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Integer]]

/**
* Return the K-means cost (sum of squared distances of points to their nearest center) for this
* model on the given data.
*/
def computeCost(data: RDD[Vector]): Double =
data.map(p => pointOps.findClosest(centers, pointOps.vectorToPoint(p))._2).sum()

}
Loading