From 010d07692f40d42486cc53a053173f7e7bd6d44a Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 15 Jul 2014 10:34:58 -0700 Subject: [PATCH 01/14] modify NaiveBayesModel and GLM to use broadcast --- .../apache/spark/mllib/classification/NaiveBayes.scala | 8 +++++++- .../mllib/regression/GeneralizedLinearAlgorithm.scala | 7 +++++-- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index b6e0c4a80e27..6c7be0a4f1dc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -54,7 +54,13 @@ class NaiveBayesModel private[mllib] ( } } - override def predict(testData: RDD[Vector]): RDD[Double] = testData.map(predict) + override def predict(testData: RDD[Vector]): RDD[Double] = { + val bcModel = testData.context.broadcast(this) + testData.mapPartitions { iter => + val model = bcModel.value + iter.map(model.predict) + } + } override def predict(testData: Vector): Double = { labels(brzArgmax(brzPi + brzTheta * testData.toBreeze)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index fe41863bce98..54854252d747 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -56,9 +56,12 @@ abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double // A small optimization to avoid serializing the entire model. Only the weightsMatrix // and intercept is needed. val localWeights = weights + val bcWeights = testData.context.broadcast(localWeights) val localIntercept = intercept - - testData.map(v => predictPoint(v, localWeights, localIntercept)) + testData.mapPartitions { iter => + val w = bcWeights.value + iter.map(v => predictPoint(v, w, localIntercept)) + } } /** From e00c2daa95c36db509006398ebdcb88874c525df Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 15 Jul 2014 13:27:08 -0700 Subject: [PATCH 02/14] use broadcast in GD, KMeans --- .../spark/mllib/clustering/KMeans.scala | 19 ++++++++++++------- .../spark/mllib/clustering/KMeansModel.scala | 6 ++++-- .../mllib/optimization/GradientDescent.scala | 6 ++++-- 3 files changed, 20 insertions(+), 11 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 de22fbb6ffc1..09dcea509eb7 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 @@ -165,18 +165,21 @@ class KMeans private ( val activeCenters = activeRuns.map(r => centers(r)).toArray val costAccums = activeRuns.map(_ => sc.accumulator(0.0)) + val bcActiveCenters = data.context.broadcast(activeCenters) + // Find the sum and count of points mapping to each center val totalContribs = data.mapPartitions { points => - val runs = activeCenters.length - val k = activeCenters(0).length - val dims = activeCenters(0)(0).vector.length + val thisActiveCenters = bcActiveCenters.value + val runs = thisActiveCenters.length + val k = thisActiveCenters(0).length + val dims = thisActiveCenters(0)(0).vector.length val sums = Array.fill(runs, k)(BDV.zeros[Double](dims).asInstanceOf[BV[Double]]) val counts = Array.fill(runs, k)(0L) points.foreach { point => (0 until runs).foreach { i => - val (bestCenter, cost) = KMeans.findClosest(activeCenters(i), point) + val (bestCenter, cost) = KMeans.findClosest(thisActiveCenters(i), point) costAccums(i) += cost sums(i)(bestCenter) += point.vector counts(i)(bestCenter) += 1 @@ -264,16 +267,17 @@ class KMeans private ( // to their squared distance from that run's current centers var step = 0 while (step < initializationSteps) { + val bcCenters = data.context.broadcast(centers) val sumCosts = data.flatMap { point => (0 until runs).map { r => - (r, KMeans.pointCost(centers(r), point)) + (r, KMeans.pointCost(bcCenters.value(r), point)) } }.reduceByKey(_ + _).collectAsMap() val chosen = data.mapPartitionsWithIndex { (index, points) => val rand = new XORShiftRandom(seed ^ (step << 16) ^ index) points.flatMap { p => (0 until runs).filter { r => - rand.nextDouble() < 2.0 * KMeans.pointCost(centers(r), p) * k / sumCosts(r) + rand.nextDouble() < 2.0 * KMeans.pointCost(bcCenters.value(r), p) * k / sumCosts(r) }.map((_, p)) } }.collect() @@ -286,9 +290,10 @@ class KMeans private ( // 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 bcCenters = data.context.broadcast(centers) val weightMap = data.flatMap { p => (0 until runs).map { r => - ((r, KMeans.findClosest(centers(r), p)._1), 1.0) + ((r, KMeans.findClosest(bcCenters.value(r), p)._1), 1.0) } }.reduceByKey(_ + _).collectAsMap() val finalCenters = (0 until runs).map { r => 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 fba21aefaaac..5823cb6e52e7 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 @@ -38,7 +38,8 @@ class KMeansModel private[mllib] (val clusterCenters: Array[Vector]) extends Ser /** Maps given points to their cluster indices. */ def predict(points: RDD[Vector]): RDD[Int] = { val centersWithNorm = clusterCentersWithNorm - points.map(p => KMeans.findClosest(centersWithNorm, new BreezeVectorWithNorm(p))._1) + val bcCentersWithNorm = points.context.broadcast(centersWithNorm) + points.map(p => KMeans.findClosest(bcCentersWithNorm.value, new BreezeVectorWithNorm(p))._1) } /** Maps given points to their cluster indices. */ @@ -51,7 +52,8 @@ class KMeansModel private[mllib] (val clusterCenters: Array[Vector]) extends Ser */ def computeCost(data: RDD[Vector]): Double = { val centersWithNorm = clusterCentersWithNorm - data.map(p => KMeans.pointCost(centersWithNorm, new BreezeVectorWithNorm(p))).sum() + val bcCentersWithNorm = data.context.broadcast(centersWithNorm) + data.map(p => KMeans.pointCost(bcCentersWithNorm.value, new BreezeVectorWithNorm(p))).sum() } private def clusterCentersWithNorm: Iterable[BreezeVectorWithNorm] = diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index 7030eeabe400..9fd760bf7808 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -163,6 +163,7 @@ object GradientDescent extends Logging { // Initialize weights as a column vector var weights = Vectors.dense(initialWeights.toArray) + val n = weights.size /** * For the first iteration, the regVal will be initialized as sum of weight squares @@ -172,12 +173,13 @@ object GradientDescent extends Logging { weights, Vectors.dense(new Array[Double](weights.size)), 0, 1, regParam)._2 for (i <- 1 to numIterations) { + val bcWeights = data.context.broadcast(weights) // Sample a subset (fraction miniBatchFraction) of the total data // compute and sum up the subgradients on this subset (this is one map-reduce) val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42 + i) - .aggregate((BDV.zeros[Double](weights.size), 0.0))( + .aggregate((BDV.zeros[Double](n), 0.0))( seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) => - val l = gradient.compute(features, label, weights, Vectors.fromBreeze(grad)) + val l = gradient.compute(features, label, bcWeights.value, Vectors.fromBreeze(grad)) (grad, loss + l) }, combOp = (c1, c2) => (c1, c2) match { case ((grad1, loss1), (grad2, loss2)) => From 1928a5a6d0c989b0124ce025e3a7245627cc4bb0 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 15 Jul 2014 14:02:46 -0700 Subject: [PATCH 03/14] add test for KMeans task size --- .../spark/mllib/clustering/KMeans.scala | 2 +- .../spark/mllib/clustering/KMeansSuite.scala | 30 +++++++++++++++++-- 2 files changed, 29 insertions(+), 3 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 09dcea509eb7..db425d866bba 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 @@ -165,7 +165,7 @@ class KMeans private ( val activeCenters = activeRuns.map(r => centers(r)).toArray val costAccums = activeRuns.map(_ => sc.accumulator(0.0)) - val bcActiveCenters = data.context.broadcast(activeCenters) + val bcActiveCenters = sc.broadcast(activeCenters) // Find the sum and count of points mapping to each center val totalContribs = data.mapPartitions { points => 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 560a4ad71a4d..00c194530352 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,14 +17,17 @@ package org.apache.spark.mllib.clustering +import java.util.Random + import org.scalatest.FunSuite -import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.LocalSparkContext class KMeansSuite extends FunSuite with LocalSparkContext { - import KMeans.{RANDOM, K_MEANS_PARALLEL} + import org.apache.spark.mllib.clustering.KMeans.{K_MEANS_PARALLEL, RANDOM} test("single cluster") { val data = sc.parallelize(Array( @@ -194,3 +197,26 @@ class KMeansSuite extends FunSuite with LocalSparkContext { } } } + +class KMeansTaskSuite extends FunSuite { + + test("task size should be small in both training and prediction") { + val conf = new SparkConf() + .setMaster("local-cluster[2, 1, 512]") + .setAppName("test k-means task size") + .set("spark.akka.frameSize", "1") + val sc = new SparkContext(conf) + val m = 10 + val n = 1000000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => Vectors.dense(Array.fill(n)(random.nextDouble))) + }.cache() + for (initMode <- Seq(KMeans.RANDOM, KMeans.K_MEANS_PARALLEL)) { + val model = KMeans.train(points, 2, 2, 1, initMode) + val predictions = model.predict(points).collect() + val cost = model.computeCost(points) + } + sc.stop() + } +} From 174cb157c208e359455872da1c270a8ed8ccf7d2 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 15 Jul 2014 16:30:47 -0700 Subject: [PATCH 04/14] use local-cluster for test with a small akka.frameSize --- .../org/apache/spark/mllib/util/LocalSparkContext.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) 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 0d4868f3d9e4..2394fc13260c 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 @@ -20,13 +20,17 @@ package org.apache.spark.mllib.util import org.scalatest.Suite import org.scalatest.BeforeAndAfterAll -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} trait LocalSparkContext extends BeforeAndAfterAll { self: Suite => @transient var sc: SparkContext = _ override def beforeAll() { - sc = new SparkContext("local", "test") + val conf = new SparkConf() + .setMaster("local-cluster[2, 1, 512]") + .setAppName("test") + .set("spark.akka.frameSize", "1") // set to 1MB to detect direct serialization of data + sc = new SparkContext(conf) super.beforeAll() } From e73d68edc2b93f41749d95434a651d0f02c7ab55 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 15 Jul 2014 16:31:10 -0700 Subject: [PATCH 05/14] update tests for k-means --- .../spark/mllib/clustering/KMeansSuite.scala | 15 ++++----------- 1 file changed, 4 insertions(+), 11 deletions(-) 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 00c194530352..ecbc03948089 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 @@ -196,27 +196,20 @@ class KMeansSuite extends FunSuite with LocalSparkContext { assert(predicts(0) != predicts(3)) } } -} - -class KMeansTaskSuite extends FunSuite { test("task size should be small in both training and prediction") { - val conf = new SparkConf() - .setMaster("local-cluster[2, 1, 512]") - .setAppName("test k-means task size") - .set("spark.akka.frameSize", "1") - val sc = new SparkContext(conf) - val m = 10 - val n = 1000000 + val m = 4 + val n = 200000 val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => val random = new Random(idx) iter.map(i => Vectors.dense(Array.fill(n)(random.nextDouble))) }.cache() for (initMode <- Seq(KMeans.RANDOM, KMeans.K_MEANS_PARALLEL)) { + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. val model = KMeans.train(points, 2, 2, 1, initMode) val predictions = model.predict(points).collect() val cost = model.computeCost(points) } - sc.stop() } } From 02103ba017e22f7b9a288dc21635eca998b96068 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 15 Jul 2014 16:31:31 -0700 Subject: [PATCH 06/14] remove print --- .../spark/mllib/classification/JavaLogisticRegressionSuite.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java index faa675b59cd5..862221d48798 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java @@ -92,8 +92,6 @@ public void runLRUsingStaticMethods() { testRDD.rdd(), 100, 1.0, 1.0); int numAccurate = validatePrediction(validationData, model); - System.out.println(numAccurate); Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); } - } From bccab926c6d3a62e57c536de1ae344dbf21a35c5 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 15 Jul 2014 18:58:16 -0700 Subject: [PATCH 07/14] add task size test to LBFGS --- .../spark/mllib/optimization/LBFGS.scala | 25 ++++++----- .../spark/mllib/optimization/LBFGSSuite.scala | 28 +++++++++++-- .../mllib/util/LocalClusterSparkContext.scala | 42 +++++++++++++++++++ .../spark/mllib/util/LocalSparkContext.scala | 3 +- 4 files changed, 81 insertions(+), 17 deletions(-) create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index 7bbed9c8fdbe..26d98a6edf46 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -195,18 +195,21 @@ object LBFGS extends Logging { override def calculate(weights: BDV[Double]) = { // Have a local copy to avoid the serialization of CostFun object which is not serializable. - val localData = data val localGradient = gradient - - val (gradientSum, lossSum) = localData.aggregate((BDV.zeros[Double](weights.size), 0.0))( - seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) => - val l = localGradient.compute( - features, label, Vectors.fromBreeze(weights), Vectors.fromBreeze(grad)) - (grad, loss + l) - }, - combOp = (c1, c2) => (c1, c2) match { case ((grad1, loss1), (grad2, loss2)) => - (grad1 += grad2, loss1 + loss2) - }) + val n = weights.length + val bcWeights = data.context.broadcast(weights) + + val (gradientSum, lossSum) = data.mapPartitions { iter => + val cumGrad = Vectors.dense(new Array[Double](n)) + val thisWeights = Vectors.fromBreeze(bcWeights.value) + var loss = 0.0 + iter.foreach { case (label, features) => + loss += localGradient.compute(features, label, thisWeights, cumGrad) + } + Iterator((cumGrad.toBreeze.asInstanceOf[BDV[Double]], loss)) + }.reduce { case ((grad1, loss1), (grad2, loss2)) => + (grad1 += grad2, loss1 + loss2) + } /** * regVal is sum of weight squares if it's L2 updater; diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala index fe7a9033cd5f..cc63b51efcf1 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala @@ -17,12 +17,13 @@ package org.apache.spark.mllib.optimization -import org.scalatest.FunSuite -import org.scalatest.Matchers +import java.util.Random + +import org.scalatest.{FunSuite, Matchers} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { @@ -230,3 +231,22 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { "The weight differences between LBFGS and GD should be within 2%.") } } + +class LBFGSTaskSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small") { + val m = 10 + val n = 200000 + val examples = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => (1.0, Vectors.dense(Array.fill(n)(random.nextDouble)))) + }.cache() + val lbfgs = new LBFGS(new LogisticGradient, new SquaredL2Updater) + .setNumCorrections(1) + .setConvergenceTol(1e-12) + .setMaxNumIterations(1) + .setRegParam(1.0) + val random = new Random(0) + val weights = lbfgs.optimize(examples, Vectors.dense(Array.fill(n)(random.nextDouble))) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala new file mode 100644 index 000000000000..5e9101cdd380 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala @@ -0,0 +1,42 @@ +/* + * 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, BeforeAndAfterAll} + +import org.apache.spark.{SparkConf, SparkContext} + +trait LocalClusterSparkContext extends BeforeAndAfterAll { self: Suite => + @transient var sc: SparkContext = _ + + override def beforeAll() { + val conf = new SparkConf() + .setMaster("local-cluster[2, 1, 512]") + .setAppName("test-cluster") + .set("spark.akka.frameSize", "1") // set to 1MB to detect direct serialization of data + sc = new SparkContext(conf) + super.beforeAll() + } + + override def afterAll() { + if (sc != null) { + sc.stop() + } + super.afterAll() + } +} 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 2394fc13260c..7857d9e5ee5c 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 @@ -27,9 +27,8 @@ trait LocalSparkContext extends BeforeAndAfterAll { self: Suite => override def beforeAll() { val conf = new SparkConf() - .setMaster("local-cluster[2, 1, 512]") + .setMaster("local") .setAppName("test") - .set("spark.akka.frameSize", "1") // set to 1MB to detect direct serialization of data sc = new SparkContext(conf) super.beforeAll() } From 380778cc09dc64a904032652aabbf787421f188f Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 15 Jul 2014 19:02:11 -0700 Subject: [PATCH 08/14] update KMeans test --- .../spark/mllib/clustering/KMeansSuite.scala | 54 ++++++++++--------- .../spark/mllib/optimization/LBFGSSuite.scala | 4 +- 2 files changed, 31 insertions(+), 27 deletions(-) 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 ecbc03948089..4858b6e2ffda 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 @@ -21,9 +21,8 @@ import java.util.Random import org.scalatest.FunSuite -import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} class KMeansSuite extends FunSuite with LocalSparkContext { @@ -41,26 +40,26 @@ 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 - var model = KMeans.train(data, k=1, maxIterations=1) + var model = KMeans.train(data, k = 1, maxIterations = 1) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=2) + model = KMeans.train(data, k = 1, maxIterations = 2) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=5) + model = KMeans.train(data, k = 1, maxIterations = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) assert(model.clusterCenters.head === center) model = KMeans.train( - data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) + data, k = 1, maxIterations = 1, runs = 1, initializationMode = K_MEANS_PARALLEL) assert(model.clusterCenters.head === center) } @@ -77,26 +76,26 @@ class KMeansSuite extends FunSuite with LocalSparkContext { val center = Vectors.dense(1.0, 3.0, 4.0) - var model = KMeans.train(data, k=1, maxIterations=1) + var model = KMeans.train(data, k = 1, maxIterations = 1) assert(model.clusterCenters.size === 1) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=2) + model = KMeans.train(data, k = 1, maxIterations = 2) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=5) + model = KMeans.train(data, k = 1, maxIterations = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = K_MEANS_PARALLEL) assert(model.clusterCenters.head === center) } @@ -122,25 +121,25 @@ class KMeansSuite extends FunSuite with LocalSparkContext { val center = Vectors.sparse(n, Seq((0, 1.0), (1, 3.0), (2, 4.0))) - var model = KMeans.train(data, k=1, maxIterations=1) + var model = KMeans.train(data, k = 1, maxIterations = 1) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=2) + model = KMeans.train(data, k = 1, maxIterations = 2) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=5) + model = KMeans.train(data, k = 1, maxIterations = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = K_MEANS_PARALLEL) assert(model.clusterCenters.head === center) data.unpersist() @@ -160,15 +159,15 @@ class KMeansSuite extends FunSuite with LocalSparkContext { // it will make at least five passes, and it will give non-zero probability to each // unselected point as long as it hasn't yet selected all of them - var model = KMeans.train(rdd, k=5, maxIterations=1) + var model = KMeans.train(rdd, k = 5, maxIterations = 1) assert(Set(model.clusterCenters: _*) === Set(points: _*)) // Iterations of Lloyd's should not change the answer either - model = KMeans.train(rdd, k=5, maxIterations=10) + model = KMeans.train(rdd, k = 5, maxIterations = 10) assert(Set(model.clusterCenters: _*) === Set(points: _*)) // Neither should more runs - model = KMeans.train(rdd, k=5, maxIterations=10, runs=5) + model = KMeans.train(rdd, k = 5, maxIterations = 10, runs = 5) assert(Set(model.clusterCenters: _*) === Set(points: _*)) } @@ -196,6 +195,9 @@ class KMeansSuite extends FunSuite with LocalSparkContext { assert(predicts(0) != predicts(3)) } } +} + +class KMeansClusterSuite extends FunSuite with LocalClusterSparkContext { test("task size should be small in both training and prediction") { val m = 4 diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala index cc63b51efcf1..da263b569db6 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala @@ -232,7 +232,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { } } -class LBFGSTaskSuite extends FunSuite with LocalClusterSparkContext { +class LBFGSClusterSuite extends FunSuite with LocalClusterSparkContext { test("task size should be small") { val m = 10 @@ -247,6 +247,8 @@ class LBFGSTaskSuite extends FunSuite with LocalClusterSparkContext { .setMaxNumIterations(1) .setRegParam(1.0) val random = new Random(0) + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. val weights = lbfgs.optimize(examples, Vectors.dense(Array.fill(n)(random.nextDouble))) } } From 28a841123b1eb02dc495d1dd422a11a31a6b5456 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 15 Jul 2014 19:10:31 -0700 Subject: [PATCH 09/14] add test for NaiveBayes --- .../classification/NaiveBayesSuite.scala | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala index 516895d04222..06cdd04f5fda 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala @@ -23,7 +23,7 @@ import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} object NaiveBayesSuite { @@ -96,3 +96,21 @@ class NaiveBayesSuite extends FunSuite with LocalSparkContext { validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } } + +class NaiveBayesClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 10 + val n = 200000 + val examples = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map { i => + LabeledPoint(random.nextInt(2), Vectors.dense(Array.fill(n)(random.nextDouble()))) + } + } + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = NaiveBayes.train(examples) + val predictions = model.predict(examples.map(_.features)) + } +} From e0a5cf2876ccef1ad0e6be2c805cfacee849468b Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 15 Jul 2014 19:18:41 -0700 Subject: [PATCH 10/14] add task size test to GD --- .../optimization/GradientDescentSuite.scala | 34 +++++++++++++++---- 1 file changed, 28 insertions(+), 6 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala index 951b4f7c6e6f..dfb2eb7f0d14 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala @@ -17,15 +17,14 @@ package org.apache.spark.mllib.optimization -import scala.util.Random import scala.collection.JavaConversions._ +import scala.util.Random -import org.scalatest.FunSuite -import org.scalatest.Matchers +import org.scalatest.{FunSuite, Matchers} -import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression._ +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} object GradientDescentSuite { @@ -46,7 +45,7 @@ object GradientDescentSuite { val rnd = new Random(seed) val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - val unifRand = new scala.util.Random(45) + val unifRand = new Random(45) val rLogis = (0 until nPoints).map { i => val u = unifRand.nextDouble() math.log(u) - math.log(1.0-u) @@ -144,3 +143,26 @@ class GradientDescentSuite extends FunSuite with LocalSparkContext with Matchers "should be initialWeightsWithIntercept.") } } + +class GradientDescentClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => (1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val (weights, loss) = GradientDescent.runMiniBatchSGD( + points, + new LogisticGradient, + new SquaredL2Updater, + 0.1, + 2, + 1.0, + 1.0, + Vectors.dense(new Array[Double](n))) + } +} From 9427bf0cd3df6fb9728462dbf1c909119ec9a972 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 15 Jul 2014 19:31:34 -0700 Subject: [PATCH 11/14] add task size tests to linear methods --- .../LogisticRegressionSuite.scala | 18 ++++++++++++- .../spark/mllib/classification/SVMSuite.scala | 25 +++++++++++++++---- .../spark/mllib/clustering/KMeansSuite.scala | 6 +++-- .../spark/mllib/regression/LassoSuite.scala | 21 +++++++++++++++- .../regression/LinearRegressionSuite.scala | 21 +++++++++++++++- .../regression/RidgeRegressionSuite.scala | 23 +++++++++++++++-- 6 files changed, 102 insertions(+), 12 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index 44b757b6a1fb..3f6ff859374c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -25,7 +25,7 @@ import org.scalatest.Matchers import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} object LogisticRegressionSuite { @@ -126,3 +126,19 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Match validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } } + +class LogisticRegressionClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = LogisticRegressionWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala index 886c71dde3af..65e5df58db4c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala @@ -17,17 +17,16 @@ package org.apache.spark.mllib.classification -import scala.util.Random import scala.collection.JavaConversions._ - -import org.scalatest.FunSuite +import scala.util.Random import org.jblas.DoubleMatrix +import org.scalatest.FunSuite import org.apache.spark.SparkException -import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression._ +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} object SVMSuite { @@ -193,3 +192,19 @@ class SVMSuite extends FunSuite with LocalSparkContext { new SVMWithSGD().setValidateData(false).run(testRDDInvalid) } } + +class SVMClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = SVMWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + } +} 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 4858b6e2ffda..80e6235657f7 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 @@ -95,7 +95,8 @@ class KMeansSuite extends FunSuite with LocalSparkContext { model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = K_MEANS_PARALLEL) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, + initializationMode = K_MEANS_PARALLEL) assert(model.clusterCenters.head === center) } @@ -139,7 +140,8 @@ class KMeansSuite extends FunSuite with LocalSparkContext { model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = K_MEANS_PARALLEL) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, + initializationMode = K_MEANS_PARALLEL) assert(model.clusterCenters.head === center) data.unpersist() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala index bfa42959c8ea..c7572ded3378 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala @@ -20,7 +20,10 @@ package org.apache.spark.mllib.regression import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, + LocalSparkContext} + +import scala.util.Random class LassoSuite extends FunSuite with LocalSparkContext { @@ -113,3 +116,19 @@ class LassoSuite extends FunSuite with LocalSparkContext { validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } } + +class LassoClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = LassoWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala index 7aaad7d7a3e3..064a890e7daa 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala @@ -20,7 +20,10 @@ package org.apache.spark.mllib.regression import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, + LocalSparkContext} + +import scala.util.Random class LinearRegressionSuite extends FunSuite with LocalSparkContext { @@ -122,3 +125,19 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext { sparseValidationData.map(row => model.predict(row.features)), sparseValidationData) } } + +class LinearRegressionClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = LinearRegressionWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala index 67768e17fbe6..727bbd051ff1 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala @@ -17,11 +17,14 @@ package org.apache.spark.mllib.regression -import org.scalatest.FunSuite +import scala.util.Random import org.jblas.DoubleMatrix +import org.scalatest.FunSuite -import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, + LocalSparkContext} class RidgeRegressionSuite extends FunSuite with LocalSparkContext { @@ -73,3 +76,19 @@ class RidgeRegressionSuite extends FunSuite with LocalSparkContext { "ridgeError (" + ridgeErr + ") was not less than linearError(" + linearErr + ")") } } + +class RidgeRegressionClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = RidgeRegressionWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + } +} From 9ebadcc60b49b8773e773bac3f578e2d15616b4a Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 15 Jul 2014 19:39:45 -0700 Subject: [PATCH 12/14] add task size test to RowMatrix --- .../linalg/distributed/RowMatrixSuite.scala | 28 ++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala index a961f89456a1..6717156d6b45 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -21,9 +21,11 @@ import org.scalatest.FunSuite import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, norm => brzNorm, svd => brzSvd} -import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} import org.apache.spark.mllib.linalg.{Matrices, Vectors, Vector} +import scala.util.Random + class RowMatrixSuite extends FunSuite with LocalSparkContext { val m = 4 @@ -193,3 +195,27 @@ class RowMatrixSuite extends FunSuite with LocalSparkContext { } } } + +class RowMatrixClusterSuite extends FunSuite with LocalClusterSparkContext { + + var mat: RowMatrix = _ + + override def beforeAll() { + super.beforeAll() + val m = 4 + val n = 200000 + val rows = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => Vectors.dense(Array.fill(n)(random.nextDouble()))) + } + mat = new RowMatrix(rows) + } + + test("task size should be small in svd") { + val svd = mat.computeSVD(1, computeU = true) + } + + test("task size should be small in summarize") { + val summary = mat.computeColumnSummaryStatistics() + } +} From b97c1848afd88ddf025b80903981aa4d8bbe3eca Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 15 Jul 2014 19:53:32 -0700 Subject: [PATCH 13/14] minimal change to LBFGS --- .../spark/mllib/optimization/LBFGS.scala | 20 +++++++++---------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index 26d98a6edf46..179cd4a3f162 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -199,17 +199,15 @@ object LBFGS extends Logging { val n = weights.length val bcWeights = data.context.broadcast(weights) - val (gradientSum, lossSum) = data.mapPartitions { iter => - val cumGrad = Vectors.dense(new Array[Double](n)) - val thisWeights = Vectors.fromBreeze(bcWeights.value) - var loss = 0.0 - iter.foreach { case (label, features) => - loss += localGradient.compute(features, label, thisWeights, cumGrad) - } - Iterator((cumGrad.toBreeze.asInstanceOf[BDV[Double]], loss)) - }.reduce { case ((grad1, loss1), (grad2, loss2)) => - (grad1 += grad2, loss1 + loss2) - } + val (gradientSum, lossSum) = data.aggregate((BDV.zeros[Double](n), 0.0))( + seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) => + val l = localGradient.compute( + features, label, Vectors.fromBreeze(bcWeights.value), Vectors.fromBreeze(grad)) + (grad, loss + l) + }, + combOp = (c1, c2) => (c1, c2) match { case ((grad1, loss1), (grad2, loss2)) => + (grad1 += grad2, loss1 + loss2) + }) /** * regVal is sum of weight squares if it's L2 updater; From b9a1228a360f2a9a45136f8da4b51990406441ff Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 15 Jul 2014 19:58:28 -0700 Subject: [PATCH 14/14] style update --- .../org/apache/spark/mllib/clustering/KMeansSuite.scala | 2 +- .../spark/mllib/linalg/distributed/RowMatrixSuite.scala | 7 +++---- .../org/apache/spark/mllib/optimization/LBFGSSuite.scala | 2 +- .../org/apache/spark/mllib/regression/LassoSuite.scala | 4 ++-- .../spark/mllib/regression/LinearRegressionSuite.scala | 4 ++-- 5 files changed, 9 insertions(+), 10 deletions(-) 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 80e6235657f7..82b2a656c44f 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,7 +17,7 @@ package org.apache.spark.mllib.clustering -import java.util.Random +import scala.util.Random import org.scalatest.FunSuite diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala index 6717156d6b45..325b817980f6 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -17,14 +17,13 @@ package org.apache.spark.mllib.linalg.distributed -import org.scalatest.FunSuite +import scala.util.Random import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, norm => brzNorm, svd => brzSvd} +import org.scalatest.FunSuite -import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} import org.apache.spark.mllib.linalg.{Matrices, Vectors, Vector} - -import scala.util.Random +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} class RowMatrixSuite extends FunSuite with LocalSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala index da263b569db6..ff414742e839 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.optimization -import java.util.Random +import scala.util.Random import org.scalatest.{FunSuite, Matchers} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala index c7572ded3378..7aa96421aed8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala @@ -17,14 +17,14 @@ package org.apache.spark.mllib.regression +import scala.util.Random + import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, LocalSparkContext} -import scala.util.Random - class LassoSuite extends FunSuite with LocalSparkContext { def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala index 064a890e7daa..4f89112b650c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala @@ -17,14 +17,14 @@ package org.apache.spark.mllib.regression +import scala.util.Random + import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, LocalSparkContext} -import scala.util.Random - class LinearRegressionSuite extends FunSuite with LocalSparkContext { def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) {