From 07ffaf2b7f3300a6c0afc2a21a0134c76d3ec8dc Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 10 Mar 2014 14:27:46 -0700 Subject: [PATCH 01/24] add dense/sparse vector data models and conversions to/from breeze vectors use breeze to implement KMeans in order to support both dense and sparse data --- NOTICE | 15 ++ mllib/pom.xml | 5 + .../spark/mllib/clustering/KMeans.scala | 121 ++++++++++++---- .../spark/mllib/clustering/KMeansModel.scala | 20 +++ .../spark/mllib/clustering/LocalKMeans.scala | 43 ++++-- .../apache/spark/mllib/linalg/Vectors.scala | 136 ++++++++++++++++++ .../spark/mllib/clustering/KMeansSuite.scala | 42 +++++- .../linalg/BreezeVectorConversionSuite.scala | 58 ++++++++ .../spark/mllib/linalg/VectorSuite.scala | 48 +++++++ project/SparkBuild.scala | 3 +- 10 files changed, 447 insertions(+), 44 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorSuite.scala diff --git a/NOTICE b/NOTICE index 7cbb114b2ae2d..33e19418f4e56 100644 --- a/NOTICE +++ b/NOTICE @@ -3,3 +3,18 @@ Copyright 2013 The Apache Software Foundation. This product includes software developed at The Apache Software Foundation (http://www.apache.org/). + +Numerical linear algebra support in MLlib is provided by the breeze package, +which depends on the following packages that are not distributed under +Apache authorized licenses: + +- netlib-core, which is open source software written by Samuel Halliday, + and copyright by the University of Tennessee, the University of Tennessee + Research Foundation, the University of California at Berkeley, and the + University of Colorado at Denver. The original software is available from + https://github.com/fommil/netlib-java + +- JTransforms, which is open source software written by Piotr Wendykier, + and distributed under the the terms of the MPL/LGPL/GPL tri-license. + The original software is available from + https://sites.google.com/site/piotrwendykier/software/jtransforms diff --git a/mllib/pom.xml b/mllib/pom.xml index 760a2a85d5ffa..44dfb7dfb11c1 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -60,6 +60,11 @@ jblas 1.2.3 + + org.scalanlp + breeze_${scala.binary.version} + 0.7-SNAPSHOT + org.scalatest scalatest_${scala.binary.version} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index e508b76c3f8c5..a6ecf64922713 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -19,16 +19,15 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable.ArrayBuffer -import org.jblas.DoubleMatrix +import breeze.linalg.{DenseVector => BDV, Vector => BV, squaredDistance => breezeSquaredDistance} -import org.apache.spark.SparkContext +import org.apache.spark.{Logging, SparkContext} import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD -import org.apache.spark.Logging +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom - /** * K-means clustering with support for multiple parallel runs and a k-means++ like initialization * mode (the k-means|| algorithm by Bahmani et al). When multiple concurrent runs are requested, @@ -46,8 +45,6 @@ class KMeans private ( var epsilon: Double) extends Serializable with Logging { - private type ClusterCenters = Array[Array[Double]] - def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 5, 1e-4) /** Set the number of clusters to create (k). Default: 2. */ @@ -114,6 +111,23 @@ class KMeans private ( * performance, because this is an iterative algorithm. */ def run(data: RDD[Array[Double]]): KMeansModel = { + val breezeData = data.map(v => new BDV[Double](v).asInstanceOf[BV[Double]]) + runBreeze(breezeData) + } + + /** + * Train a K-means model on the given set of points; `data` should be cached for high + * performance, because this is an iterative algorithm. + */ + def run(data: RDD[Vector])(implicit d: DummyImplicit): KMeansModel = { + val breezeData = data.map(v => v.toBreeze) + runBreeze(breezeData) + } + + /** + * Implementation using Breeze. + */ + private def runBreeze(data: RDD[BV[Double]]): KMeansModel = { // TODO: check whether data is persistent; this needs RDD.storageLevel to be publicly readable val sc = data.sparkContext @@ -132,9 +146,9 @@ class KMeans private ( // Execute iterations of Lloyd's algorithm until all runs have converged while (iteration < maxIterations && !activeRuns.isEmpty) { - type WeightedPoint = (DoubleMatrix, Long) + type WeightedPoint = (BDV[Double], Long) def mergeContribs(p1: WeightedPoint, p2: WeightedPoint): WeightedPoint = { - (p1._1.addi(p2._1), p1._2 + p2._2) + (p1._1 += p2._1, p1._2 + p2._2) } val activeCenters = activeRuns.map(r => centers(r)).toArray @@ -146,13 +160,13 @@ class KMeans private ( val k = activeCenters(0).length val dims = activeCenters(0)(0).length - val sums = Array.fill(runs, k)(new DoubleMatrix(dims)) + val sums = Array.fill(runs, k)(BDV.zeros[Double](dims)) val counts = Array.fill(runs, k)(0L) for (point <- points; (centers, runIndex) <- activeCenters.zipWithIndex) { val (bestCenter, cost) = KMeans.findClosest(centers, point) costAccums(runIndex) += cost - sums(runIndex)(bestCenter).addi(new DoubleMatrix(point)) + sums(runIndex)(bestCenter) += point counts(runIndex)(bestCenter) += 1 } @@ -168,8 +182,9 @@ class KMeans private ( for (j <- 0 until k) { val (sum, count) = totalContribs((i, j)) if (count != 0) { - val newCenter = sum.divi(count).data - if (MLUtils.squaredDistance(newCenter, centers(run)(j)) > epsilon * epsilon) { + sum /= count.toDouble + val newCenter = sum + if (breezeSquaredDistance(newCenter, centers(run)(j)) > epsilon * epsilon) { changed = true } centers(run)(j) = newCenter @@ -187,16 +202,20 @@ class KMeans private ( } val bestRun = costs.zipWithIndex.min._2 - new KMeansModel(centers(bestRun)) + new KMeansModel(centers(bestRun).map { v => + v.toArray + }) } /** * Initialize `runs` sets of cluster centers at random. */ - private def initRandom(data: RDD[Array[Double]]): Array[ClusterCenters] = { + private def initRandom(data: RDD[BV[Double]]): Array[Array[BV[Double]]] = { // Sample all the cluster centers in one pass to avoid repeated scans val sample = data.takeSample(true, runs * k, new XORShiftRandom().nextInt()).toSeq - Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k).toArray) + Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k).map { v => + v.toDenseVector + }.toArray) } /** @@ -208,41 +227,39 @@ class KMeans private ( * * The original paper can be found at http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf. */ - private def initKMeansParallel(data: RDD[Array[Double]]): Array[ClusterCenters] = { + private def initKMeansParallel(data: RDD[BV[Double]]): Array[Array[BV[Double]]] = { // Initialize each run's center to a random point val seed = new XORShiftRandom().nextInt() val sample = data.takeSample(true, runs, seed).toSeq - val centers = Array.tabulate(runs)(r => ArrayBuffer(sample(r))) + val centers = Array.tabulate(runs)(r => ArrayBuffer(sample(r).toDenseVector)) // On each step, sample 2 * k points on average for each run with probability proportional // to their squared distance from that run's current centers for (step <- 0 until initializationSteps) { - val centerArrays = centers.map(_.toArray) val sumCosts = data.flatMap { point => - for (r <- 0 until runs) yield (r, KMeans.pointCost(centerArrays(r), point)) + for (r <- 0 until runs) yield (r, KMeans.pointCost(centers(r), point)) }.reduceByKey(_ + _).collectAsMap() val chosen = data.mapPartitionsWithIndex { (index, points) => val rand = new XORShiftRandom(seed ^ (step << 16) ^ index) for { p <- points r <- 0 until runs - if rand.nextDouble() < KMeans.pointCost(centerArrays(r), p) * 2 * k / sumCosts(r) + if rand.nextDouble() < KMeans.pointCost(centers(r), p) * 2 * k / sumCosts(r) } yield (r, p) }.collect() for ((r, p) <- chosen) { - centers(r) += p + centers(r) += p.toDenseVector } } // Finally, we might have a set of more than k candidate centers for each run; weigh each // candidate by the number of points in the dataset mapping to it and run a local k-means++ // on the weighted centers to pick just k of them - val centerArrays = centers.map(_.toArray) val weightMap = data.flatMap { p => - for (r <- 0 until runs) yield ((r, KMeans.findClosest(centerArrays(r), p)._1), 1.0) + for (r <- 0 until runs) yield ((r, KMeans.findClosest(centers(r), p)._1), 1.0) }.reduceByKey(_ + _).collectAsMap() val finalCenters = (0 until runs).map { r => - val myCenters = centers(r).toArray + val myCenters = centers(r).toArray.asInstanceOf[Array[BV[Double]]] val myWeights = (0 until myCenters.length).map(i => weightMap.getOrElse((r, i), 0.0)).toArray LocalKMeans.kMeansPlusPlus(r, myCenters, myWeights, k, 30) } @@ -256,6 +273,7 @@ class KMeans private ( * Top-level methods for calling K-means clustering. */ object KMeans { + // Initialization mode names val RANDOM = "random" val K_MEANS_PARALLEL = "k-means||" @@ -268,6 +286,28 @@ object KMeans { initializationMode: String) : KMeansModel = { + new KMeans().setK(k) + .setMaxIterations(maxIterations) + .setRuns(runs) + .setInitializationMode(initializationMode) + .run(data) + } + + def train(data: RDD[Array[Double]], k: Int, maxIterations: Int, runs: Int): KMeansModel = { + train(data, k, maxIterations, runs, K_MEANS_PARALLEL) + } + + def train(data: RDD[Array[Double]], k: Int, maxIterations: Int): KMeansModel = { + train(data, k, maxIterations, 1, K_MEANS_PARALLEL) + } + + def train( + data: RDD[Vector], + k: Int, + maxIterations: Int, + runs: Int, + initializationMode: String + )(implicit d: DummyImplicit): KMeansModel = { new KMeans().setK(k) .setMaxIterations(maxIterations) .setRuns(runs) @@ -275,11 +315,13 @@ object KMeans { .run(data) } - def train(data: RDD[Array[Double]], k: Int, maxIterations: Int, runs: Int): KMeansModel = { + def train(data: RDD[Vector], k: Int, maxIterations: Int, runs: Int) + (implicit d: DummyImplicit): KMeansModel = { train(data, k, maxIterations, runs, K_MEANS_PARALLEL) } - def train(data: RDD[Array[Double]], k: Int, maxIterations: Int): KMeansModel = { + def train(data: RDD[Vector], k: Int, maxIterations: Int) + (implicit d: DummyImplicit): KMeansModel = { train(data, k, maxIterations, 1, K_MEANS_PARALLEL) } @@ -301,6 +343,25 @@ object KMeans { (bestIndex, bestDistance) } + /** + * Returns the index of the closest center to the given point, as well as the squared distance. + */ + private[mllib] def findClosest(centers: TraversableOnce[BV[Double]], point: BV[Double]) + : (Int, Double) = { + var bestDistance = Double.PositiveInfinity + var bestIndex = 0 + var i = 0 + centers.foreach { v => + val distance: Double = breezeSquaredDistance(v, point) + if (distance < bestDistance) { + bestDistance = distance + bestIndex = i + } + i += 1 + } + (bestIndex, bestDistance) + } + /** * Return the K-means cost of a given point against the given cluster centers. */ @@ -315,6 +376,12 @@ object KMeans { bestDistance } + /** + * Returns the K-means cost of a given point against the given cluster centers. + */ + private[mllib] def pointCost(centers: TraversableOnce[BV[Double]], point: BV[Double]): Double = + findClosest(centers, point)._2 + def main(args: Array[String]) { if (args.length < 4) { println("Usage: KMeans []") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index 980be931576dc..06cf2e3ceb36a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -17,13 +17,21 @@ package org.apache.spark.mllib.clustering +import breeze.linalg.{DenseVector => BreezeDenseVector} + import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.Vector /** * A clustering model for K-means. Each point belongs to the cluster with the closest center. */ class KMeansModel(val clusterCenters: Array[Array[Double]]) extends Serializable { + + private val breezeClusterCenters = clusterCenters.map { v => + new BreezeDenseVector[Double](v) + } + /** Total number of clusters. */ def k: Int = clusterCenters.length @@ -32,6 +40,10 @@ class KMeansModel(val clusterCenters: Array[Array[Double]]) extends Serializable KMeans.findClosest(clusterCenters, point)._1 } + def predict(point: Vector): Int = { + KMeans.findClosest(breezeClusterCenters, point.toBreeze)._1 + } + /** * Return the K-means cost (sum of squared distances of points to their nearest center) for this * model on the given data. @@ -39,4 +51,12 @@ class KMeansModel(val clusterCenters: Array[Array[Double]]) extends Serializable def computeCost(data: RDD[Array[Double]]): Double = { data.map(p => KMeans.pointCost(clusterCenters, p)).sum() } + + /** + * 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])(implicit d: DummyImplicit): Double = { + data.map(p => KMeans.pointCost(breezeClusterCenters, p.toBreeze)).sum() + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala index baf8251d8fc53..9226538fac3ad 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala @@ -19,31 +19,43 @@ package org.apache.spark.mllib.clustering import scala.util.Random -import org.jblas.{DoubleMatrix, SimpleBlas} +import breeze.linalg.{Vector => BV, DenseVector => BDV} /** * An utility object to run K-means locally. This is private to the ML package because it's used * in the initialization of KMeans but not meant to be publicly exposed. */ private[mllib] object LocalKMeans { + + def kMeansPlusPlus( + seed: Int, + points: Array[Array[Double]], + weights: Array[Double], + k: Int, + maxIterations: Int + ): Array[Array[Double]] = { + val breezePoints = points.map(v => new BDV[Double](v).asInstanceOf[BV[Double]]) + val breezeCenters = kMeansPlusPlus(seed, breezePoints, weights, k, maxIterations) + breezeCenters.map(_.toArray) + } + /** * Run K-means++ on the weighted point set `points`. This first does the K-means++ - * initialization procedure and then roudns of Lloyd's algorithm. + * initialization procedure and then rounds of Lloyd's algorithm. */ def kMeansPlusPlus( seed: Int, - points: Array[Array[Double]], + points: Array[BV[Double]], weights: Array[Double], k: Int, - maxIterations: Int) - : Array[Array[Double]] = - { + maxIterations: Int + )(implicit d: DummyImplicit): Array[BV[Double]] = { val rand = new Random(seed) val dimensions = points(0).length - val centers = new Array[Array[Double]](k) + val centers = new Array[BV[Double]](k) - // Initialize centers by sampling using the k-means++ procedure - centers(0) = pickWeighted(rand, points, weights) + // Initialize centers by sampling using the k-means++ procedure. + centers(0) = (pickWeighted(rand, points, weights)).toDenseVector for (i <- 1 until k) { // Pick the next center with a probability proportional to cost under current centers val curCenters = centers.slice(0, i) @@ -57,7 +69,7 @@ private[mllib] object LocalKMeans { cumulativeScore += weights(j) * KMeans.pointCost(curCenters, points(j)) j += 1 } - centers(i) = points(j-1) + centers(i) = points(j-1).toDenseVector } // Run up to maxIterations iterations of Lloyd's algorithm @@ -66,11 +78,13 @@ private[mllib] object LocalKMeans { var moved = true while (moved && iteration < maxIterations) { moved = false - val sums = Array.fill(k)(new DoubleMatrix(dimensions)) + val sums = Array.fill(k)( + new BDV[Double](new Array[Double](dimensions)).asInstanceOf[BV[Double]] + ) val counts = Array.fill(k)(0.0) for ((p, i) <- points.zipWithIndex) { val index = KMeans.findClosest(centers, p)._1 - SimpleBlas.axpy(weights(i), new DoubleMatrix(p), sums(index)) + breeze.linalg.axpy(weights(i), p, sums(index)) counts(index) += weights(i) if (index != oldClosest(i)) { moved = true @@ -81,9 +95,10 @@ private[mllib] object LocalKMeans { for (i <- 0 until k) { if (counts(i) == 0.0) { // Assign center to a random point - centers(i) = points(rand.nextInt(points.length)) + centers(i) = points(rand.nextInt(points.length)).toDenseVector } else { - centers(i) = sums(i).divi(counts(i)).data + sums(i) /= counts(i) + centers(i) = sums(i) } } iteration += 1 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala new file mode 100644 index 0000000000000..b95889f9a44ff --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -0,0 +1,136 @@ +/* + * 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.linalg + +import breeze.linalg.{Vector => BreezeVector, DenseVector => BreezeDenseVector, + SparseVector => BreezeSparseVector} + +/** + * Represents a numeric vector, whose index type is Int and value type is Double. + */ +trait Vector extends Serializable { + + /** + * Size of the vector. + */ + def size: Int + + /** + * Converts the instance to a Mahout vector wrapper. + */ + private[mllib] def toBreeze: BreezeVector[Double] +} + +/** + * Represents a vector with random access to its elements. + * + */ +trait RandomAccessVector extends Vector { + // empty +} + +/** + * Factory methods for [[org.apache.spark.mllib.linalg.Vector]]. + */ +object Vectors { + + /** Creates a dense vector. */ + def dense(values: Array[Double]): Vector = new DenseVector(values) + + /** + * Creates a sparse vector providing its index array and value array. + * + * @param size vector size. + * @param indices index array, must be strictly increasing. + * @param values value array, must have the same length as indices. + */ + def sparse(size: Int, indices: Array[Int], values: Array[Double]): Vector = + new SparseVector(size, indices, values) + + /** + * Creates a sparse vector using unordered (index, value) pairs. + * + * @param size vector size. + * @param elements vector elements in (index, value) pairs. + */ + def sparse(size: Int, elements: Iterable[(Int, Double)]): Vector = { + + require(size > 0) + + val (indices, values) = elements.toArray.sortBy(_._1).unzip + var prev = -1 + indices.foreach { i => + require(prev < i, "Found duplicate indices: " + i) + prev = i + } + require(prev < size) + + new SparseVector(size, indices.toArray, values.toArray) + } + + /** + * Creates a vector instance from a breeze vector. + */ + private[mllib] def fromBreeze(breezeVector: BreezeVector[Double]): Vector = { + breezeVector match { + case v: BreezeDenseVector[Double] => { + require(v.offset == 0) + require(v.stride == 1) + new DenseVector(v.data) + } + case v: BreezeSparseVector[Double] => { + new SparseVector(v.length, v.index, v.data) + } + case v: BreezeVector[_] => { + sys.error("Unsupported Breeze vector type: " + v.getClass.getName) + } + } + } +} + +/** + * A dense vector represented by a value array. + * + * @param values + */ +class DenseVector(var values: Array[Double]) extends RandomAccessVector { + + override def size: Int = values.length + + override def toString = values.mkString("[", ",", "]") + + private[mllib] override def toBreeze = new BreezeDenseVector[Double](values) +} + +/** + * A sparse vector represented by an index array and an value array. + * + * @param n size of the vector. + * @param indices index array, assume to be strictly increasing. + * @param values value array, must have the same length as the index array. + */ +class SparseVector(var n: Int, var indices: Array[Int], var values: Array[Double]) extends Vector { + + override def size: Int = n + + override def toString = { + "(" + n + "," + indices.zip(values).mkString("[", "," ,"]") + ")" + } + + private[mllib] override def toBreeze = new BreezeSparseVector[Double](indices, values, n) +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index 4ef1d1f64ff06..cabadd15731cb 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -17,11 +17,10 @@ package org.apache.spark.mllib.clustering - -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.linalg.{Vectors, Vector} class KMeansSuite extends FunSuite with LocalSparkContext { @@ -131,6 +130,45 @@ class KMeansSuite extends FunSuite with LocalSparkContext { assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) } + test("single cluster with sparse data") { + val n = 1000 + val smallData = Array( + Vectors.sparse(n, Seq((0, 1.0), (1, 2.0), (2, 6.0))), + Vectors.sparse(n, Seq((0, 1.0), (1, 3.0))), + Vectors.sparse(n, Seq((0, 1.0), (1, 4.0), (2, 6.0))) + ) + val data = sc.parallelize((1 to 100).flatMap(_ => smallData), 4) + + // No matter how many runs or iterations we use, we should get one cluster, + // centered at the mean of the points + + val center = new Array[Double](n) + center(0) = 1.0 + center(1) = 3.0 + center(2) = 4.0 + + var model = KMeans.train(data, k=1, maxIterations=1) + assertSetsEqual(model.clusterCenters, Array(center)) + + model = KMeans.train(data, k=1, maxIterations=2) + assertSetsEqual(model.clusterCenters, Array(center)) + + model = KMeans.train(data, k=1, maxIterations=5) + assertSetsEqual(model.clusterCenters, Array(center)) + + model = KMeans.train(data, k=1, maxIterations=1, runs=5) + assertSetsEqual(model.clusterCenters, Array(center)) + + model = KMeans.train(data, k=1, maxIterations=1, runs=5) + assertSetsEqual(model.clusterCenters, Array(center)) + + model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) + assertSetsEqual(model.clusterCenters, Array(center)) + + model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) + assertSetsEqual(model.clusterCenters, Array(center)) + } + test("k-means|| initialization") { val points = Array( Array(1.0, 2.0, 6.0), diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala new file mode 100644 index 0000000000000..aacaa300849aa --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala @@ -0,0 +1,58 @@ +/* + * 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.linalg + +import org.scalatest.FunSuite + +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} + +/** + * Test Breeze vector conversions. + */ +class BreezeVectorConversionSuite extends FunSuite { + + val arr = Array(0.1, 0.2, 0.3, 0.4) + val n = 20 + val indices = Array(0, 3, 5, 10, 13) + val values = Array(0.1, 0.5, 0.3, -0.8, -1.0) + + test("dense to breeze") { + val vec = Vectors.dense(arr) + assert(vec.toBreeze === new BDV[Double](arr)) + } + + test("sparse to breeze") { + val vec = Vectors.sparse(n, indices, values) + assert(vec.toBreeze === new BSV[Double](indices, values, n)) + } + + test("dense breeze to vector") { + val breeze = new BDV[Double](arr) + val vec = Vectors.fromBreeze(breeze).asInstanceOf[DenseVector] + assert(vec.size === arr.length) + assert(vec.values.eq(arr), "should not copy data") + } + + test("sparse breeze to vector") { + val breeze = new BSV[Double](indices, values, n) + val vec = Vectors.fromBreeze(breeze).asInstanceOf[SparseVector] + assert(vec.size === n) + assert(vec.indices.eq(indices), "should not copy data") + assert(vec.values.eq(values), "should not copy data") + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorSuite.scala new file mode 100644 index 0000000000000..e3ee97121f822 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorSuite.scala @@ -0,0 +1,48 @@ +/* + * 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.linalg + +import org.scalatest.FunSuite + +class VectorSuite extends FunSuite { + + val arr = Array(0.1, 0.2, 0.3, 0.4) + val n = 20 + val indices = Array(0, 3, 5, 10, 13) + val values = Array(0.1, 0.5, 0.3, -0.8, -1.0) + + test("dense vector construction") { + val vec = Vectors.dense(arr).asInstanceOf[DenseVector] + assert(vec.size === arr.length) + assert(vec.values.eq(arr)) + } + + test("sparse vector construction") { + val vec = Vectors.sparse(n, indices, values).asInstanceOf[SparseVector] + assert(vec.size === n) + assert(vec.indices.eq(indices)) + assert(vec.values.eq(values)) + } + + test("sparse vector construction with unordered elements") { + val vec = Vectors.sparse(n, indices.zip(values).reverse).asInstanceOf[SparseVector] + assert(vec.size === n) + assert(vec.indices === indices) + assert(vec.values === values) + } +} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 138aad7561043..83dd439720172 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -350,7 +350,8 @@ object SparkBuild extends Build { def mllibSettings = sharedSettings ++ Seq( name := "spark-mllib", libraryDependencies ++= Seq( - "org.jblas" % "jblas" % "1.2.3" + "org.jblas" % "jblas" % "1.2.3", + "org.scalanlp" %% "breeze" % "0.7-SNAPSHOT" ) ) From 4e7d5ca4f12508f35b8f0680acba3fc33b2eed09 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 10 Mar 2014 14:53:15 -0700 Subject: [PATCH 02/24] minor style update --- .../scala/org/apache/spark/mllib/clustering/KMeans.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index a6ecf64922713..f0aeec0c882d8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -309,10 +309,10 @@ object KMeans { initializationMode: String )(implicit d: DummyImplicit): KMeansModel = { new KMeans().setK(k) - .setMaxIterations(maxIterations) - .setRuns(runs) - .setInitializationMode(initializationMode) - .run(data) + .setMaxIterations(maxIterations) + .setRuns(runs) + .setInitializationMode(initializationMode) + .run(data) } def train(data: RDD[Vector], k: Int, maxIterations: Int, runs: Int) From ab74f67c0168dbe2c010f2d3dc262bd4ca987640 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 9 Mar 2014 20:26:03 -0700 Subject: [PATCH 03/24] add fastSquaredDistance for KMeans --- .../org/apache/spark/mllib/util/MLUtils.scala | 51 ++++++++++++++++++- .../spark/mllib/util/MLUtilsSuite.scala | 47 +++++++++++++++++ 2 files changed, 97 insertions(+), 1 deletion(-) create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 64c6136a8b89d..4b461f4000208 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -21,9 +21,14 @@ import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ +import org.apache.commons.math3.util.Precision.EPSILON + import org.jblas.DoubleMatrix + import org.apache.spark.mllib.regression.LabeledPoint +import breeze.linalg.{Vector => BV, SparseVector => BSV, squaredDistance => breezeSquaredDistance} + /** * Helper methods to load, save and pre-process data used in ML Lib. */ @@ -120,4 +125,48 @@ object MLUtils { } sum } -} + + /** + * Returns the squared Euclidean distance between two vectors. The following formula will be used + * if it does not introduce too much numerical error: + *
+   *   \|a - b\|_2^2 = \|a\|_2^2 + \|b\|_2^2 - 2 a^T b.
+   * 
+ * When both vector norms are given, this is faster than computing the squared distance directly, + * especially when one of the vectors is a sparse vector. + * + * @param v1 the first vector + * @param squaredNorm1 the squared norm of the first vector, non-negative + * @param v2 the second vector + * @param squaredNorm2 the squared norm of the second vector, non-negative + * @param precision desired relative precision for the squared distance + * @return squared distance between v1 and v2 within the specified precision + */ + private[mllib] def fastSquaredDistance( + v1: BV[Double], + squaredNorm1: Double, + v2: BV[Double], + squaredNorm2: Double, + precision: Double = 1e-6): Double = { + val n = v1.size + require(v2.size == n) + require(squaredNorm1 >= 0.0 && squaredNorm2 >= 0.0) + val sumSquaredNorm = squaredNorm1 + squaredNorm2 + val normDiff = math.sqrt(squaredNorm1) - math.sqrt(squaredNorm2) + var sqDist = 0.0 + val precisionBound1 = 2.0 * EPSILON * sumSquaredNorm / (normDiff * normDiff + EPSILON) + if (precisionBound1 < precision) { + sqDist = sumSquaredNorm - 2.0 * v1.dot(v2) + } else if (v1.isInstanceOf[BSV[Double]] || v2.isInstanceOf[BSV[Double]]) { + val dot = v1.dot(v2) + sqDist = math.max(sumSquaredNorm - 2.0 * dot, 0.0) + val precisionBound2 = EPSILON * (sumSquaredNorm + 2.0 * math.abs(dot)) / (sqDist + EPSILON) + if (precisionBound2 > precision) { + sqDist = breezeSquaredDistance(v1, v2) + } + } else { + sqDist = breezeSquaredDistance(v1, v2) + } + sqDist + } +} \ No newline at end of file diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala new file mode 100644 index 0000000000000..e1c5d93220579 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -0,0 +1,47 @@ +/* + * 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.util + +import org.scalatest.FunSuite + +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, norm => breezeNorm, + squaredDistance => breezeSquaredDistance} + +import org.apache.spark.mllib.util.MLUtils._ + +class MLUtilsSuite extends FunSuite { + + test("fast squared distance") { + val a = (30 to 0 by -1).map(math.pow(2.0, _)).toArray + val n = a.length + val v1 = new BDV[Double](a) + val norm1: Double = breezeNorm(v1) + val squaredNorm1 = norm1 * norm1 + val precision = 1e-6 + for (m <- 0 until n) { + val indices = (0 to m).toArray + val values = indices.map(i => a(i)) + val v2 = new BSV[Double](indices, values, n) + val norm2: Double = breezeNorm(v2) + val squaredNorm2 = norm2 * norm2 + val squaredDist: Double = breezeSquaredDistance(v1, v2) + val fastSquaredDist = fastSquaredDistance(v1, squaredNorm1, v2, squaredNorm2, precision) + assert((fastSquaredDist - squaredDist) <= precision * squaredDist, s"failed with m = $m") + } + } +} From f3554119dd7a555fd1ec4d9d61edd0cbf5787e77 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 10 Mar 2014 22:51:20 -0700 Subject: [PATCH 04/24] add BreezeVectorWithSquaredNorm case class --- .../spark/mllib/clustering/KMeans.scala | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index f0aeec0c882d8..52e2b20a4883b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -28,6 +28,9 @@ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom +private[clustering] +case class BreezeVectorWithSquaredNorm(vector: BV[Double], squaredNorm: Double) + /** * K-means clustering with support for multiple parallel runs and a k-means++ like initialization * mode (the k-means|| algorithm by Bahmani et al). When multiple concurrent runs are requested, @@ -362,6 +365,28 @@ object KMeans { (bestIndex, bestDistance) } + /** + * Returns the index of the closest center to the given point, as well as the squared distance. + */ + private[mllib] def findClosest( + centers: TraversableOnce[BreezeVectorWithSquaredNorm], + point: BreezeVectorWithSquaredNorm): (Int, Double) = { + var bestDistance = Double.PositiveInfinity + var bestIndex = 0 + var i = 0 + centers.foreach { center => + val distance: Double = MLUtils.fastSquaredDistance( + center.vector, center.squaredNorm, point.vector, point.squaredNorm + ) + if (distance < bestDistance) { + bestDistance = distance + bestIndex = i + } + i += 1 + } + (bestIndex, bestDistance) + } + /** * Return the K-means cost of a given point against the given cluster centers. */ From 0ff804609a2e66d6a380cf69b0684bc1a74c3eb2 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 11 Mar 2014 14:40:56 -0700 Subject: [PATCH 05/24] update KMeans to use fastSquaredDistance --- .../spark/mllib/clustering/KMeans.scala | 83 +++++++++++++------ .../spark/mllib/clustering/LocalKMeans.scala | 32 +++---- .../spark/mllib/clustering/KMeansSuite.scala | 23 +++-- 3 files changed, 92 insertions(+), 46 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 52e2b20a4883b..931ecee7486d1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -19,17 +19,25 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable.ArrayBuffer -import breeze.linalg.{DenseVector => BDV, Vector => BV, squaredDistance => breezeSquaredDistance} +import breeze.linalg.{DenseVector => BDV, Vector => BV, squaredDistance => breezeSquaredDistance, + norm => breezeNorm} import org.apache.spark.{Logging, SparkContext} import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.{Vectors, Vector} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom private[clustering] -case class BreezeVectorWithSquaredNorm(vector: BV[Double], squaredNorm: Double) +class BreezeVectorWithSquaredNorm(val vector: BV[Double], val squaredNorm: Double) + extends Serializable { + def this(vector: BV[Double]) = { + this(vector, {val nrm = breezeNorm(vector, 2.0); nrm * nrm}) + } + /** Converts the vector to a dense vector. */ + def toDense = new BreezeVectorWithSquaredNorm(vector.toDenseVector, squaredNorm) +} /** * K-means clustering with support for multiple parallel runs and a k-means++ like initialization @@ -114,8 +122,7 @@ class KMeans private ( * performance, because this is an iterative algorithm. */ def run(data: RDD[Array[Double]]): KMeansModel = { - val breezeData = data.map(v => new BDV[Double](v).asInstanceOf[BV[Double]]) - runBreeze(breezeData) + run(data.map(v => Vectors.dense(v))) } /** @@ -123,15 +130,23 @@ class KMeans private ( * performance, because this is an iterative algorithm. */ def run(data: RDD[Vector])(implicit d: DummyImplicit): KMeansModel = { - val breezeData = data.map(v => v.toBreeze) - runBreeze(breezeData) + val squaredNorms = data.map { v => + val nrm = breezeNorm(v.toBreeze, 2.0) + nrm * nrm + } + squaredNorms.persist() + val breezeData = data.map(_.toBreeze).zip(squaredNorms).map { case (v, squaredNorm) => + new BreezeVectorWithSquaredNorm(v, squaredNorm) + } + val model = runBreeze(breezeData) + squaredNorms.unpersist() + model } /** * Implementation using Breeze. */ - private def runBreeze(data: RDD[BV[Double]]): KMeansModel = { - // TODO: check whether data is persistent; this needs RDD.storageLevel to be publicly readable + private def runBreeze(data: RDD[BreezeVectorWithSquaredNorm]): KMeansModel = { val sc = data.sparkContext @@ -149,7 +164,7 @@ class KMeans private ( // Execute iterations of Lloyd's algorithm until all runs have converged while (iteration < maxIterations && !activeRuns.isEmpty) { - type WeightedPoint = (BDV[Double], Long) + type WeightedPoint = (BV[Double], Long) def mergeContribs(p1: WeightedPoint, p2: WeightedPoint): WeightedPoint = { (p1._1 += p2._1, p1._2 + p2._2) } @@ -161,15 +176,15 @@ class KMeans private ( val totalContribs = data.mapPartitions { points => val runs = activeCenters.length val k = activeCenters(0).length - val dims = activeCenters(0)(0).length + val dims = activeCenters(0)(0).vector.length - val sums = Array.fill(runs, k)(BDV.zeros[Double](dims)) + val sums = Array.fill(runs, k)(BDV.zeros[Double](dims).asInstanceOf[BV[Double]]) val counts = Array.fill(runs, k)(0L) for (point <- points; (centers, runIndex) <- activeCenters.zipWithIndex) { val (bestCenter, cost) = KMeans.findClosest(centers, point) costAccums(runIndex) += cost - sums(runIndex)(bestCenter) += point + sums(runIndex)(bestCenter) += point.vector counts(runIndex)(bestCenter) += 1 } @@ -186,8 +201,8 @@ class KMeans private ( val (sum, count) = totalContribs((i, j)) if (count != 0) { sum /= count.toDouble - val newCenter = sum - if (breezeSquaredDistance(newCenter, centers(run)(j)) > epsilon * epsilon) { + val newCenter = new BreezeVectorWithSquaredNorm(sum) + if (KMeans.fastSquaredDistance(newCenter, centers(run)(j)) > epsilon * epsilon) { changed = true } centers(run)(j) = newCenter @@ -206,18 +221,19 @@ class KMeans private ( val bestRun = costs.zipWithIndex.min._2 new KMeansModel(centers(bestRun).map { v => - v.toArray + v.vector.toArray }) } /** * Initialize `runs` sets of cluster centers at random. */ - private def initRandom(data: RDD[BV[Double]]): Array[Array[BV[Double]]] = { + private def initRandom(data: RDD[BreezeVectorWithSquaredNorm]) + : Array[Array[BreezeVectorWithSquaredNorm]] = { // Sample all the cluster centers in one pass to avoid repeated scans val sample = data.takeSample(true, runs * k, new XORShiftRandom().nextInt()).toSeq Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k).map { v => - v.toDenseVector + new BreezeVectorWithSquaredNorm(v.vector.toDenseVector, v.squaredNorm) }.toArray) } @@ -230,11 +246,12 @@ class KMeans private ( * * The original paper can be found at http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf. */ - private def initKMeansParallel(data: RDD[BV[Double]]): Array[Array[BV[Double]]] = { + private def initKMeansParallel(data: RDD[BreezeVectorWithSquaredNorm]) + : Array[Array[BreezeVectorWithSquaredNorm]] = { // Initialize each run's center to a random point val seed = new XORShiftRandom().nextInt() val sample = data.takeSample(true, runs, seed).toSeq - val centers = Array.tabulate(runs)(r => ArrayBuffer(sample(r).toDenseVector)) + val centers = Array.tabulate(runs)(r => ArrayBuffer(sample(r).toDense)) // On each step, sample 2 * k points on average for each run with probability proportional // to their squared distance from that run's current centers @@ -251,7 +268,7 @@ class KMeans private ( } yield (r, p) }.collect() for ((r, p) <- chosen) { - centers(r) += p.toDenseVector + centers(r) += p.toDense } } @@ -262,7 +279,7 @@ class KMeans private ( for (r <- 0 until runs) yield ((r, KMeans.findClosest(centers(r), p)._1), 1.0) }.reduceByKey(_ + _).collectAsMap() val finalCenters = (0 until runs).map { r => - val myCenters = centers(r).toArray.asInstanceOf[Array[BV[Double]]] + val myCenters = centers(r).toArray val myWeights = (0 until myCenters.length).map(i => weightMap.getOrElse((r, i), 0.0)).toArray LocalKMeans.kMeansPlusPlus(r, myCenters, myWeights, k, 30) } @@ -375,9 +392,7 @@ object KMeans { var bestIndex = 0 var i = 0 centers.foreach { center => - val distance: Double = MLUtils.fastSquaredDistance( - center.vector, center.squaredNorm, point.vector, point.squaredNorm - ) + val distance: Double = fastSquaredDistance(center, point) if (distance < bestDistance) { bestDistance = distance bestIndex = i @@ -407,6 +422,24 @@ object KMeans { private[mllib] def pointCost(centers: TraversableOnce[BV[Double]], point: BV[Double]): Double = findClosest(centers, point)._2 + /** + * Returns the K-means cost of a given point against the given cluster centers. + */ + private[mllib] def pointCost( + centers: TraversableOnce[BreezeVectorWithSquaredNorm], + point: BreezeVectorWithSquaredNorm): Double = + findClosest(centers, point)._2 + + /** + * Returns the squared Euclidean distance between two vectors computed by + * [[org.apache.spark.mllib.util.MLUtils.fastSquaredDistance()]]. + */ + private[clustering] + def fastSquaredDistance(v1: BreezeVectorWithSquaredNorm, v2: BreezeVectorWithSquaredNorm) + : Double = { + MLUtils.fastSquaredDistance(v1.vector, v1.squaredNorm, v2.vector, v2.squaredNorm) + } + def main(args: Array[String]) { if (args.length < 4) { println("Usage: KMeans []") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala index 9226538fac3ad..5627bdbc6cce3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.clustering import scala.util.Random -import breeze.linalg.{Vector => BV, DenseVector => BDV} +import breeze.linalg.{Vector => BV, DenseVector => BDV, norm => breezeNorm} /** * An utility object to run K-means locally. This is private to the ML package because it's used @@ -34,9 +34,13 @@ private[mllib] object LocalKMeans { k: Int, maxIterations: Int ): Array[Array[Double]] = { - val breezePoints = points.map(v => new BDV[Double](v).asInstanceOf[BV[Double]]) + val breezePoints = points.map { v => + val bv = new BDV[Double](v) + val norm: Double = breezeNorm(bv, 2.0) + new BreezeVectorWithSquaredNorm(bv, norm * norm) + } val breezeCenters = kMeansPlusPlus(seed, breezePoints, weights, k, maxIterations) - breezeCenters.map(_.toArray) + breezeCenters.map(_.vector.toArray) } /** @@ -45,20 +49,20 @@ private[mllib] object LocalKMeans { */ def kMeansPlusPlus( seed: Int, - points: Array[BV[Double]], + points: Array[BreezeVectorWithSquaredNorm], weights: Array[Double], k: Int, maxIterations: Int - )(implicit d: DummyImplicit): Array[BV[Double]] = { + )(implicit d: DummyImplicit): Array[BreezeVectorWithSquaredNorm] = { val rand = new Random(seed) - val dimensions = points(0).length - val centers = new Array[BV[Double]](k) + val dimensions = points(0).vector.length + val centers = new Array[BreezeVectorWithSquaredNorm](k) // Initialize centers by sampling using the k-means++ procedure. - centers(0) = (pickWeighted(rand, points, weights)).toDenseVector + centers(0) = pickWeighted(rand, points, weights).toDense for (i <- 1 until k) { // Pick the next center with a probability proportional to cost under current centers - val curCenters = centers.slice(0, i) + val curCenters = centers.view.take(i) val sum = points.zip(weights).map { case (p, w) => w * KMeans.pointCost(curCenters, p) }.sum @@ -69,7 +73,7 @@ private[mllib] object LocalKMeans { cumulativeScore += weights(j) * KMeans.pointCost(curCenters, points(j)) j += 1 } - centers(i) = points(j-1).toDenseVector + centers(i) = points(j-1).toDense } // Run up to maxIterations iterations of Lloyd's algorithm @@ -79,12 +83,12 @@ private[mllib] object LocalKMeans { while (moved && iteration < maxIterations) { moved = false val sums = Array.fill(k)( - new BDV[Double](new Array[Double](dimensions)).asInstanceOf[BV[Double]] + BDV.zeros[Double](dimensions).asInstanceOf[BV[Double]] ) val counts = Array.fill(k)(0.0) for ((p, i) <- points.zipWithIndex) { val index = KMeans.findClosest(centers, p)._1 - breeze.linalg.axpy(weights(i), p, sums(index)) + breeze.linalg.axpy(weights(i), p.vector, sums(index)) counts(index) += weights(i) if (index != oldClosest(i)) { moved = true @@ -95,10 +99,10 @@ private[mllib] object LocalKMeans { for (i <- 0 until k) { if (counts(i) == 0.0) { // Assign center to a random point - centers(i) = points(rand.nextInt(points.length)).toDenseVector + centers(i) = points(rand.nextInt(points.length)).toDense } else { sums(i) /= counts(i) - centers(i) = sums(i) + centers(i) = new BreezeVectorWithSquaredNorm(sums(i)) } } iteration += 1 diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index cabadd15731cb..637e11b3d7182 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -131,13 +131,20 @@ class KMeansSuite extends FunSuite with LocalSparkContext { } test("single cluster with sparse data") { - val n = 1000 - val smallData = Array( - Vectors.sparse(n, Seq((0, 1.0), (1, 2.0), (2, 6.0))), - Vectors.sparse(n, Seq((0, 1.0), (1, 3.0))), - Vectors.sparse(n, Seq((0, 1.0), (1, 4.0), (2, 6.0))) - ) - val data = sc.parallelize((1 to 100).flatMap(_ => smallData), 4) + val n = 1000000 + val data = sc.parallelize((1 to 100).flatMap { i => + val x = i / 1000.0 + Array( + Vectors.sparse(n, Seq((0, 1.0 + x), (1, 2.0), (2, 6.0))), + Vectors.sparse(n, Seq((0, 1.0 - x), (1, 2.0), (2, 6.0))), + Vectors.sparse(n, Seq((0, 1.0), (1, 3.0 + x))), + Vectors.sparse(n, Seq((0, 1.0), (1, 3.0 - x))), + Vectors.sparse(n, Seq((0, 1.0), (1, 4.0), (2, 6.0 + x))), + Vectors.sparse(n, Seq((0, 1.0), (1, 4.0), (2, 6.0 - x))) + ) + }, 4) + + data.persist() // No matter how many runs or iterations we use, we should get one cluster, // centered at the mean of the points @@ -167,6 +174,8 @@ class KMeansSuite extends FunSuite with LocalSparkContext { model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) assertSetsEqual(model.clusterCenters, Array(center)) + + data.unpersist() } test("k-means|| initialization") { From 87bc75506dc9309cbd43f0b02a90fce67e1a9988 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 11 Mar 2014 17:50:22 -0700 Subject: [PATCH 06/24] tuned the KMeans code: changed some for loops to while, use view to avoid copying arrays added some log messages --- .../spark/mllib/clustering/KMeans.scala | 59 ++++++++++++++----- .../spark/mllib/clustering/LocalKMeans.scala | 31 +++++++--- .../spark/mllib/clustering/KMeansSuite.scala | 3 +- 3 files changed, 67 insertions(+), 26 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 931ecee7486d1..b1e3c19bac4e3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -150,18 +150,26 @@ class KMeans private ( val sc = data.sparkContext + val initStartTime = System.nanoTime() + val centers = if (initializationMode == KMeans.RANDOM) { initRandom(data) } else { initKMeansParallel(data) } + val initTimeInSeconds = (System.nanoTime() - initStartTime) / 1e9 + logInfo(s"Initialization with $initializationMode took " + "%.3f".format(initTimeInSeconds) + + " seconds.") + val active = Array.fill(runs)(true) val costs = Array.fill(runs)(0.0) var activeRuns = new ArrayBuffer[Int] ++ (0 until runs) var iteration = 0 + val iterationStartTime = System.nanoTime() + // Execute iterations of Lloyd's algorithm until all runs have converged while (iteration < maxIterations && !activeRuns.isEmpty) { type WeightedPoint = (BV[Double], Long) @@ -181,11 +189,13 @@ class KMeans private ( val sums = Array.fill(runs, k)(BDV.zeros[Double](dims).asInstanceOf[BV[Double]]) val counts = Array.fill(runs, k)(0L) - for (point <- points; (centers, runIndex) <- activeCenters.zipWithIndex) { - val (bestCenter, cost) = KMeans.findClosest(centers, point) - costAccums(runIndex) += cost - sums(runIndex)(bestCenter) += point.vector - counts(runIndex)(bestCenter) += 1 + points.foreach { point => + activeRuns.foreach { r => + val (bestCenter, cost) = KMeans.findClosest(centers(r), point) + costAccums(r) += cost + sums(r)(bestCenter) += point.vector + counts(r)(bestCenter) += 1 + } } val contribs = for (i <- 0 until runs; j <- 0 until k) yield { @@ -195,9 +205,10 @@ class KMeans private ( }.reduceByKey(mergeContribs).collectAsMap() // Update the cluster centers and costs for each active run - for ((run, i) <- activeRuns.zipWithIndex) { + for ((run, i) <- activeRuns.view.zipWithIndex) { var changed = false - for (j <- 0 until k) { + var j = 0 + while (j < k) { val (sum, count) = totalContribs((i, j)) if (count != 0) { sum /= count.toDouble @@ -207,6 +218,7 @@ class KMeans private ( } centers(run)(j) = newCenter } + j += 1 } if (!changed) { active(run) = false @@ -219,6 +231,15 @@ class KMeans private ( iteration += 1 } + val iterationTimeInSeconds = (System.nanoTime() - iterationStartTime) / 1e9 + logInfo(s"Iterations took " + "%.3f".format(iterationTimeInSeconds) + " seconds.") + + if (iteration == maxIterations) { + logInfo(s"KMeans reached the max number of iterations: $maxIterations.") + } else { + logInfo(s"Kmeans converged in $iteration iterations.") + } + val bestRun = costs.zipWithIndex.min._2 new KMeansModel(centers(bestRun).map { v => v.vector.toArray @@ -255,28 +276,34 @@ class KMeans private ( // On each step, sample 2 * k points on average for each run with probability proportional // to their squared distance from that run's current centers - for (step <- 0 until initializationSteps) { + var step = 0 + while (step < initializationSteps) { val sumCosts = data.flatMap { point => - for (r <- 0 until runs) yield (r, KMeans.pointCost(centers(r), point)) + (0 until runs).map { r => + (r, KMeans.pointCost(centers(r), point)) + } }.reduceByKey(_ + _).collectAsMap() val chosen = data.mapPartitionsWithIndex { (index, points) => val rand = new XORShiftRandom(seed ^ (step << 16) ^ index) - for { - p <- points - r <- 0 until runs - if rand.nextDouble() < KMeans.pointCost(centers(r), p) * 2 * k / sumCosts(r) - } yield (r, p) + points.flatMap { p => + (0 until runs).filter { r => + rand.nextDouble() < 2.0 * KMeans.pointCost(centers(r), p) * k / sumCosts(r) + }.map((_, p)) + } }.collect() - for ((r, p) <- chosen) { + chosen.foreach { case (r, p) => centers(r) += p.toDense } + step += 1 } // Finally, we might have a set of more than k candidate centers for each run; weigh each // candidate by the number of points in the dataset mapping to it and run a local k-means++ // on the weighted centers to pick just k of them val weightMap = data.flatMap { p => - for (r <- 0 until runs) yield ((r, KMeans.findClosest(centers(r), p)._1), 1.0) + (0 until runs).map { r => + ((r, KMeans.findClosest(centers(r), p)._1), 1.0) + } }.reduceByKey(_ + _).collectAsMap() val finalCenters = (0 until runs).map { r => val myCenters = centers(r).toArray diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala index 5627bdbc6cce3..f694e95e537d7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala @@ -21,11 +21,13 @@ import scala.util.Random import breeze.linalg.{Vector => BV, DenseVector => BDV, norm => breezeNorm} +import org.apache.spark.Logging + /** * An utility object to run K-means locally. This is private to the ML package because it's used * in the initialization of KMeans but not meant to be publicly exposed. */ -private[mllib] object LocalKMeans { +private[mllib] object LocalKMeans extends Logging { def kMeansPlusPlus( seed: Int, @@ -63,7 +65,7 @@ private[mllib] object LocalKMeans { for (i <- 1 until k) { // Pick the next center with a probability proportional to cost under current centers val curCenters = centers.view.take(i) - val sum = points.zip(weights).map { case (p, w) => + val sum = points.view.zip(weights).map { case (p, w) => w * KMeans.pointCost(curCenters, p) }.sum val r = rand.nextDouble() * sum @@ -82,11 +84,13 @@ private[mllib] object LocalKMeans { var moved = true while (moved && iteration < maxIterations) { moved = false + val counts = Array.fill(k)(0.0) val sums = Array.fill(k)( BDV.zeros[Double](dimensions).asInstanceOf[BV[Double]] ) - val counts = Array.fill(k)(0.0) - for ((p, i) <- points.zipWithIndex) { + var i = 0 + while (i < points.length) { + val p = points(i) val index = KMeans.findClosest(centers, p)._1 breeze.linalg.axpy(weights(i), p.vector, sums(index)) counts(index) += weights(i) @@ -94,20 +98,29 @@ private[mllib] object LocalKMeans { moved = true oldClosest(i) = index } + i += 1 } // Update centers - for (i <- 0 until k) { - if (counts(i) == 0.0) { + var j = 0 + while (j < k) { + if (counts(j) == 0.0) { // Assign center to a random point - centers(i) = points(rand.nextInt(points.length)).toDense + centers(j) = points(rand.nextInt(points.length)).toDense } else { - sums(i) /= counts(i) - centers(i) = new BreezeVectorWithSquaredNorm(sums(i)) + sums(j) /= counts(j) + centers(j) = new BreezeVectorWithSquaredNorm(sums(j)) } + j += 1 } iteration += 1 } + if (iteration == maxIterations) { + logInfo(s"Local KMeans++ reached the max number of iterations: $maxIterations.") + } else { + logInfo(s"Local KMeans++ converged in $iteration iterations.") + } + centers } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index 637e11b3d7182..b0ef1eec67591 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -131,7 +131,8 @@ class KMeansSuite extends FunSuite with LocalSparkContext { } test("single cluster with sparse data") { - val n = 1000000 + + val n = 10000 val data = sc.parallelize((1 to 100).flatMap { i => val x = i / 1000.0 Array( From 0107e198d9c73565def84633c30a4061a6a96c10 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 11 Mar 2014 20:20:06 -0700 Subject: [PATCH 07/24] update NOTICE --- NOTICE | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/NOTICE b/NOTICE index 33e19418f4e56..25bcb0b8ee3a3 100644 --- a/NOTICE +++ b/NOTICE @@ -5,14 +5,8 @@ This product includes software developed at The Apache Software Foundation (http://www.apache.org/). Numerical linear algebra support in MLlib is provided by the breeze package, -which depends on the following packages that are not distributed under -Apache authorized licenses: - -- netlib-core, which is open source software written by Samuel Halliday, - and copyright by the University of Tennessee, the University of Tennessee - Research Foundation, the University of California at Berkeley, and the - University of Colorado at Denver. The original software is available from - https://github.com/fommil/netlib-java +which depends on the following package that are not distributed under +an Apache authorized license: - JTransforms, which is open source software written by Piotr Wendykier, and distributed under the the terms of the MPL/LGPL/GPL tri-license. From 3ed1a245db784bbae3d850fee401f5d727bfa39e Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 11 Mar 2014 20:23:27 -0700 Subject: [PATCH 08/24] add doc to BreezeVectorWithSquaredNorm --- .../scala/org/apache/spark/mllib/clustering/KMeans.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index b1e3c19bac4e3..709f614f7b73b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -29,6 +29,10 @@ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom +/** + * A breeze vector with its squared norm for fast distance computation. + * See [[org.apache.spark.mllib.clustering.KMeans.fastSquaredDistance()]]. + */ private[clustering] class BreezeVectorWithSquaredNorm(val vector: BV[Double], val squaredNorm: Double) extends Serializable { @@ -130,6 +134,7 @@ class KMeans private ( * performance, because this is an iterative algorithm. */ def run(data: RDD[Vector])(implicit d: DummyImplicit): KMeansModel = { + // Compute squared norms and cache them. val squaredNorms = data.map { v => val nrm = breezeNorm(v.toBreeze, 2.0) nrm * nrm @@ -144,7 +149,7 @@ class KMeans private ( } /** - * Implementation using Breeze. + * Implementation of K-Means using breeze. */ private def runBreeze(data: RDD[BreezeVectorWithSquaredNorm]): KMeansModel = { From 42b4e500d0e4b2bdf08bd9a7489eb9384263e975 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 11 Mar 2014 22:20:11 -0700 Subject: [PATCH 09/24] line feed at the end --- mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 4b461f4000208..b5fabc10e7ec4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -169,4 +169,4 @@ object MLUtils { } sqDist } -} \ No newline at end of file +} From d6e6c07f8cdaeacbb37b52211bc57e1a032dcba8 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 12 Mar 2014 13:26:45 -0700 Subject: [PATCH 10/24] add predict(RDD[Vector]) to KMeansModel add a test for two clusteres --- .../spark/mllib/clustering/KMeansModel.scala | 13 ++++++--- .../spark/mllib/clustering/KMeansSuite.scala | 27 ++++++++++++++++++- 2 files changed, 35 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index 06cf2e3ceb36a..6192f0779b2a7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -28,10 +28,6 @@ import org.apache.spark.mllib.linalg.Vector */ class KMeansModel(val clusterCenters: Array[Array[Double]]) extends Serializable { - private val breezeClusterCenters = clusterCenters.map { v => - new BreezeDenseVector[Double](v) - } - /** Total number of clusters. */ def k: Int = clusterCenters.length @@ -40,10 +36,18 @@ class KMeansModel(val clusterCenters: Array[Array[Double]]) extends Serializable KMeans.findClosest(clusterCenters, point)._1 } + /** Returns the cluster index that a given point belongs to. */ def predict(point: Vector): Int = { + val breezeClusterCenters = clusterCenters.view.map(new BreezeDenseVector[Double](_)) KMeans.findClosest(breezeClusterCenters, point.toBreeze)._1 } + /** Maps given points to their cluster indices. */ + def predict(points: RDD[Vector]): RDD[Int] = { + val breezeClusterCenters = clusterCenters.map(new BreezeDenseVector[Double](_)) + points.map(p => KMeans.findClosest(breezeClusterCenters, p.toBreeze)._1) + } + /** * Return the K-means cost (sum of squared distances of points to their nearest center) for this * model on the given data. @@ -57,6 +61,7 @@ class KMeansModel(val clusterCenters: Array[Array[Double]]) extends Serializable * model on the given data. */ def computeCost(data: RDD[Vector])(implicit d: DummyImplicit): Double = { + val breezeClusterCenters = clusterCenters.map(new BreezeDenseVector[Double](_)) data.map(p => KMeans.pointCost(breezeClusterCenters, p.toBreeze)).sum() } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index b0ef1eec67591..e73ea9b53918e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.clustering import org.scalatest.FunSuite import org.apache.spark.mllib.util.LocalSparkContext -import org.apache.spark.mllib.linalg.{Vectors, Vector} +import org.apache.spark.mllib.linalg.Vectors class KMeansSuite extends FunSuite with LocalSparkContext { @@ -204,4 +204,29 @@ class KMeansSuite extends FunSuite with LocalSparkContext { model = KMeans.train(rdd, k=5, maxIterations=10, runs=5) assertSetsEqual(model.clusterCenters, points) } + + test("two clusters") { + val points = Array( + Array(0.0, 0.0), + Array(0.0, 0.1), + Array(0.1, 0.0), + Array(9.0, 0.0), + Array(9.0, 0.2), + Array(9.2, 0.0) + ).map(Vectors.dense) + val rdd = sc.parallelize(points, 3) + + for (initMode <- Seq(RANDOM, K_MEANS_PARALLEL)) { + // Two iterations are sufficient no matter where the initial centers are. + val model = KMeans.train(rdd, k = 2, maxIterations = 2, runs = 1, initMode) + + val predicts = model.predict(rdd).collect() + + assert(predicts(0) === predicts(1)) + assert(predicts(0) === predicts(2)) + assert(predicts(3) === predicts(4)) + assert(predicts(3) === predicts(5)) + assert(predicts(0) != predicts(3)) + } + } } From 6f5cddea3864aa25a0433ae6b2ce78d5bb42be81 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 12 Mar 2014 16:48:11 -0700 Subject: [PATCH 11/24] fix a bug in filtering finished runs --- .../spark/mllib/clustering/KMeans.scala | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 709f614f7b73b..852cfecfc9171 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -195,11 +195,11 @@ class KMeans private ( val counts = Array.fill(runs, k)(0L) points.foreach { point => - activeRuns.foreach { r => - val (bestCenter, cost) = KMeans.findClosest(centers(r), point) - costAccums(r) += cost - sums(r)(bestCenter) += point.vector - counts(r)(bestCenter) += 1 + (0 until runs).foreach { i => + val (bestCenter, cost) = KMeans.findClosest(activeCenters(i), point) + costAccums(i) += cost + sums(i)(bestCenter) += point.vector + counts(i)(bestCenter) += 1 } } @@ -210,7 +210,7 @@ class KMeans private ( }.reduceByKey(mergeContribs).collectAsMap() // Update the cluster centers and costs for each active run - for ((run, i) <- activeRuns.view.zipWithIndex) { + for ((run, i) <- activeRuns.zipWithIndex) { var changed = false var j = 0 while (j < k) { @@ -242,10 +242,13 @@ class KMeans private ( if (iteration == maxIterations) { logInfo(s"KMeans reached the max number of iterations: $maxIterations.") } else { - logInfo(s"Kmeans converged in $iteration iterations.") + logInfo(s"KMeans converged in $iteration iterations.") } - val bestRun = costs.zipWithIndex.min._2 + val (minCost, bestRun) = costs.zipWithIndex.min + + logInfo(s"The cost for the best run is $minCost.") + new KMeansModel(centers(bestRun).map { v => v.vector.toArray }) From 07c3cf25734f19204d71ec0995cea82566d037a7 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 18 Mar 2014 11:44:28 -0700 Subject: [PATCH 12/24] change Mahout to breeze in doc use a simple lower bound to avoid unnecessary distance computation --- .../org/apache/spark/mllib/clustering/KMeans.scala | 14 ++++++++++---- .../org/apache/spark/mllib/linalg/Vectors.scala | 2 +- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 852cfecfc9171..ad9df556a8ddc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -427,10 +427,16 @@ object KMeans { var bestIndex = 0 var i = 0 centers.foreach { center => - val distance: Double = fastSquaredDistance(center, point) - if (distance < bestDistance) { - bestDistance = distance - bestIndex = i + // Since `\|a - b\| \geq |\|a\| - \|b\||`, we can use this lower bound to avoid unnecessary + // distance computation. + var lowerBoundOfSqDist = math.sqrt(center.squaredNorm) - math.sqrt(point.squaredNorm) + lowerBoundOfSqDist = lowerBoundOfSqDist * lowerBoundOfSqDist + if (lowerBoundOfSqDist < bestDistance) { + val distance: Double = fastSquaredDistance(center, point) + if (distance < bestDistance) { + bestDistance = distance + bestIndex = i + } } i += 1 } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index b95889f9a44ff..c102084caa997 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -31,7 +31,7 @@ trait Vector extends Serializable { def size: Int /** - * Converts the instance to a Mahout vector wrapper. + * Converts the instance to a breeze vector. */ private[mllib] def toBreeze: BreezeVector[Double] } From 27858e49f5866092bcd4076f16f3ece4af14695e Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 19 Mar 2014 12:54:03 -0700 Subject: [PATCH 13/24] update breeze version to 0.7 --- mllib/pom.xml | 2 +- project/SparkBuild.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/pom.xml b/mllib/pom.xml index 16c4e60c734fd..fec1cc94b2642 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -63,7 +63,7 @@ org.scalanlp breeze_${scala.binary.version} - 0.7-SNAPSHOT + 0.7 org.scalatest diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 99515c9825c55..bb454d1268766 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -357,7 +357,7 @@ object SparkBuild extends Build { name := "spark-mllib", libraryDependencies ++= Seq( "org.jblas" % "jblas" % "1.2.3", - "org.scalanlp" %% "breeze" % "0.7-SNAPSHOT" + "org.scalanlp" %% "breeze" % "0.7" ) ) From 712cb8881cee541e74f307c67c0c34f2170cb3e2 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 19 Mar 2014 17:06:47 -0700 Subject: [PATCH 14/24] make Vectors.sparse Java friendly rename VectorSuite to VectorsSuite --- .../apache/spark/mllib/linalg/Vectors.scala | 21 +++++++++++++++---- .../{VectorSuite.scala => VectorsSuite.scala} | 11 +++++++++- 2 files changed, 27 insertions(+), 5 deletions(-) rename mllib/src/test/scala/org/apache/spark/mllib/linalg/{VectorSuite.scala => VectorsSuite.scala} (81%) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index c102084caa997..16a19df4472f4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -17,6 +17,10 @@ package org.apache.spark.mllib.linalg +import java.lang.{Iterable => JavaIterable} + +import scala.collection.JavaConverters._ + import breeze.linalg.{Vector => BreezeVector, DenseVector => BreezeDenseVector, SparseVector => BreezeSparseVector} @@ -68,11 +72,11 @@ object Vectors { * @param size vector size. * @param elements vector elements in (index, value) pairs. */ - def sparse(size: Int, elements: Iterable[(Int, Double)]): Vector = { + def sparse(size: Int, elements: Seq[(Int, Double)]): Vector = { require(size > 0) - val (indices, values) = elements.toArray.sortBy(_._1).unzip + val (indices, values) = elements.sortBy(_._1).unzip var prev = -1 indices.foreach { i => require(prev < i, "Found duplicate indices: " + i) @@ -83,14 +87,23 @@ object Vectors { new SparseVector(size, indices.toArray, values.toArray) } + /** + * Creates a sparse vector using unordered (index, value) pairs. + * + * @param size vector size. + * @param elements vector elements in (index, value) pairs. + */ + def sparse(size: Int, elements: JavaIterable[(Int, Double)]): Vector = + sparse(size, elements.asScala.toSeq) + /** * Creates a vector instance from a breeze vector. */ private[mllib] def fromBreeze(breezeVector: BreezeVector[Double]): Vector = { breezeVector match { case v: BreezeDenseVector[Double] => { - require(v.offset == 0) - require(v.stride == 1) + require(v.offset == 0, s"Do not support non-zero offset ${v.offset}.") + require(v.stride == 1, s"Do not support stride other than 1, but got ${v.stride}.") new DenseVector(v.data) } case v: BreezeSparseVector[Double] => { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala similarity index 81% rename from mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index e3ee97121f822..adf2005b84f1d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -17,9 +17,11 @@ package org.apache.spark.mllib.linalg +import scala.collection.JavaConverters._ + import org.scalatest.FunSuite -class VectorSuite extends FunSuite { +class VectorsSuite extends FunSuite { val arr = Array(0.1, 0.2, 0.3, 0.4) val n = 20 @@ -45,4 +47,11 @@ class VectorSuite extends FunSuite { assert(vec.indices === indices) assert(vec.values === values) } + + test("sparse vector construction with unordered elements stored as Java Iterable") { + val vec = Vectors.sparse(n, indices.toSeq.zip(values).reverse.asJava).asInstanceOf[SparseVector] + assert(vec.size === n) + assert(vec.indices === indices) + assert(vec.values === values) + } } From 72bde334cb31d93d3b904701d0d709102c89e6b3 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 19 Mar 2014 18:07:14 -0700 Subject: [PATCH 15/24] clean up code for distance computation --- .../spark/mllib/clustering/KMeans.scala | 126 +++++------------- .../spark/mllib/clustering/KMeansModel.scala | 21 +-- .../spark/mllib/clustering/LocalKMeans.scala | 24 +--- .../org/apache/spark/mllib/util/MLUtils.scala | 30 +---- .../spark/mllib/util/LocalSparkContext.scala | 17 +++ .../spark/mllib/util/MLUtilsSuite.scala | 14 +- 6 files changed, 80 insertions(+), 152 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index ad9df556a8ddc..d09f29f882310 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -19,28 +19,30 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable.ArrayBuffer -import breeze.linalg.{DenseVector => BDV, Vector => BV, squaredDistance => breezeSquaredDistance, - norm => breezeNorm} - +import breeze.linalg.{DenseVector => BDV, Vector => BV, norm => breezeNorm} import org.apache.spark.{Logging, SparkContext} import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.linalg.{Vectors, Vector} +import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom /** - * A breeze vector with its squared norm for fast distance computation. - * See [[org.apache.spark.mllib.clustering.KMeans.fastSquaredDistance()]]. + * A breeze vector with its norm for fast distance computation. + * + * @see [[org.apache.spark.mllib.clustering.KMeans#fastSquaredDistance]] */ private[clustering] -class BreezeVectorWithSquaredNorm(val vector: BV[Double], val squaredNorm: Double) - extends Serializable { - def this(vector: BV[Double]) = { - this(vector, {val nrm = breezeNorm(vector, 2.0); nrm * nrm}) - } +class BreezeVectorWithNorm(val vector: BV[Double], val norm: Double) extends Serializable { + + def this(vector: BV[Double]) = this(vector, breezeNorm(vector, 2.0)) + + def this(array: Array[Double]) = this(new BDV[Double](array)) + + def this(v: Vector) = this(v.toBreeze) + /** Converts the vector to a dense vector. */ - def toDense = new BreezeVectorWithSquaredNorm(vector.toDenseVector, squaredNorm) + def toDense = new BreezeVectorWithNorm(vector.toDenseVector, norm) } /** @@ -135,23 +137,20 @@ class KMeans private ( */ def run(data: RDD[Vector])(implicit d: DummyImplicit): KMeansModel = { // Compute squared norms and cache them. - val squaredNorms = data.map { v => - val nrm = breezeNorm(v.toBreeze, 2.0) - nrm * nrm - } - squaredNorms.persist() - val breezeData = data.map(_.toBreeze).zip(squaredNorms).map { case (v, squaredNorm) => - new BreezeVectorWithSquaredNorm(v, squaredNorm) + val norms = data.map(v => breezeNorm(v.toBreeze, 2.0)) + norms.persist() + val breezeData = data.map(_.toBreeze).zip(norms).map { case (v, norm) => + new BreezeVectorWithNorm(v, norm) } val model = runBreeze(breezeData) - squaredNorms.unpersist() + norms.unpersist() model } /** * Implementation of K-Means using breeze. */ - private def runBreeze(data: RDD[BreezeVectorWithSquaredNorm]): KMeansModel = { + private def runBreeze(data: RDD[BreezeVectorWithNorm]): KMeansModel = { val sc = data.sparkContext @@ -217,7 +216,7 @@ class KMeans private ( val (sum, count) = totalContribs((i, j)) if (count != 0) { sum /= count.toDouble - val newCenter = new BreezeVectorWithSquaredNorm(sum) + val newCenter = new BreezeVectorWithNorm(sum) if (KMeans.fastSquaredDistance(newCenter, centers(run)(j)) > epsilon * epsilon) { changed = true } @@ -257,12 +256,12 @@ class KMeans private ( /** * Initialize `runs` sets of cluster centers at random. */ - private def initRandom(data: RDD[BreezeVectorWithSquaredNorm]) - : Array[Array[BreezeVectorWithSquaredNorm]] = { + private def initRandom(data: RDD[BreezeVectorWithNorm]) + : Array[Array[BreezeVectorWithNorm]] = { // Sample all the cluster centers in one pass to avoid repeated scans val sample = data.takeSample(true, runs * k, new XORShiftRandom().nextInt()).toSeq Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k).map { v => - new BreezeVectorWithSquaredNorm(v.vector.toDenseVector, v.squaredNorm) + new BreezeVectorWithNorm(v.vector.toDenseVector, v.norm) }.toArray) } @@ -275,8 +274,8 @@ class KMeans private ( * * The original paper can be found at http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf. */ - private def initKMeansParallel(data: RDD[BreezeVectorWithSquaredNorm]) - : Array[Array[BreezeVectorWithSquaredNorm]] = { + private def initKMeansParallel(data: RDD[BreezeVectorWithNorm]) + : Array[Array[BreezeVectorWithNorm]] = { // Initialize each run's center to a random point val seed = new XORShiftRandom().nextInt() val sample = data.takeSample(true, runs, seed).toSeq @@ -380,56 +379,19 @@ object KMeans { train(data, k, maxIterations, 1, K_MEANS_PARALLEL) } - /** - * Return the index of the closest point in `centers` to `point`, as well as its distance. - */ - private[mllib] def findClosest(centers: Array[Array[Double]], point: Array[Double]) - : (Int, Double) = - { - var bestDistance = Double.PositiveInfinity - var bestIndex = 0 - for (i <- 0 until centers.length) { - val distance = MLUtils.squaredDistance(point, centers(i)) - if (distance < bestDistance) { - bestDistance = distance - bestIndex = i - } - } - (bestIndex, bestDistance) - } - - /** - * Returns the index of the closest center to the given point, as well as the squared distance. - */ - private[mllib] def findClosest(centers: TraversableOnce[BV[Double]], point: BV[Double]) - : (Int, Double) = { - var bestDistance = Double.PositiveInfinity - var bestIndex = 0 - var i = 0 - centers.foreach { v => - val distance: Double = breezeSquaredDistance(v, point) - if (distance < bestDistance) { - bestDistance = distance - bestIndex = i - } - i += 1 - } - (bestIndex, bestDistance) - } - /** * Returns the index of the closest center to the given point, as well as the squared distance. */ private[mllib] def findClosest( - centers: TraversableOnce[BreezeVectorWithSquaredNorm], - point: BreezeVectorWithSquaredNorm): (Int, Double) = { + centers: TraversableOnce[BreezeVectorWithNorm], + point: BreezeVectorWithNorm): (Int, Double) = { var bestDistance = Double.PositiveInfinity var bestIndex = 0 var i = 0 centers.foreach { center => // Since `\|a - b\| \geq |\|a\| - \|b\||`, we can use this lower bound to avoid unnecessary // distance computation. - var lowerBoundOfSqDist = math.sqrt(center.squaredNorm) - math.sqrt(point.squaredNorm) + var lowerBoundOfSqDist = center.norm - point.norm lowerBoundOfSqDist = lowerBoundOfSqDist * lowerBoundOfSqDist if (lowerBoundOfSqDist < bestDistance) { val distance: Double = fastSquaredDistance(center, point) @@ -443,42 +405,22 @@ object KMeans { (bestIndex, bestDistance) } - /** - * Return the K-means cost of a given point against the given cluster centers. - */ - private[mllib] def pointCost(centers: Array[Array[Double]], point: Array[Double]): Double = { - var bestDistance = Double.PositiveInfinity - for (i <- 0 until centers.length) { - val distance = MLUtils.squaredDistance(point, centers(i)) - if (distance < bestDistance) { - bestDistance = distance - } - } - bestDistance - } - - /** - * Returns the K-means cost of a given point against the given cluster centers. - */ - private[mllib] def pointCost(centers: TraversableOnce[BV[Double]], point: BV[Double]): Double = - findClosest(centers, point)._2 - /** * Returns the K-means cost of a given point against the given cluster centers. */ private[mllib] def pointCost( - centers: TraversableOnce[BreezeVectorWithSquaredNorm], - point: BreezeVectorWithSquaredNorm): Double = + centers: TraversableOnce[BreezeVectorWithNorm], + point: BreezeVectorWithNorm): Double = findClosest(centers, point)._2 /** * Returns the squared Euclidean distance between two vectors computed by - * [[org.apache.spark.mllib.util.MLUtils.fastSquaredDistance()]]. + * [[org.apache.spark.mllib.util.MLUtils#fastSquaredDistance]]. */ private[clustering] - def fastSquaredDistance(v1: BreezeVectorWithSquaredNorm, v2: BreezeVectorWithSquaredNorm) + def fastSquaredDistance(v1: BreezeVectorWithNorm, v2: BreezeVectorWithNorm) : Double = { - MLUtils.fastSquaredDistance(v1.vector, v1.squaredNorm, v2.vector, v2.squaredNorm) + MLUtils.fastSquaredDistance(v1.vector, v1.norm, v2.vector, v2.norm) } def main(args: Array[String]) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index 6192f0779b2a7..5ec263ff44fef 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -17,8 +17,6 @@ package org.apache.spark.mllib.clustering -import breeze.linalg.{DenseVector => BreezeDenseVector} - import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vector @@ -33,19 +31,18 @@ class KMeansModel(val clusterCenters: Array[Array[Double]]) extends Serializable /** Return the cluster index that a given point belongs to. */ def predict(point: Array[Double]): Int = { - KMeans.findClosest(clusterCenters, point)._1 + KMeans.findClosest(clusterCentersWithNorm, new BreezeVectorWithNorm(point))._1 } /** Returns the cluster index that a given point belongs to. */ def predict(point: Vector): Int = { - val breezeClusterCenters = clusterCenters.view.map(new BreezeDenseVector[Double](_)) - KMeans.findClosest(breezeClusterCenters, point.toBreeze)._1 + KMeans.findClosest(clusterCentersWithNorm, new BreezeVectorWithNorm(point))._1 } /** Maps given points to their cluster indices. */ def predict(points: RDD[Vector]): RDD[Int] = { - val breezeClusterCenters = clusterCenters.map(new BreezeDenseVector[Double](_)) - points.map(p => KMeans.findClosest(breezeClusterCenters, p.toBreeze)._1) + val centersWithNorm = clusterCentersWithNorm + points.map(p => KMeans.findClosest(centersWithNorm, new BreezeVectorWithNorm(p))._1) } /** @@ -53,7 +50,8 @@ class KMeansModel(val clusterCenters: Array[Array[Double]]) extends Serializable * model on the given data. */ def computeCost(data: RDD[Array[Double]]): Double = { - data.map(p => KMeans.pointCost(clusterCenters, p)).sum() + val centersWithNorm = clusterCentersWithNorm + data.map(p => KMeans.pointCost(centersWithNorm, new BreezeVectorWithNorm(p))).sum() } /** @@ -61,7 +59,10 @@ class KMeansModel(val clusterCenters: Array[Array[Double]]) extends Serializable * model on the given data. */ def computeCost(data: RDD[Vector])(implicit d: DummyImplicit): Double = { - val breezeClusterCenters = clusterCenters.map(new BreezeDenseVector[Double](_)) - data.map(p => KMeans.pointCost(breezeClusterCenters, p.toBreeze)).sum() + val centersWithNorm = clusterCentersWithNorm + data.map(p => KMeans.pointCost(centersWithNorm, new BreezeVectorWithNorm(p))).sum() } + + private def clusterCentersWithNorm: Iterable[BreezeVectorWithNorm] = + clusterCenters.map(new BreezeVectorWithNorm(_)) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala index f694e95e537d7..2e3a4ce783de7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala @@ -29,36 +29,20 @@ import org.apache.spark.Logging */ private[mllib] object LocalKMeans extends Logging { - def kMeansPlusPlus( - seed: Int, - points: Array[Array[Double]], - weights: Array[Double], - k: Int, - maxIterations: Int - ): Array[Array[Double]] = { - val breezePoints = points.map { v => - val bv = new BDV[Double](v) - val norm: Double = breezeNorm(bv, 2.0) - new BreezeVectorWithSquaredNorm(bv, norm * norm) - } - val breezeCenters = kMeansPlusPlus(seed, breezePoints, weights, k, maxIterations) - breezeCenters.map(_.vector.toArray) - } - /** * Run K-means++ on the weighted point set `points`. This first does the K-means++ * initialization procedure and then rounds of Lloyd's algorithm. */ def kMeansPlusPlus( seed: Int, - points: Array[BreezeVectorWithSquaredNorm], + points: Array[BreezeVectorWithNorm], weights: Array[Double], k: Int, maxIterations: Int - )(implicit d: DummyImplicit): Array[BreezeVectorWithSquaredNorm] = { + ): Array[BreezeVectorWithNorm] = { val rand = new Random(seed) val dimensions = points(0).vector.length - val centers = new Array[BreezeVectorWithSquaredNorm](k) + val centers = new Array[BreezeVectorWithNorm](k) // Initialize centers by sampling using the k-means++ procedure. centers(0) = pickWeighted(rand, points, weights).toDense @@ -108,7 +92,7 @@ private[mllib] object LocalKMeans extends Logging { centers(j) = points(rand.nextInt(points.length)).toDense } else { sums(j) /= counts(j) - centers(j) = new BreezeVectorWithSquaredNorm(sums(j)) + centers(j) = new BreezeVectorWithNorm(sums(j)) } j += 1 } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index b5fabc10e7ec4..55196f2aa2243 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -110,22 +110,6 @@ object MLUtils { (yMean, xColMean, xColSd) } - /** - * Return the squared Euclidean distance between two vectors. - */ - def squaredDistance(v1: Array[Double], v2: Array[Double]): Double = { - if (v1.length != v2.length) { - throw new IllegalArgumentException("Vector sizes don't match") - } - var i = 0 - var sum = 0.0 - while (i < v1.length) { - sum += (v1(i) - v2(i)) * (v1(i) - v2(i)) - i += 1 - } - sum - } - /** * Returns the squared Euclidean distance between two vectors. The following formula will be used * if it does not introduce too much numerical error: @@ -136,23 +120,23 @@ object MLUtils { * especially when one of the vectors is a sparse vector. * * @param v1 the first vector - * @param squaredNorm1 the squared norm of the first vector, non-negative + * @param norm1 the norm of the first vector, non-negative * @param v2 the second vector - * @param squaredNorm2 the squared norm of the second vector, non-negative + * @param norm2 the norm of the second vector, non-negative * @param precision desired relative precision for the squared distance * @return squared distance between v1 and v2 within the specified precision */ private[mllib] def fastSquaredDistance( v1: BV[Double], - squaredNorm1: Double, + norm1: Double, v2: BV[Double], - squaredNorm2: Double, + norm2: Double, precision: Double = 1e-6): Double = { val n = v1.size require(v2.size == n) - require(squaredNorm1 >= 0.0 && squaredNorm2 >= 0.0) - val sumSquaredNorm = squaredNorm1 + squaredNorm2 - val normDiff = math.sqrt(squaredNorm1) - math.sqrt(squaredNorm2) + require(norm1 >= 0.0 && norm2 >= 0.0) + val sumSquaredNorm = norm1 * norm1 + norm2 * norm2 + val normDiff = norm1 - norm2 var sqDist = 0.0 val precisionBound1 = 2.0 * EPSILON * sumSquaredNorm / (normDiff * normDiff + EPSILON) if (precisionBound1 < precision) { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala index 7d840043e5c6b..212fbe9288f0d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala @@ -1,3 +1,20 @@ +/* + * 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.util import org.scalatest.Suite diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index e1c5d93220579..f2ac4205472ed 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -30,18 +30,18 @@ class MLUtilsSuite extends FunSuite { val a = (30 to 0 by -1).map(math.pow(2.0, _)).toArray val n = a.length val v1 = new BDV[Double](a) - val norm1: Double = breezeNorm(v1) - val squaredNorm1 = norm1 * norm1 + val norm1 = breezeNorm(v1, 2.0) val precision = 1e-6 for (m <- 0 until n) { val indices = (0 to m).toArray val values = indices.map(i => a(i)) val v2 = new BSV[Double](indices, values, n) - val norm2: Double = breezeNorm(v2) - val squaredNorm2 = norm2 * norm2 - val squaredDist: Double = breezeSquaredDistance(v1, v2) - val fastSquaredDist = fastSquaredDistance(v1, squaredNorm1, v2, squaredNorm2, precision) - assert((fastSquaredDist - squaredDist) <= precision * squaredDist, s"failed with m = $m") + val norm2 = breezeNorm(v2, 2.0) + val squaredDist = breezeSquaredDistance(v1, v2) + val fastSquaredDist1 = fastSquaredDistance(v1, norm1, v2, norm2, precision) + assert((fastSquaredDist1 - squaredDist) <= precision * squaredDist, s"failed with m = $m") + val fastSquaredDist2 = fastSquaredDistance(v1, norm1, v2.toDenseVector, norm2, precision) + assert((fastSquaredDist2 - squaredDist) <= precision * squaredDist, s"failed with m = $m") } } } From e69b10cce0125a60efd5341c02498302e162d42b Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 20 Mar 2014 16:26:57 -0700 Subject: [PATCH 16/24] remove examples/JavaKMeans.java, which is replaced by mllib/examples/JavaKMeans.java --- .../org/apache/spark/examples/JavaKMeans.java | 138 ------------------ 1 file changed, 138 deletions(-) delete mode 100644 examples/src/main/java/org/apache/spark/examples/JavaKMeans.java diff --git a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java deleted file mode 100644 index 2d797279d5bcc..0000000000000 --- a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * 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; - -import scala.Tuple2; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.util.Vector; - -import java.util.List; -import java.util.Map; -import java.util.regex.Pattern; - -/** - * K-means clustering using Java API. - */ -public final class JavaKMeans { - - private static final Pattern SPACE = Pattern.compile(" "); - - /** Parses numbers split by whitespace to a vector */ - static Vector parseVector(String line) { - String[] splits = SPACE.split(line); - double[] data = new double[splits.length]; - int i = 0; - for (String s : splits) { - data[i] = Double.parseDouble(s); - i++; - } - return new Vector(data); - } - - /** Computes the vector to which the input vector is closest using squared distance */ - static int closestPoint(Vector p, List centers) { - int bestIndex = 0; - double closest = Double.POSITIVE_INFINITY; - for (int i = 0; i < centers.size(); i++) { - double tempDist = p.squaredDist(centers.get(i)); - if (tempDist < closest) { - closest = tempDist; - bestIndex = i; - } - } - return bestIndex; - } - - /** Computes the mean across all vectors in the input set of vectors */ - static Vector average(List ps) { - int numVectors = ps.size(); - Vector out = new Vector(ps.get(0).elements()); - // start from i = 1 since we already copied index 0 above - for (int i = 1; i < numVectors; i++) { - out.addInPlace(ps.get(i)); - } - return out.divide(numVectors); - } - - public static void main(String[] args) throws Exception { - if (args.length < 4) { - System.err.println("Usage: JavaKMeans "); - System.exit(1); - } - JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans", - System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaKMeans.class)); - String path = args[1]; - int K = Integer.parseInt(args[2]); - double convergeDist = Double.parseDouble(args[3]); - - JavaRDD data = sc.textFile(path).map( - new Function() { - @Override - public Vector call(String line) { - return parseVector(line); - } - } - ).cache(); - - final List centroids = data.takeSample(false, K, 42); - - double tempDist; - do { - // allocate each vector to closest centroid - JavaPairRDD closest = data.mapToPair( - new PairFunction() { - @Override - public Tuple2 call(Vector vector) { - return new Tuple2( - closestPoint(vector, centroids), vector); - } - } - ); - - // group by cluster id and average the vectors within each cluster to compute centroids - JavaPairRDD> pointsGroup = closest.groupByKey(); - Map newCentroids = pointsGroup.mapValues( - new Function, Vector>() { - @Override - public Vector call(List ps) { - return average(ps); - } - }).collectAsMap(); - tempDist = 0.0; - for (int i = 0; i < K; i++) { - tempDist += centroids.get(i).squaredDist(newCentroids.get(i)); - } - for (Map.Entry t: newCentroids.entrySet()) { - centroids.set(t.getKey(), t.getValue()); - } - System.out.println("Finished iteration (delta = " + tempDist + ")"); - } while (tempDist > convergeDist); - - System.out.println("Final centers:"); - for (Vector c : centroids) { - System.out.println(c); - } - - System.exit(0); - - } -} From b28ba2feccb23b7f193aca813e915726634cfb18 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 21 Mar 2014 00:33:18 -0700 Subject: [PATCH 17/24] add toArray to Vector add Java friendly methods to Vectors remove RDD[Array[Double]] support from KMeans update Java KMeans API --- .../spark/mllib/examples/JavaKMeans.java | 23 +-- .../mllib/api/python/PythonMLLibAPI.scala | 12 +- .../spark/mllib/clustering/KMeans.scala | 107 +++++--------- .../spark/mllib/clustering/KMeansModel.scala | 18 +-- .../apache/spark/mllib/linalg/Vectors.scala | 74 +++++++--- .../mllib/clustering/JavaKMeansSuite.java | 88 +++++------- .../spark/mllib/clustering/KMeansSuite.scala | 136 +++++++----------- .../spark/mllib/linalg/VectorsSuite.scala | 33 ++++- 8 files changed, 222 insertions(+), 269 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java index 76ebdccfd6b67..7b0ec36424e97 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java @@ -17,32 +17,33 @@ package org.apache.spark.mllib.examples; +import java.util.regex.Pattern; + import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.clustering.KMeans; import org.apache.spark.mllib.clustering.KMeansModel; - -import java.util.Arrays; -import java.util.regex.Pattern; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; /** * Example using MLLib KMeans from Java. */ public final class JavaKMeans { - static class ParsePoint implements Function { + private static class ParsePoint implements Function { private static final Pattern SPACE = Pattern.compile(" "); @Override - public double[] call(String line) { + public Vector call(String line) { String[] tok = SPACE.split(line); double[] point = new double[tok.length]; for (int i = 0; i < tok.length; ++i) { point[i] = Double.parseDouble(tok[i]); } - return point; + return Vectors.dense(point); } } @@ -65,15 +66,15 @@ public static void main(String[] args) { JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans", System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaKMeans.class)); - JavaRDD lines = sc.textFile(args[1]); + JavaRDD lines = sc.textFile(inputFile); - JavaRDD points = lines.map(new ParsePoint()); + JavaRDD points = lines.map(new ParsePoint()); - KMeansModel model = KMeans.train(points.rdd(), k, iterations, runs); + KMeansModel model = KMeans.train(points.rdd(), k, iterations, runs, KMeans.K_MEANS_PARALLEL()); System.out.println("Cluster centers:"); - for (double[] center : model.clusterCenters()) { - System.out.println(" " + Arrays.toString(center)); + for (Vector center : model.clusterCenters()) { + System.out.println(" " + center); } double cost = model.computeCost(points.rdd()); System.out.println("Cost: " + cost); diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index efe99a31beac4..3449c698da60b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -16,14 +16,16 @@ */ package org.apache.spark.mllib.api.python + +import java.nio.{ByteBuffer, ByteOrder} + import org.apache.spark.api.java.JavaRDD -import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.recommendation._ +import org.apache.spark.mllib.regression._ import org.apache.spark.rdd.RDD -import java.nio.ByteBuffer -import java.nio.ByteOrder /** * The Java stubs necessary for the Python mllib bindings. @@ -205,10 +207,10 @@ class PythonMLLibAPI extends Serializable { def trainKMeansModel(dataBytesJRDD: JavaRDD[Array[Byte]], k: Int, maxIterations: Int, runs: Int, initializationMode: String): java.util.List[java.lang.Object] = { - val data = dataBytesJRDD.rdd.map(xBytes => deserializeDoubleVector(xBytes)) + val data = dataBytesJRDD.rdd.map(xBytes => Vectors.dense(deserializeDoubleVector(xBytes))) val model = KMeans.train(data, k, maxIterations, runs, initializationMode) val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(serializeDoubleMatrix(model.clusterCenters)) + ret.add(serializeDoubleMatrix(model.clusterCenters.map(_.toArray))) ret } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index d09f29f882310..b412738e3f00a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable.ArrayBuffer import breeze.linalg.{DenseVector => BDV, Vector => BV, norm => breezeNorm} + import org.apache.spark.{Logging, SparkContext} import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.{Vector, Vectors} @@ -27,24 +28,6 @@ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom -/** - * A breeze vector with its norm for fast distance computation. - * - * @see [[org.apache.spark.mllib.clustering.KMeans#fastSquaredDistance]] - */ -private[clustering] -class BreezeVectorWithNorm(val vector: BV[Double], val norm: Double) extends Serializable { - - def this(vector: BV[Double]) = this(vector, breezeNorm(vector, 2.0)) - - def this(array: Array[Double]) = this(new BDV[Double](array)) - - def this(v: Vector) = this(v.toBreeze) - - /** Converts the vector to a dense vector. */ - def toDense = new BreezeVectorWithNorm(vector.toDenseVector, norm) -} - /** * K-means clustering with support for multiple parallel runs and a k-means++ like initialization * mode (the k-means|| algorithm by Bahmani et al). When multiple concurrent runs are requested, @@ -60,8 +43,7 @@ class KMeans private ( var initializationMode: String, var initializationSteps: Int, var epsilon: Double) - extends Serializable with Logging -{ + extends Serializable with Logging { def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 5, 1e-4) /** Set the number of clusters to create (k). Default: 2. */ @@ -127,15 +109,7 @@ class KMeans private ( * Train a K-means model on the given set of points; `data` should be cached for high * performance, because this is an iterative algorithm. */ - def run(data: RDD[Array[Double]]): KMeansModel = { - run(data.map(v => Vectors.dense(v))) - } - - /** - * Train a K-means model on the given set of points; `data` should be cached for high - * performance, because this is an iterative algorithm. - */ - def run(data: RDD[Vector])(implicit d: DummyImplicit): KMeansModel = { + def run(data: RDD[Vector]): KMeansModel = { // Compute squared norms and cache them. val norms = data.map(v => breezeNorm(v.toBreeze, 2.0)) norms.persist() @@ -248,9 +222,7 @@ class KMeans private ( logInfo(s"The cost for the best run is $minCost.") - new KMeansModel(centers(bestRun).map { v => - v.vector.toArray - }) + new KMeansModel(centers(bestRun).map(c => Vectors.fromBreeze(c.vector))) } /** @@ -332,36 +304,21 @@ object KMeans { val RANDOM = "random" val K_MEANS_PARALLEL = "k-means||" - def train( - data: RDD[Array[Double]], - k: Int, - maxIterations: Int, - runs: Int, - initializationMode: String) - : KMeansModel = - { - new KMeans().setK(k) - .setMaxIterations(maxIterations) - .setRuns(runs) - .setInitializationMode(initializationMode) - .run(data) - } - - def train(data: RDD[Array[Double]], k: Int, maxIterations: Int, runs: Int): KMeansModel = { - train(data, k, maxIterations, runs, K_MEANS_PARALLEL) - } - - def train(data: RDD[Array[Double]], k: Int, maxIterations: Int): KMeansModel = { - train(data, k, maxIterations, 1, K_MEANS_PARALLEL) - } - + /** + * Trains a k-means model using the given set of parameters. + * + * @param data training points stored as `RDD[Array[Double]]` + * @param k number of clusters + * @param maxIterations max number of iterations + * @param runs number of parallel runs, defaults to 1. The best model is returned. + * @param initializationMode initialization model, either "random" or "k-means||" (default). + */ def train( data: RDD[Vector], k: Int, maxIterations: Int, - runs: Int, - initializationMode: String - )(implicit d: DummyImplicit): KMeansModel = { + runs: Int = 1, + initializationMode: String = K_MEANS_PARALLEL): KMeansModel = { new KMeans().setK(k) .setMaxIterations(maxIterations) .setRuns(runs) @@ -369,16 +326,6 @@ object KMeans { .run(data) } - def train(data: RDD[Vector], k: Int, maxIterations: Int, runs: Int) - (implicit d: DummyImplicit): KMeansModel = { - train(data, k, maxIterations, runs, K_MEANS_PARALLEL) - } - - def train(data: RDD[Vector], k: Int, maxIterations: Int) - (implicit d: DummyImplicit): KMeansModel = { - train(data, k, maxIterations, 1, K_MEANS_PARALLEL) - } - /** * Returns the index of the closest center to the given point, as well as the squared distance. */ @@ -431,14 +378,34 @@ object KMeans { val (master, inputFile, k, iters) = (args(0), args(1), args(2).toInt, args(3).toInt) val runs = if (args.length >= 5) args(4).toInt else 1 val sc = new SparkContext(master, "KMeans") - val data = sc.textFile(inputFile).map(line => line.split(' ').map(_.toDouble)).cache() + val data = sc.textFile(inputFile) + .map(line => Vectors.dense(line.split(' ').map(_.toDouble))) + .cache() val model = KMeans.train(data, k, iters, runs) val cost = model.computeCost(data) println("Cluster centers:") for (c <- model.clusterCenters) { - println(" " + c.mkString(" ")) + println(" " + c) } println("Cost: " + cost) System.exit(0) } } + +/** + * A breeze vector with its norm for fast distance computation. + * + * @see [[org.apache.spark.mllib.clustering.KMeans#fastSquaredDistance]] + */ +private[clustering] +class BreezeVectorWithNorm(val vector: BV[Double], val norm: Double) extends Serializable { + + def this(vector: BV[Double]) = this(vector, breezeNorm(vector, 2.0)) + + def this(array: Array[Double]) = this(new BDV[Double](array)) + + def this(v: Vector) = this(v.toBreeze) + + /** Converts the vector to a dense vector. */ + def toDense = new BreezeVectorWithNorm(vector.toDenseVector, norm) +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index 5ec263ff44fef..18abbf2758b86 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -24,16 +24,11 @@ import org.apache.spark.mllib.linalg.Vector /** * A clustering model for K-means. Each point belongs to the cluster with the closest center. */ -class KMeansModel(val clusterCenters: Array[Array[Double]]) extends Serializable { +class KMeansModel(val clusterCenters: Array[Vector]) extends Serializable { /** Total number of clusters. */ def k: Int = clusterCenters.length - /** Return the cluster index that a given point belongs to. */ - def predict(point: Array[Double]): Int = { - KMeans.findClosest(clusterCentersWithNorm, new BreezeVectorWithNorm(point))._1 - } - /** Returns the cluster index that a given point belongs to. */ def predict(point: Vector): Int = { KMeans.findClosest(clusterCentersWithNorm, new BreezeVectorWithNorm(point))._1 @@ -49,16 +44,7 @@ class KMeansModel(val clusterCenters: Array[Array[Double]]) extends Serializable * 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[Array[Double]]): Double = { - val centersWithNorm = clusterCentersWithNorm - data.map(p => KMeans.pointCost(centersWithNorm, new BreezeVectorWithNorm(p))).sum() - } - - /** - * 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])(implicit d: DummyImplicit): Double = { + def computeCost(data: RDD[Vector]): Double = { val centersWithNorm = clusterCentersWithNorm data.map(p => KMeans.pointCost(centersWithNorm, new BreezeVectorWithNorm(p))).sum() } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 16a19df4472f4..f772ff32bebe6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -19,10 +19,10 @@ package org.apache.spark.mllib.linalg import java.lang.{Iterable => JavaIterable} +import scala.annotation.varargs import scala.collection.JavaConverters._ -import breeze.linalg.{Vector => BreezeVector, DenseVector => BreezeDenseVector, - SparseVector => BreezeSparseVector} +import breeze.linalg.{Vector => BV, DenseVector => BDV, SparseVector => BSV} /** * Represents a numeric vector, whose index type is Int and value type is Double. @@ -34,10 +34,25 @@ trait Vector extends Serializable { */ def size: Int + /** + * Converts the instance to a double array. + */ + def toArray: Array[Double] + + override def equals(other: Any): Boolean = { + other match { + case v: Vector => + this.toArray.view.equals(v.toArray.view) + case _ => false + } + } + + override def hashCode(): Int = toArray.view.hashCode() + /** * Converts the instance to a breeze vector. */ - private[mllib] def toBreeze: BreezeVector[Double] + private[mllib] def toBreeze: BV[Double] } /** @@ -53,8 +68,17 @@ trait RandomAccessVector extends Vector { */ object Vectors { - /** Creates a dense vector. */ - def dense(values: Array[Double]): Vector = new DenseVector(values) + /** + * Creates a dense vector. + */ + @varargs + def dense(values: Double*): Vector = new DenseVector(values.toArray) + + // A dummy implicit is used to avoid signature collision with the one generated by @varargs. + /** + * Creates a dense vector from a double array. + */ + def dense(values: Array[Double])(implicit d: DummyImplicit): Vector = new DenseVector(values) /** * Creates a sparse vector providing its index array and value array. @@ -79,7 +103,7 @@ object Vectors { val (indices, values) = elements.sortBy(_._1).unzip var prev = -1 indices.foreach { i => - require(prev < i, "Found duplicate indices: " + i) + require(prev < i, s"Found duplicate indices: $i.") prev = i } require(prev < size) @@ -93,41 +117,39 @@ object Vectors { * @param size vector size. * @param elements vector elements in (index, value) pairs. */ - def sparse(size: Int, elements: JavaIterable[(Int, Double)]): Vector = + def sparse(size: Int, elements: JavaIterable[(Int, Double)]): Vector = { sparse(size, elements.asScala.toSeq) + } /** * Creates a vector instance from a breeze vector. */ - private[mllib] def fromBreeze(breezeVector: BreezeVector[Double]): Vector = { + private[mllib] def fromBreeze(breezeVector: BV[Double]): Vector = { breezeVector match { - case v: BreezeDenseVector[Double] => { + case v: BDV[Double] => require(v.offset == 0, s"Do not support non-zero offset ${v.offset}.") require(v.stride == 1, s"Do not support stride other than 1, but got ${v.stride}.") new DenseVector(v.data) - } - case v: BreezeSparseVector[Double] => { + case v: BSV[Double] => new SparseVector(v.length, v.index, v.data) - } - case v: BreezeVector[_] => { + case v: BV[_] => sys.error("Unsupported Breeze vector type: " + v.getClass.getName) - } } } } /** * A dense vector represented by a value array. - * - * @param values */ -class DenseVector(var values: Array[Double]) extends RandomAccessVector { +class DenseVector(val values: Array[Double]) extends RandomAccessVector { override def size: Int = values.length - override def toString = values.mkString("[", ",", "]") + override def toString: String = values.mkString("[", ",", "]") - private[mllib] override def toBreeze = new BreezeDenseVector[Double](values) + override def toArray: Array[Double] = values + + private[mllib] override def toBreeze: BV[Double] = new BDV[Double](values) } /** @@ -137,13 +159,21 @@ class DenseVector(var values: Array[Double]) extends RandomAccessVector { * @param indices index array, assume to be strictly increasing. * @param values value array, must have the same length as the index array. */ -class SparseVector(var n: Int, var indices: Array[Int], var values: Array[Double]) extends Vector { +class SparseVector(val n: Int, val indices: Array[Int], val values: Array[Double]) extends Vector { override def size: Int = n - override def toString = { + override def toString: String = { "(" + n + "," + indices.zip(values).mkString("[", "," ,"]") + ")" } - private[mllib] override def toBreeze = new BreezeSparseVector[Double](indices, values, n) + override def toArray: Array[Double] = { + val data = new Array[Double](n) + indices.view.zip(values).foreach { case (i, x) => + data.update(i, x) + } + data + } + + private[mllib] override def toBreeze: BV[Double] = new BSV[Double](indices, values, n) } diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java index 33b99f4bd3bcf..49a614bd90cab 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java @@ -18,16 +18,19 @@ package org.apache.spark.mllib.clustering; import java.io.Serializable; -import java.util.ArrayList; import java.util.List; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import static org.junit.Assert.*; + +import com.google.common.collect.Lists; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; public class JavaKMeansSuite implements Serializable { private transient JavaSparkContext sc; @@ -44,72 +47,45 @@ public void tearDown() { System.clearProperty("spark.driver.port"); } - // L1 distance between two points - double distance1(double[] v1, double[] v2) { - double distance = 0.0; - for (int i = 0; i < v1.length; ++i) { - distance = Math.max(distance, Math.abs(v1[i] - v2[i])); - } - return distance; - } - - // Assert that two sets of points are equal, within EPSILON tolerance - void assertSetsEqual(double[][] v1, double[][] v2) { - double EPSILON = 1e-4; - Assert.assertTrue(v1.length == v2.length); - for (int i = 0; i < v1.length; ++i) { - double minDistance = Double.MAX_VALUE; - for (int j = 0; j < v2.length; ++j) { - minDistance = Math.min(minDistance, distance1(v1[i], v2[j])); - } - Assert.assertTrue(minDistance <= EPSILON); - } - - for (int i = 0; i < v2.length; ++i) { - double minDistance = Double.MAX_VALUE; - for (int j = 0; j < v1.length; ++j) { - minDistance = Math.min(minDistance, distance1(v2[i], v1[j])); - } - Assert.assertTrue(minDistance <= EPSILON); - } - } - - @Test public void runKMeansUsingStaticMethods() { - List points = new ArrayList(); - points.add(new double[]{1.0, 2.0, 6.0}); - points.add(new double[]{1.0, 3.0, 0.0}); - points.add(new double[]{1.0, 4.0, 6.0}); + List points = Lists.newArrayList( + Vectors.dense(1.0, 2.0, 6.0), + Vectors.dense(1.0, 3.0, 0.0), + Vectors.dense(1.0, 4.0, 6.0) + ); - double[][] expectedCenter = { {1.0, 3.0, 4.0} }; + Vector expectedCenter = Vectors.dense(1.0, 3.0, 4.0); - JavaRDD data = sc.parallelize(points, 2); - KMeansModel model = KMeans.train(data.rdd(), 1, 1); - assertSetsEqual(model.clusterCenters(), expectedCenter); + JavaRDD data = sc.parallelize(points, 2); + KMeansModel model = KMeans.train(data.rdd(), 1, 1, 1, KMeans.K_MEANS_PARALLEL()); + assertEquals(1, model.clusterCenters().length); + assertEquals(expectedCenter, model.clusterCenters()[0]); model = KMeans.train(data.rdd(), 1, 1, 1, KMeans.RANDOM()); - assertSetsEqual(model.clusterCenters(), expectedCenter); + assertEquals(expectedCenter, model.clusterCenters()[0]); } @Test public void runKMeansUsingConstructor() { - List points = new ArrayList(); - points.add(new double[]{1.0, 2.0, 6.0}); - points.add(new double[]{1.0, 3.0, 0.0}); - points.add(new double[]{1.0, 4.0, 6.0}); + List points = Lists.newArrayList( + Vectors.dense(1.0, 2.0, 6.0), + Vectors.dense(1.0, 3.0, 0.0), + Vectors.dense(1.0, 4.0, 6.0) + ); - double[][] expectedCenter = { {1.0, 3.0, 4.0} }; + Vector expectedCenter = Vectors.dense(1.0, 3.0, 4.0); - JavaRDD data = sc.parallelize(points, 2); + JavaRDD data = sc.parallelize(points, 2); KMeansModel model = new KMeans().setK(1).setMaxIterations(5).run(data.rdd()); - assertSetsEqual(model.clusterCenters(), expectedCenter); - - model = new KMeans().setK(1) - .setMaxIterations(1) - .setRuns(1) - .setInitializationMode(KMeans.RANDOM()) - .run(data.rdd()); - assertSetsEqual(model.clusterCenters(), expectedCenter); + assertEquals(1, model.clusterCenters().length); + assertEquals(expectedCenter, model.clusterCenters()[0]); + + model = new KMeans() + .setK(1) + .setMaxIterations(1) + .setInitializationMode(KMeans.RANDOM()) + .run(data.rdd()); + assertEquals(expectedCenter, model.clusterCenters()[0]); } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index e73ea9b53918e..560a4ad71a4de 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -24,110 +24,77 @@ import org.apache.spark.mllib.linalg.Vectors class KMeansSuite extends FunSuite with LocalSparkContext { - val EPSILON = 1e-4 - import KMeans.{RANDOM, K_MEANS_PARALLEL} - def prettyPrint(point: Array[Double]): String = point.mkString("(", ", ", ")") - - def prettyPrint(points: Array[Array[Double]]): String = { - points.map(prettyPrint).mkString("(", "; ", ")") - } - - // L1 distance between two points - def distance1(v1: Array[Double], v2: Array[Double]): Double = { - v1.zip(v2).map{ case (a, b) => math.abs(a-b) }.max - } - - // Assert that two vectors are equal within tolerance EPSILON - def assertEqual(v1: Array[Double], v2: Array[Double]) { - def errorMessage = prettyPrint(v1) + " did not equal " + prettyPrint(v2) - assert(v1.length == v2.length, errorMessage) - assert(distance1(v1, v2) <= EPSILON, errorMessage) - } - - // Assert that two sets of points are equal, within EPSILON tolerance - def assertSetsEqual(set1: Array[Array[Double]], set2: Array[Array[Double]]) { - def errorMessage = prettyPrint(set1) + " did not equal " + prettyPrint(set2) - assert(set1.length == set2.length, errorMessage) - for (v <- set1) { - val closestDistance = set2.map(w => distance1(v, w)).min - if (closestDistance > EPSILON) { - fail(errorMessage) - } - } - for (v <- set2) { - val closestDistance = set1.map(w => distance1(v, w)).min - if (closestDistance > EPSILON) { - fail(errorMessage) - } - } - } - test("single cluster") { val data = sc.parallelize(Array( - Array(1.0, 2.0, 6.0), - Array(1.0, 3.0, 0.0), - Array(1.0, 4.0, 6.0) + Vectors.dense(1.0, 2.0, 6.0), + Vectors.dense(1.0, 3.0, 0.0), + Vectors.dense(1.0, 4.0, 6.0) )) + val center = Vectors.dense(1.0, 3.0, 4.0) + // No matter how many runs or iterations we use, we should get one cluster, // centered at the mean of the points var model = KMeans.train(data, k=1, maxIterations=1) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=2) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=5) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train( data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) } test("single cluster with big dataset") { val smallData = Array( - Array(1.0, 2.0, 6.0), - Array(1.0, 3.0, 0.0), - Array(1.0, 4.0, 6.0) + Vectors.dense(1.0, 2.0, 6.0), + Vectors.dense(1.0, 3.0, 0.0), + Vectors.dense(1.0, 4.0, 6.0) ) val data = sc.parallelize((1 to 100).flatMap(_ => smallData), 4) // No matter how many runs or iterations we use, we should get one cluster, // centered at the mean of the points + val center = Vectors.dense(1.0, 3.0, 4.0) + var model = KMeans.train(data, k=1, maxIterations=1) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.size === 1) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=2) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=5) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) } test("single cluster with sparse data") { @@ -150,42 +117,39 @@ class KMeansSuite extends FunSuite with LocalSparkContext { // No matter how many runs or iterations we use, we should get one cluster, // centered at the mean of the points - val center = new Array[Double](n) - center(0) = 1.0 - center(1) = 3.0 - center(2) = 4.0 + val center = Vectors.sparse(n, Seq((0, 1.0), (1, 3.0), (2, 4.0))) var model = KMeans.train(data, k=1, maxIterations=1) - assertSetsEqual(model.clusterCenters, Array(center)) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=2) - assertSetsEqual(model.clusterCenters, Array(center)) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=5) - assertSetsEqual(model.clusterCenters, Array(center)) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assertSetsEqual(model.clusterCenters, Array(center)) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assertSetsEqual(model.clusterCenters, Array(center)) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) - assertSetsEqual(model.clusterCenters, Array(center)) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) - assertSetsEqual(model.clusterCenters, Array(center)) + assert(model.clusterCenters.head === center) data.unpersist() } test("k-means|| initialization") { - val points = Array( - Array(1.0, 2.0, 6.0), - Array(1.0, 3.0, 0.0), - Array(1.0, 4.0, 6.0), - Array(1.0, 0.0, 1.0), - Array(1.0, 1.0, 1.0) + val points = Seq( + Vectors.dense(1.0, 2.0, 6.0), + Vectors.dense(1.0, 3.0, 0.0), + Vectors.dense(1.0, 4.0, 6.0), + Vectors.dense(1.0, 0.0, 1.0), + Vectors.dense(1.0, 1.0, 1.0) ) val rdd = sc.parallelize(points) @@ -194,26 +158,26 @@ class KMeansSuite extends FunSuite with LocalSparkContext { // unselected point as long as it hasn't yet selected all of them var model = KMeans.train(rdd, k=5, maxIterations=1) - assertSetsEqual(model.clusterCenters, points) + assert(Set(model.clusterCenters: _*) === Set(points: _*)) // Iterations of Lloyd's should not change the answer either model = KMeans.train(rdd, k=5, maxIterations=10) - assertSetsEqual(model.clusterCenters, points) + assert(Set(model.clusterCenters: _*) === Set(points: _*)) // Neither should more runs model = KMeans.train(rdd, k=5, maxIterations=10, runs=5) - assertSetsEqual(model.clusterCenters, points) + assert(Set(model.clusterCenters: _*) === Set(points: _*)) } test("two clusters") { - val points = Array( - Array(0.0, 0.0), - Array(0.0, 0.1), - Array(0.1, 0.0), - Array(9.0, 0.0), - Array(9.0, 0.2), - Array(9.2, 0.0) - ).map(Vectors.dense) + val points = Seq( + Vectors.dense(0.0, 0.0), + Vectors.dense(0.0, 0.1), + Vectors.dense(0.1, 0.0), + Vectors.dense(9.0, 0.0), + Vectors.dense(9.0, 0.2), + Vectors.dense(9.2, 0.0) + ) val rdd = sc.parallelize(points, 3) for (initMode <- Seq(RANDOM, K_MEANS_PARALLEL)) { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index adf2005b84f1d..11056eb6d5611 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -28,10 +28,16 @@ class VectorsSuite extends FunSuite { val indices = Array(0, 3, 5, 10, 13) val values = Array(0.1, 0.5, 0.3, -0.8, -1.0) - test("dense vector construction") { + test("dense vector construction with varargs") { + val vec = Vectors.dense(arr: _*).asInstanceOf[DenseVector] + assert(vec.size === arr.length) + assert(vec.values.eq(arr)) + } + + test("dense vector construction from a double array") { val vec = Vectors.dense(arr).asInstanceOf[DenseVector] - assert(vec.size === arr.length) - assert(vec.values.eq(arr)) + assert(vec.size === arr.length) + assert(vec.values.eq(arr)) } test("sparse vector construction") { @@ -54,4 +60,25 @@ class VectorsSuite extends FunSuite { assert(vec.indices === indices) assert(vec.values === values) } + + test("vector equals") { + val dv1 = Vectors.dense(1.0, 0.0, 2.0) + val dv2 = Vectors.dense(1.0, 0.0, 2.0) + val sv1 = Vectors.sparse(3, Seq((0, 1.0), (2, 2.0))) + val sv2 = Vectors.sparse(3, Seq((0, 1.0), (2, 2.0))) + + val vectors = Seq(dv1, dv2, sv1, sv2) + + for (v <- vectors; u <- vectors) { + assert(v === u) + assert(v.## === u.##) + } + + val another = Vectors.dense(1.0, 1.0, 2.0) + + for (v <- vectors) { + assert(v != another) + assert(v.## != another.##) + } + } } From 238ba34b4b3a3a582035628464420f010321b31b Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 21 Mar 2014 14:56:15 -0700 Subject: [PATCH 18/24] add VectorRDDs with a converter from RDD[Array[Double]] --- .../spark/mllib/linalg/rdd/VectorRDDs.scala | 32 ++++++++++++++++++ .../mllib/linalg/rdd/VectorRDDsSuite.scala | 33 +++++++++++++++++++ 2 files changed, 65 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/VectorRDDs.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/VectorRDDsSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/VectorRDDs.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/VectorRDDs.scala new file mode 100644 index 0000000000000..35f1998002216 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/VectorRDDs.scala @@ -0,0 +1,32 @@ +/* + * 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.linalg.rdd + +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.{Vectors, Vector} + +/** + * Factory methods for `RDD[Vector]`. + */ +object VectorRDDs { + + /** + * Converts an `RDD[Array[Double]]` to `RDD[Vector]`. + */ + def fromArrayRDD(rdd: RDD[Array[Double]]): RDD[Vector] = rdd.map(v => Vectors.dense(v)) +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/VectorRDDsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/VectorRDDsSuite.scala new file mode 100644 index 0000000000000..dfbcf78318a54 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/VectorRDDsSuite.scala @@ -0,0 +1,33 @@ +/* + * 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.linalg.rdd + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.LocalSparkContext + +class VectorRDDsSuite extends FunSuite with LocalSparkContext { + + test("from array rdd") { + val data = Seq(Array(1.0, 2.0), Array(3.0, 4.0)) + val arrayRdd = sc.parallelize(data, 2) + val vectorRdd = VectorRDDs.fromArrayRDD(arrayRdd) + assert(arrayRdd.collect().map(v => Vectors.dense(v)).view === vectorRdd.collect().view) + } +} From 226d2cdd3d98b763dfc04970cdaaff8dfd5a9de3 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sat, 22 Mar 2014 10:41:34 -0700 Subject: [PATCH 19/24] update Java friendly methods in Vectors --- .../apache/spark/mllib/linalg/Vectors.scala | 31 ++++++------- .../spark/mllib/linalg/JavaVectorsSuite.java | 44 +++++++++++++++++++ .../spark/mllib/linalg/VectorsSuite.scala | 35 ++++++++------- 3 files changed, 75 insertions(+), 35 deletions(-) create mode 100644 mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index f772ff32bebe6..6696e7216f22e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -17,7 +17,8 @@ package org.apache.spark.mllib.linalg -import java.lang.{Iterable => JavaIterable} +import java.lang.{Iterable => JavaIterable, Integer => JavaInteger, Double => JavaDouble} +import java.util.Arrays import scala.annotation.varargs import scala.collection.JavaConverters._ @@ -42,12 +43,12 @@ trait Vector extends Serializable { override def equals(other: Any): Boolean = { other match { case v: Vector => - this.toArray.view.equals(v.toArray.view) + Arrays.equals(this.toArray, v.toArray) case _ => false } } - override def hashCode(): Int = toArray.view.hashCode() + override def hashCode(): Int = Arrays.hashCode(this.toArray) /** * Converts the instance to a breeze vector. @@ -55,14 +56,6 @@ trait Vector extends Serializable { private[mllib] def toBreeze: BV[Double] } -/** - * Represents a vector with random access to its elements. - * - */ -trait RandomAccessVector extends Vector { - // empty -} - /** * Factory methods for [[org.apache.spark.mllib.linalg.Vector]]. */ @@ -72,13 +65,14 @@ object Vectors { * Creates a dense vector. */ @varargs - def dense(values: Double*): Vector = new DenseVector(values.toArray) + def dense(firstValue: Double, otherValues: Double*): Vector = + new DenseVector((firstValue +: otherValues).toArray) // A dummy implicit is used to avoid signature collision with the one generated by @varargs. /** * Creates a dense vector from a double array. */ - def dense(values: Array[Double])(implicit d: DummyImplicit): Vector = new DenseVector(values) + def dense(values: Array[Double]): Vector = new DenseVector(values) /** * Creates a sparse vector providing its index array and value array. @@ -97,7 +91,6 @@ object Vectors { * @param elements vector elements in (index, value) pairs. */ def sparse(size: Int, elements: Seq[(Int, Double)]): Vector = { - require(size > 0) val (indices, values) = elements.sortBy(_._1).unzip @@ -112,13 +105,15 @@ object Vectors { } /** - * Creates a sparse vector using unordered (index, value) pairs. + * Creates a sparse vector using unordered (index, value) pairs in a Java friendly way. * * @param size vector size. * @param elements vector elements in (index, value) pairs. */ - def sparse(size: Int, elements: JavaIterable[(Int, Double)]): Vector = { - sparse(size, elements.asScala.toSeq) + def sparse(size: Int, elements: JavaIterable[(JavaInteger, JavaDouble)]): Vector = { + sparse(size, elements.asScala.map { case (i, x) => + (i.intValue(), x.doubleValue()) + }.toSeq) } /** @@ -141,7 +136,7 @@ object Vectors { /** * A dense vector represented by a value array. */ -class DenseVector(val values: Array[Double]) extends RandomAccessVector { +class DenseVector(val values: Array[Double]) extends Vector { override def size: Int = values.length diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java new file mode 100644 index 0000000000000..2c4d795f96e4e --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java @@ -0,0 +1,44 @@ +/* + * 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.linalg; + +import java.io.Serializable; + +import com.google.common.collect.Lists; + +import scala.Tuple2; + +import org.junit.Test; +import static org.junit.Assert.*; + +public class JavaVectorsSuite implements Serializable { + + @Test + public void denseArrayConstruction() { + Vector v = Vectors.dense(1.0, 2.0, 3.0); + assertArrayEquals(new double[]{1.0, 2.0, 3.0}, v.toArray(), 0.0); + } + + @Test + public void sparseArrayConstruction() { + Vector v = Vectors.sparse(3, Lists.newArrayList( + new Tuple2(0, 2.0), + new Tuple2(2, 3.0))); + assertArrayEquals(new double[]{2.0, 0.0, 3.0}, v.toArray(), 0.0); + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index 11056eb6d5611..8a200310e0bb1 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -17,19 +17,17 @@ package org.apache.spark.mllib.linalg -import scala.collection.JavaConverters._ - import org.scalatest.FunSuite class VectorsSuite extends FunSuite { - val arr = Array(0.1, 0.2, 0.3, 0.4) - val n = 20 - val indices = Array(0, 3, 5, 10, 13) - val values = Array(0.1, 0.5, 0.3, -0.8, -1.0) + val arr = Array(0.1, 0.0, 0.3, 0.4) + val n = 4 + val indices = Array(0, 2, 3) + val values = Array(0.1, 0.3, 0.4) test("dense vector construction with varargs") { - val vec = Vectors.dense(arr: _*).asInstanceOf[DenseVector] + val vec = Vectors.dense(arr).asInstanceOf[DenseVector] assert(vec.size === arr.length) assert(vec.values.eq(arr)) } @@ -54,18 +52,21 @@ class VectorsSuite extends FunSuite { assert(vec.values === values) } - test("sparse vector construction with unordered elements stored as Java Iterable") { - val vec = Vectors.sparse(n, indices.toSeq.zip(values).reverse.asJava).asInstanceOf[SparseVector] - assert(vec.size === n) - assert(vec.indices === indices) - assert(vec.values === values) + test("dense to array") { + val vec = Vectors.dense(arr).asInstanceOf[DenseVector] + assert(vec.toArray.eq(arr)) + } + + test("sparse to array") { + val vec = Vectors.sparse(n, indices, values).asInstanceOf[SparseVector] + assert(vec.toArray === arr) } test("vector equals") { - val dv1 = Vectors.dense(1.0, 0.0, 2.0) - val dv2 = Vectors.dense(1.0, 0.0, 2.0) - val sv1 = Vectors.sparse(3, Seq((0, 1.0), (2, 2.0))) - val sv2 = Vectors.sparse(3, Seq((0, 1.0), (2, 2.0))) + val dv1 = Vectors.dense(arr.clone()) + val dv2 = Vectors.dense(arr.clone()) + val sv1 = Vectors.sparse(n, indices.clone(), values.clone()) + val sv2 = Vectors.sparse(n, indices.clone(), values.clone()) val vectors = Seq(dv1, dv2, sv1, sv2) @@ -74,7 +75,7 @@ class VectorsSuite extends FunSuite { assert(v.## === u.##) } - val another = Vectors.dense(1.0, 1.0, 2.0) + val another = Vectors.dense(0.1, 0.2, 0.3, 0.4) for (v <- vectors) { assert(v != another) From 9bb1b3148c349e03018507c46376eaa62fa1c0a4 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sat, 22 Mar 2014 10:47:31 -0700 Subject: [PATCH 20/24] optimize SparseVector.toArray --- .../main/scala/org/apache/spark/mllib/linalg/Vectors.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 6696e7216f22e..24f04eb3dd15b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -164,8 +164,10 @@ class SparseVector(val n: Int, val indices: Array[Int], val values: Array[Double override def toArray: Array[Double] = { val data = new Array[Double](n) - indices.view.zip(values).foreach { case (i, x) => - data.update(i, x) + var i = 0 + while (i < n) { + data(indices(i)) = values(i) + i += 1 } data } From 1da1033fcdd355c8a4ef97e70b36b9b054eee60c Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sat, 22 Mar 2014 11:01:28 -0700 Subject: [PATCH 21/24] remove dependency on commons-math3 and compute EPSILON directly --- .../scala/org/apache/spark/mllib/util/MLUtils.scala | 10 ++++++++-- .../org/apache/spark/mllib/util/MLUtilsSuite.scala | 5 +++++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 55196f2aa2243..08cd9ab05547b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -21,8 +21,6 @@ import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ -import org.apache.commons.math3.util.Precision.EPSILON - import org.jblas.DoubleMatrix import org.apache.spark.mllib.regression.LabeledPoint @@ -34,6 +32,14 @@ import breeze.linalg.{Vector => BV, SparseVector => BSV, squaredDistance => bree */ object MLUtils { + private[util] lazy val EPSILON = { + var eps = 1.0 + while ((1.0 + (eps / 2.0)) != 1.0) { + eps /= 2.0 + } + eps + } + /** * Load labeled data from a file. The data format used here is * , ... diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index f2ac4205472ed..60f053b381305 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -26,6 +26,11 @@ import org.apache.spark.mllib.util.MLUtils._ class MLUtilsSuite extends FunSuite { + test("epsilon computation") { + assert(1.0 + EPSILON > 1.0, s"EPSILON is too small: $EPSILON.") + assert(1.0 + EPSILON / 2.0 === 1.0, s"EPSILON is too big: $EPSILON.") + } + test("fast squared distance") { val a = (30 to 0 by -1).map(math.pow(2.0, _)).toArray val n = a.length From 67abe319a3295a60554ad5aae5d5df5922ddee4b Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sat, 22 Mar 2014 11:03:29 -0700 Subject: [PATCH 22/24] move ArrayRDDs to mllib.rdd --- .../org/apache/spark/mllib/{linalg => }/rdd/VectorRDDs.scala | 2 +- .../apache/spark/mllib/{linalg => }/rdd/VectorRDDsSuite.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) rename mllib/src/main/scala/org/apache/spark/mllib/{linalg => }/rdd/VectorRDDs.scala (96%) rename mllib/src/test/scala/org/apache/spark/mllib/{linalg => }/rdd/VectorRDDsSuite.scala (89%) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/VectorRDDs.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/VectorRDDs.scala similarity index 96% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/VectorRDDs.scala rename to mllib/src/main/scala/org/apache/spark/mllib/rdd/VectorRDDs.scala index 35f1998002216..9096d6a1a16d6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/rdd/VectorRDDs.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/VectorRDDs.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.linalg.rdd +package org.apache.spark.mllib.rdd import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.{Vectors, Vector} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/VectorRDDsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/rdd/VectorRDDsSuite.scala similarity index 89% rename from mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/VectorRDDsSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/rdd/VectorRDDsSuite.scala index dfbcf78318a54..692f025e959ae 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/rdd/VectorRDDsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/rdd/VectorRDDsSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.linalg.rdd +package org.apache.spark.mllib.rdd import org.scalatest.FunSuite @@ -28,6 +28,6 @@ class VectorRDDsSuite extends FunSuite with LocalSparkContext { val data = Seq(Array(1.0, 2.0), Array(3.0, 4.0)) val arrayRdd = sc.parallelize(data, 2) val vectorRdd = VectorRDDs.fromArrayRDD(arrayRdd) - assert(arrayRdd.collect().map(v => Vectors.dense(v)).view === vectorRdd.collect().view) + assert(arrayRdd.collect().map(v => Vectors.dense(v)) === vectorRdd.collect()) } } From 5eda0ded197ed626ccb70a4a3f2fc107537b8dc2 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sat, 22 Mar 2014 21:13:32 -0700 Subject: [PATCH 23/24] update NOTICE --- NOTICE | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/NOTICE b/NOTICE index 25bcb0b8ee3a3..42f6c3a835725 100644 --- a/NOTICE +++ b/NOTICE @@ -1,14 +1,14 @@ Apache Spark -Copyright 2013 The Apache Software Foundation. +Copyright 2014 The Apache Software Foundation. This product includes software developed at The Apache Software Foundation (http://www.apache.org/). -Numerical linear algebra support in MLlib is provided by the breeze package, -which depends on the following package that are not distributed under -an Apache authorized license: +In addition, this product includes: -- JTransforms, which is open source software written by Piotr Wendykier, - and distributed under the the terms of the MPL/LGPL/GPL tri-license. - The original software is available from - https://sites.google.com/site/piotrwendykier/software/jtransforms +- JUnit (http://www.junit.org) is a testing framework for Java. We included it + under the terms of the Eclipse Public License v1.0. + +- JTransforms (https://sites.google.com/site/piotrwendykier/software/jtransforms) + provides fast transforms in Java. It is tri-licensed, and we included it under + the terms of the Mozilla Public License v1.1. From 67b368db1db2b31cbfc6d5b100a21066a5b87f5a Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sat, 22 Mar 2014 21:18:34 -0700 Subject: [PATCH 24/24] fix SparseVector.toArray --- .../src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 24f04eb3dd15b..01c1501548f87 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -165,7 +165,8 @@ class SparseVector(val n: Int, val indices: Array[Int], val values: Array[Double override def toArray: Array[Double] = { val data = new Array[Double](n) var i = 0 - while (i < n) { + val nnz = indices.length + while (i < nnz) { data(indices(i)) = values(i) i += 1 }