From 3f346baec3424fd5ec58716dacbb144dd85d2429 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 25 Mar 2014 22:47:23 -0700 Subject: [PATCH 01/26] update some ml algorithms to use Vector --- .../spark/mllib/optimization/Gradient.scala | 52 ++++++++-------- .../mllib/optimization/GradientDescent.scala | 13 ++-- .../spark/mllib/optimization/Optimizer.scala | 5 +- .../spark/mllib/optimization/Updater.scala | 58 ++++++++++++------ .../GeneralizedLinearAlgorithm.scala | 61 ++++++++++--------- .../spark/mllib/regression/LabeledPoint.scala | 6 +- .../mllib/regression/RidgeRegression.scala | 5 +- 7 files changed, 108 insertions(+), 92 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 82124703da6cd..3239af9d5df47 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.optimization -import org.jblas.DoubleMatrix +import org.apache.spark.mllib.linalg.{Vectors, Vector} /** * Class used to compute the gradient for a loss function, given a single data point. @@ -26,17 +26,13 @@ abstract class Gradient extends Serializable { /** * Compute the gradient and loss given the features of a single data point. * - * @param data - Feature values for one data point. Column matrix of size dx1 - * where d is the number of features. - * @param label - Label for this data item. - * @param weights - Column matrix containing weights for every feature. - * - * @return A tuple of 2 elements. The first element is a column matrix containing the computed - * gradient and the second element is the loss computed at this data point. + * @param data features for one data point + * @param label label for this data point + * @param weights weights/coefficients corresponding to features * + * @return (gradient: Vector, loss: Double) */ - def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): - (DoubleMatrix, Double) + def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) } /** @@ -44,12 +40,12 @@ abstract class Gradient extends Serializable { * See also the documentation for the precise formulation. */ class LogisticGradient extends Gradient { - override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): - (DoubleMatrix, Double) = { - val margin: Double = -1.0 * data.dot(weights) + override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { + val brzData = data.toBreeze + val brzWeights = data.toBreeze + val margin: Double = -1.0 * brzWeights.dot(brzData) val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label - - val gradient = data.mul(gradientMultiplier) + val gradient = brzData * gradientMultiplier val loss = if (label > 0) { math.log(1 + math.exp(margin)) @@ -57,7 +53,7 @@ class LogisticGradient extends Gradient { math.log(1 + math.exp(margin)) - margin } - (gradient, loss) + (Vectors.fromBreeze(gradient), loss) } } @@ -68,14 +64,14 @@ class LogisticGradient extends Gradient { * See also the documentation for the precise formulation. */ class LeastSquaresGradient extends Gradient { - override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): - (DoubleMatrix, Double) = { - val diff: Double = data.dot(weights) - label - + override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { + val brzData = data.toBreeze + val brzWeights = weights.toBreeze + val diff: Double = brzWeights.dot(brzData) - label val loss = diff * diff - val gradient = data.mul(2.0 * diff) + val gradient = brzData * (2.0 * diff) - (gradient, loss) + (Vectors.fromBreeze(gradient), loss) } } @@ -85,19 +81,19 @@ class LeastSquaresGradient extends Gradient { * NOTE: This assumes that the labels are {0,1} */ class HingeGradient extends Gradient { - override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): - (DoubleMatrix, Double) = { - - val dotProduct = data.dot(weights) + override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { + val brzData = data.toBreeze + val brzWeights = weights.toBreeze + val dotProduct = brzWeights.dot(brzData) // Our loss function with {0, 1} labels is max(0, 1 - (2y – 1) (f_w(x))) // Therefore the gradient is -(2y - 1)*x val labelScaled = 2 * label - 1.0 if (1.0 > labelScaled * dotProduct) { - (data.mul(-labelScaled), 1.0 - labelScaled * dotProduct) + (Vectors.fromBreeze(brzData * (-labelScaled)), 1.0 - labelScaled * dotProduct) } else { - (DoubleMatrix.zeros(1, weights.length), 0.0) + (Vectors.dense(new Array[Double](weights.size)), 0.0) } } } 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 b967b22e818d3..e5555cc7f73e3 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 @@ -20,10 +20,10 @@ package org.apache.spark.mllib.optimization import org.apache.spark.Logging import org.apache.spark.rdd.RDD -import org.jblas.DoubleMatrix - import scala.collection.mutable.ArrayBuffer +import org.apache.spark.mllib.linalg.Vector + /** * Class used to solve an optimization problem using Gradient Descent. * @param gradient Gradient function to be used. @@ -91,8 +91,7 @@ class GradientDescent(var gradient: Gradient, var updater: Updater) this } - def optimize(data: RDD[(Double, Array[Double])], initialWeights: Array[Double]) - : Array[Double] = { + def optimize(data: RDD[(Double, Vector)], initialWeights: Vector): Vector = { val (weights, stochasticLossHistory) = GradientDescent.runMiniBatchSGD( data, @@ -133,14 +132,14 @@ object GradientDescent extends Logging { * stochastic loss computed for every iteration. */ def runMiniBatchSGD( - data: RDD[(Double, Array[Double])], + data: RDD[(Double, Vector)], gradient: Gradient, updater: Updater, stepSize: Double, numIterations: Int, regParam: Double, miniBatchFraction: Double, - initialWeights: Array[Double]) : (Array[Double], Array[Double]) = { + initialWeights: Vector): (Vector, Vector) = { val stochasticLossHistory = new ArrayBuffer[Double](numIterations) @@ -148,7 +147,7 @@ object GradientDescent extends Logging { val miniBatchSize = nexamples * miniBatchFraction // Initialize weights as a column vector - var weights = new DoubleMatrix(initialWeights.length, 1, initialWeights:_*) + var weights = initialWeights.toBreeze.toDenseVector /** * For the first iteration, the regVal will be initialized as sum of sqrt of diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala index 94d30b56f212b..a62aecae5dd0d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala @@ -19,11 +19,12 @@ package org.apache.spark.mllib.optimization import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.Vector + trait Optimizer { /** * Solve the provided convex optimization problem. */ - def optimize(data: RDD[(Double, Array[Double])], initialWeights: Array[Double]): Array[Double] - + def optimize(data: RDD[(Double, Vector)], initialWeights: Vector): Vector } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala index bf8f731459e99..6070071c5c18b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala @@ -20,6 +20,10 @@ package org.apache.spark.mllib.optimization import scala.math._ import org.jblas.DoubleMatrix +import breeze.linalg.{norm => brzNorm} + +import org.apache.spark.mllib.linalg.{Vectors, Vector} + /** * Class used to perform steps (weight update) using Gradient Descent methods. * @@ -47,8 +51,12 @@ abstract class Updater extends Serializable { * @return A tuple of 2 elements. The first element is a column matrix containing updated weights, * and the second element is the regularization value computed using updated weights. */ - def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, - regParam: Double): (DoubleMatrix, Double) + def compute( + weightsOld: Vector, + gradient: Vector, + stepSize: Double, + iter: Int, + regParam: Double): (Vector, Double) } /** @@ -56,11 +64,11 @@ abstract class Updater extends Serializable { * Uses a step-size decreasing with the square root of the number of iterations. */ class SimpleUpdater extends Updater { - override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, - stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { + override def compute(weightsOld: Vector, gradient: Vector, + stepSize: Double, iter: Int, regParam: Double): (Vector, Double) = { val thisIterStepSize = stepSize / math.sqrt(iter) - val step = gradient.mul(thisIterStepSize) - (weightsOld.sub(step), 0) + val brzWeights = weightsOld.toBreeze - gradient.toBreeze * thisIterStepSize + (Vectors.fromBreeze(brzWeights), 0) } } @@ -83,19 +91,23 @@ class SimpleUpdater extends Updater { * Equivalently, set weight component to signum(w) * max(0.0, abs(w) - shrinkageVal) */ class L1Updater extends Updater { - override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, - stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { + override def compute( + weightsOld: Vector, + gradient: Vector, + stepSize: Double, + iter: Int, + regParam: Double): (Vector, Double) = { val thisIterStepSize = stepSize / math.sqrt(iter) - val step = gradient.mul(thisIterStepSize) // Take gradient step - val newWeights = weightsOld.sub(step) + val brzWeights = weightsOld.toBreeze - gradient.toBreeze * thisIterStepSize // Apply proximal operator (soft thresholding) val shrinkageVal = regParam * thisIterStepSize - (0 until newWeights.length).foreach { i => - val wi = newWeights.get(i) - newWeights.put(i, signum(wi) * max(0.0, abs(wi) - shrinkageVal)) + (0 until brzWeights.length).foreach { i => + val wi = brzWeights(i) + brzWeights(i) = signum(wi) * max(0.0, abs(wi) - shrinkageVal) } - (newWeights, newWeights.norm1 * regParam) + + (Vectors.fromBreeze(brzWeights), brzNorm(brzWeights, 1.0) * regParam) } } @@ -105,16 +117,22 @@ class L1Updater extends Updater { * Uses a step-size decreasing with the square root of the number of iterations. */ class SquaredL2Updater extends Updater { - override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, - stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { - val thisIterStepSize = stepSize / math.sqrt(iter) - val step = gradient.mul(thisIterStepSize) + override def compute( + weightsOld: Vector, + gradient: Vector, + stepSize: Double, + iter: Int, + regParam: Double): (DoubleMatrix, Double) = { // add up both updates from the gradient of the loss (= step) as well as // the gradient of the regularizer (= regParam * weightsOld) // w' = w - thisIterStepSize * (gradient + regParam * w) // w' = (1 - thisIterStepSize * regParam) * w - thisIterStepSize * gradient - val newWeights = weightsOld.mul(1.0 - thisIterStepSize * regParam).sub(step) - (newWeights, 0.5 * pow(newWeights.norm2, 2.0) * regParam) + val thisIterStepSize = stepSize / math.sqrt(iter) + val brzWeights = weightsOld.toBreeze * (1.0 - thisIterStepSize * regParam) - + (gradient.toBreeze * thisIterStepSize) + val norm = brzNorm(brzWeights, 2.0) + + (Vectors.fromBreeze(newWeights), 0.5 * regParam * norm * norm) } } 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 b9621530efa22..e4e710a726308 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 @@ -17,11 +17,12 @@ package org.apache.spark.mllib.regression +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} + import org.apache.spark.{Logging, SparkException} import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ - -import org.jblas.DoubleMatrix +import org.apache.spark.mllib.linalg.{Vectors, Vector} /** * GeneralizedLinearModel (GLM) represents a model trained using @@ -31,12 +32,9 @@ import org.jblas.DoubleMatrix * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -abstract class GeneralizedLinearModel(val weights: Array[Double], val intercept: Double) +abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double) extends Serializable { - // Create a column vector that can be used for predictions - private val weightsMatrix = new DoubleMatrix(weights.length, 1, weights:_*) - /** * Predict the result given a data point and the weights learned. * @@ -44,8 +42,7 @@ abstract class GeneralizedLinearModel(val weights: Array[Double], val intercept: * @param weightMatrix Column vector containing the weights of the model * @param intercept Intercept of the model. */ - def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, - intercept: Double): Double + def predictPoint(dataMatrix: Vector, weightMatrix: Vector, intercept: Double): Double /** * Predict values for the given data set using the model trained. @@ -53,16 +50,13 @@ abstract class GeneralizedLinearModel(val weights: Array[Double], val intercept: * @param testData RDD representing data points to be predicted * @return RDD[Double] where each entry contains the corresponding prediction */ - def predict(testData: RDD[Array[Double]]): RDD[Double] = { + def predict(testData: RDD[Vector]): RDD[Double] = { // A small optimization to avoid serializing the entire model. Only the weightsMatrix // and intercept is needed. - val localWeights = weightsMatrix + val localWeights = weights val localIntercept = intercept - testData.map { x => - val dataMatrix = new DoubleMatrix(1, x.length, x:_*) - predictPoint(dataMatrix, localWeights, localIntercept) - } + testData.map(v => predictPoint(v, localWeights, localIntercept)) } /** @@ -71,9 +65,8 @@ abstract class GeneralizedLinearModel(val weights: Array[Double], val intercept: * @param testData array representing a single data point * @return Double prediction from the trained model */ - def predict(testData: Array[Double]): Double = { - val dataMat = new DoubleMatrix(1, testData.length, testData:_*) - predictPoint(dataMat, weightsMatrix, intercept) + def predict(testData: Vector): Double = { + predictPoint(testData, weights, intercept) } } @@ -95,7 +88,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] /** * Create a model given the weights and intercept */ - protected def createModel(weights: Array[Double], intercept: Double): M + protected def createModel(weights: Vector, intercept: Double): M /** * Set if the algorithm should add an intercept. Default true. @@ -117,17 +110,26 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] * Run the algorithm with the configured parameters on an input * RDD of LabeledPoint entries. */ - def run(input: RDD[LabeledPoint]) : M = { - val nfeatures: Int = input.first().features.length - val initialWeights = new Array[Double](nfeatures) + def run(input: RDD[LabeledPoint]): M = { + val numFeatures: Int = input.first().features.size + val initialWeights = Vectors.dense(new Array[Double](numFeatures)) run(input, initialWeights) } + private def prependOne(vector: Vector): Vector = { + val vectorWithIntercept = vector match { + case dv: BDV[Double] => BDV.vertcat(BDV.ones(1), dv) + case sv: BSV[Double] => BSV.vertcat(new BSV[Double](Array(0), Array(1.0), 1), sv) + case v: Any => throw new IllegalArgumentException("Do not support vector type " + v.getClass) + } + Vectors.fromBreeze(vectorWithIntercept) + } + /** * Run the algorithm with the configured parameters on an input RDD * of LabeledPoint entries starting from the initial weights provided. */ - def run(input: RDD[LabeledPoint], initialWeights: Array[Double]) : M = { + def run(input: RDD[LabeledPoint], initialWeights: Vector): M = { // Check the data properties before running the optimizer if (validateData && !validators.forall(func => func(input))) { @@ -136,25 +138,24 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] // Prepend an extra variable consisting of all 1.0's for the intercept. val data = if (addIntercept) { - input.map(labeledPoint => (labeledPoint.label, labeledPoint.features.+:(1.0))) + input.map(labeledPoint => (labeledPoint.label, prependOne(labeledPoint.features))) } else { input.map(labeledPoint => (labeledPoint.label, labeledPoint.features)) } val initialWeightsWithIntercept = if (addIntercept) { - initialWeights.+:(1.0) + prependOne(initialWeights) } else { initialWeights } - val weights = optimizer.optimize(data, initialWeightsWithIntercept) - val intercept = weights(0) - val weightsScaled = weights.tail + val weightsWithIntercept = optimizer.optimize(data, initialWeightsWithIntercept) + val brzWeightsWithIntercept = weightsWithIntercept.toBreeze + val intercept = if (addIntercept) brzWeightsWithIntercept(0) else 0.0 + val brzWeights = if (addIntercept) brzWeightsWithIntercept(1 to -1) else brzWeightsWithIntercept - val model = createModel(weightsScaled, intercept) + val model = createModel(Vectors.fromBreeze(brzWeights), intercept) - logInfo("Final model weights " + model.weights.mkString(",")) - logInfo("Final model intercept " + model.intercept) model } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala index 1a18292fe3f3b..3deab1ab785b9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala @@ -17,14 +17,16 @@ package org.apache.spark.mllib.regression +import org.apache.spark.mllib.linalg.Vector + /** * Class that represents the features and labels of a data point. * * @param label Label for this data point. * @param features List of features for this data point. */ -case class LabeledPoint(label: Double, features: Array[Double]) { +case class LabeledPoint(label: Double, features: Vector) { override def toString: String = { - "LabeledPoint(%s, %s)".format(label, features.mkString("[", ", ", "]")) + "LabeledPoint(%s, %s)".format(label, features) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index c504d3d40c773..b8ce4602b53ef 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -21,8 +21,7 @@ import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.util.MLUtils - -import org.jblas.DoubleMatrix +import org.apache.spark.mllib.linalg.Vector /** * Regression model trained using RidgeRegression. @@ -31,7 +30,7 @@ import org.jblas.DoubleMatrix * @param intercept Intercept computed for this model. */ class RidgeRegressionModel( - override val weights: Array[Double], + override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { From d7f629f902aab81cf3637f07f9eb9f7119d9230c Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 25 Mar 2014 23:05:35 -0700 Subject: [PATCH 02/26] fix a bug in GLM when intercept is not used --- .../GeneralizedLinearAlgorithm.scala | 21 ++++++++------- .../regression/LinearRegressionSuite.scala | 26 ++++++++++++++++++- 2 files changed, 37 insertions(+), 10 deletions(-) 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 b9621530efa22..3e1ed91bf6729 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 @@ -136,25 +136,28 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] // Prepend an extra variable consisting of all 1.0's for the intercept. val data = if (addIntercept) { - input.map(labeledPoint => (labeledPoint.label, labeledPoint.features.+:(1.0))) + input.map(labeledPoint => (labeledPoint.label, 1.0 +: labeledPoint.features)) } else { input.map(labeledPoint => (labeledPoint.label, labeledPoint.features)) } val initialWeightsWithIntercept = if (addIntercept) { - initialWeights.+:(1.0) + 0.0 +: initialWeights } else { initialWeights } - val weights = optimizer.optimize(data, initialWeightsWithIntercept) - val intercept = weights(0) - val weightsScaled = weights.tail + val weightsWithIntercept = optimizer.optimize(data, initialWeightsWithIntercept) - val model = createModel(weightsScaled, intercept) + val (intercept, weights) = if (addIntercept) { + (weightsWithIntercept(0), weightsWithIntercept.tail) + } else { + (0.0, weightsWithIntercept) + } + + logInfo("Final weights " + weights.mkString(",")) + logInfo("Final intercept " + intercept) - logInfo("Final model weights " + model.weights.mkString(",")) - logInfo("Final model intercept " + model.intercept) - model + createModel(weights, intercept) } } 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 281f9df36ddb3..5d251bcbf35db 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,7 +17,6 @@ package org.apache.spark.mllib.regression -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} @@ -57,4 +56,29 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext { // Test prediction on Array. validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } + + // Test if we can correctly learn Y = 10*X1 + 10*X2 + test("linear regression without intercept") { + val testRDD = sc.parallelize(LinearDataGenerator.generateLinearInput( + 0.0, Array(10.0, 10.0), 100, 42), 2).cache() + val linReg = new LinearRegressionWithSGD().setIntercept(false) + linReg.optimizer.setNumIterations(1000).setStepSize(1.0) + + val model = linReg.run(testRDD) + + assert(model.intercept === 0.0) + assert(model.weights.length === 2) + assert(model.weights(0) >= 9.0 && model.weights(0) <= 11.0) + assert(model.weights(1) >= 9.0 && model.weights(1) <= 11.0) + + val validationData = LinearDataGenerator.generateLinearInput( + 0.0, Array(10.0, 10.0), 100, 17) + val validationRDD = sc.parallelize(validationData, 2).cache() + + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_.features)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row.features)), validationData) + } } From 0e57aa43f61a62a70faf27aed58dea201b494809 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 26 Mar 2014 11:44:48 -0700 Subject: [PATCH 03/26] update Lasso and RidgeRegression to parse the weights correctly from GLM mark createModel protected mark predictPoint protected --- .../GeneralizedLinearAlgorithm.scala | 2 +- .../apache/spark/mllib/regression/Lasso.scala | 20 +++++++++++++------ .../mllib/regression/LinearRegression.scala | 20 +++++++++---------- .../mllib/regression/RidgeRegression.scala | 18 ++++++++++++----- 4 files changed, 38 insertions(+), 22 deletions(-) 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 3e1ed91bf6729..2166c6bb6b443 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 @@ -44,7 +44,7 @@ abstract class GeneralizedLinearModel(val weights: Array[Double], val intercept: * @param weightMatrix Column vector containing the weights of the model * @param intercept Intercept of the model. */ - def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, + protected def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, intercept: Double): Double /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index fb2bc9b92a51c..e397a573079e5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -36,8 +36,10 @@ class LassoModel( extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { - override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, - intercept: Double) = { + override protected def predictPoint( + dataMatrix: DoubleMatrix, + weightMatrix: DoubleMatrix, + intercept: Double): Double = { dataMatrix.dot(weightMatrix) + intercept } } @@ -66,7 +68,7 @@ class LassoWithSGD private ( .setMiniBatchFraction(miniBatchFraction) // We don't want to penalize the intercept, so set this to false. - setIntercept(false) + super.setIntercept(false) var yMean = 0.0 var xColMean: DoubleMatrix = _ @@ -77,10 +79,16 @@ class LassoWithSGD private ( */ def this() = this(1.0, 100, 1.0, 1.0) - def createModel(weights: Array[Double], intercept: Double) = { - val weightsMat = new DoubleMatrix(weights.length + 1, 1, (Array(intercept) ++ weights):_*) + override def setIntercept(addIntercept: Boolean): this.type = { + // TODO: Support adding intercept. + if (addIntercept) throw new UnsupportedOperationException("Adding intercept is not supported.") + this + } + + override protected def createModel(weights: Array[Double], intercept: Double) = { + val weightsMat = new DoubleMatrix(weights.length, 1, weights: _*) val weightsScaled = weightsMat.div(xColSd) - val interceptScaled = yMean - (weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0)) + val interceptScaled = yMean - weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0) new LassoModel(weightsScaled.data, interceptScaled) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 8ee40addb25d9..b4aafbe8bcaff 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -31,13 +31,14 @@ import org.jblas.DoubleMatrix * @param intercept Intercept computed for this model. */ class LinearRegressionModel( - override val weights: Array[Double], - override val intercept: Double) - extends GeneralizedLinearModel(weights, intercept) - with RegressionModel with Serializable { - - override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, - intercept: Double) = { + override val weights: Array[Double], + override val intercept: Double) + extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { + + override protected def predictPoint( + dataMatrix: DoubleMatrix, + weightMatrix: DoubleMatrix, + intercept: Double): Double = { dataMatrix.dot(weightMatrix) + intercept } } @@ -55,8 +56,7 @@ class LinearRegressionWithSGD private ( var stepSize: Double, var numIterations: Int, var miniBatchFraction: Double) - extends GeneralizedLinearAlgorithm[LinearRegressionModel] - with Serializable { + extends GeneralizedLinearAlgorithm[LinearRegressionModel] with Serializable { val gradient = new LeastSquaresGradient() val updater = new SimpleUpdater() @@ -69,7 +69,7 @@ class LinearRegressionWithSGD private ( */ def this() = this(1.0, 100, 1.0) - def createModel(weights: Array[Double], intercept: Double) = { + override protected def createModel(weights: Array[Double], intercept: Double) = { new LinearRegressionModel(weights, intercept) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index c504d3d40c773..325e78c8f2233 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -36,8 +36,10 @@ class RidgeRegressionModel( extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { - override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, - intercept: Double) = { + override protected def predictPoint( + dataMatrix: DoubleMatrix, + weightMatrix: DoubleMatrix, + intercept: Double): Double = { dataMatrix.dot(weightMatrix) + intercept } } @@ -67,7 +69,7 @@ class RidgeRegressionWithSGD private ( .setMiniBatchFraction(miniBatchFraction) // We don't want to penalize the intercept in RidgeRegression, so set this to false. - setIntercept(false) + super.setIntercept(false) var yMean = 0.0 var xColMean: DoubleMatrix = _ @@ -78,8 +80,14 @@ class RidgeRegressionWithSGD private ( */ def this() = this(1.0, 100, 1.0, 1.0) - def createModel(weights: Array[Double], intercept: Double) = { - val weightsMat = new DoubleMatrix(weights.length + 1, 1, (Array(intercept) ++ weights):_*) + override def setIntercept(addIntercept: Boolean): this.type = { + // TODO: Support adding intercept. + if (addIntercept) throw new UnsupportedOperationException("Adding intercept is not supported.") + this + } + + override protected def createModel(weights: Array[Double], intercept: Double) = { + val weightsMat = new DoubleMatrix(weights.length, 1, weights: _*) val weightsScaled = weightsMat.div(xColSd) val interceptScaled = yMean - weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0) From 834ada23f66e871576ab8e3f38a4929f0c913a12 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 26 Mar 2014 13:49:49 -0700 Subject: [PATCH 04/26] optimized MLUtils.computeStats update some ml algorithms to use Vector (cont.) --- .../mllib/optimization/GradientDescent.scala | 19 +++-- .../spark/mllib/optimization/Updater.scala | 5 +- .../GeneralizedLinearAlgorithm.scala | 4 +- .../mllib/regression/LinearRegression.scala | 17 ++--- .../mllib/regression/RidgeRegression.scala | 43 ++++++----- .../org/apache/spark/mllib/util/MLUtils.scala | 73 ++++++++++--------- .../regression/RidgeRegressionSuite.scala | 4 +- 7 files changed, 84 insertions(+), 81 deletions(-) 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 e5555cc7f73e3..ad8868093435e 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 @@ -22,7 +22,7 @@ import org.apache.spark.rdd.RDD import scala.collection.mutable.ArrayBuffer -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.{Vectors, Vector} /** * Class used to solve an optimization problem using Gradient Descent. @@ -139,7 +139,7 @@ object GradientDescent extends Logging { numIterations: Int, regParam: Double, miniBatchFraction: Double, - initialWeights: Vector): (Vector, Vector) = { + initialWeights: Vector): (Vector, Array[Double]) = { val stochasticLossHistory = new ArrayBuffer[Double](numIterations) @@ -147,24 +147,23 @@ object GradientDescent extends Logging { val miniBatchSize = nexamples * miniBatchFraction // Initialize weights as a column vector - var weights = initialWeights.toBreeze.toDenseVector + var weights = Vectors.dense(initialWeights.toArray) /** * For the first iteration, the regVal will be initialized as sum of sqrt of * weights if it's L2 update; for L1 update; the same logic is followed. */ var regVal = updater.compute( - weights, new DoubleMatrix(initialWeights.length, 1), 0, 1, regParam)._2 + weights, Vectors.dense(new Array[Double](weights.size)), 0, 1, regParam)._2 for (i <- 1 to numIterations) { // 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).map { case (y, features) => - val featuresCol = new DoubleMatrix(features.length, 1, features:_*) - val (grad, loss) = gradient.compute(featuresCol, y, weights) - (grad, loss) - }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) + val (grad, loss) = gradient.compute(features, y, weights) + (grad.toBreeze, loss) + }.reduce((a, b) => (a._1 += b._1, a._2 + b._2)) /** * NOTE(Xinghao): lossSum is computed using the weights from the previous iteration @@ -172,7 +171,7 @@ object GradientDescent extends Logging { */ stochasticLossHistory.append(lossSum / miniBatchSize + regVal) val update = updater.compute( - weights, gradientSum.div(miniBatchSize), stepSize, i, regParam) + weights, Vectors.fromBreeze(gradientSum / miniBatchSize), stepSize, i, regParam) weights = update._1 regVal = update._2 } @@ -180,6 +179,6 @@ object GradientDescent extends Logging { logInfo("GradientDescent.runMiniBatchSGD finished. Last 10 stochastic losses %s".format( stochasticLossHistory.takeRight(10).mkString(", "))) - (weights.toArray, stochasticLossHistory.toArray) + (weights, stochasticLossHistory.toArray) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala index 6070071c5c18b..e5d4c4f7d282d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala @@ -18,7 +18,6 @@ package org.apache.spark.mllib.optimization import scala.math._ -import org.jblas.DoubleMatrix import breeze.linalg.{norm => brzNorm} @@ -122,7 +121,7 @@ class SquaredL2Updater extends Updater { gradient: Vector, stepSize: Double, iter: Int, - regParam: Double): (DoubleMatrix, Double) = { + regParam: Double): (Vector, Double) = { // add up both updates from the gradient of the loss (= step) as well as // the gradient of the regularizer (= regParam * weightsOld) // w' = w - thisIterStepSize * (gradient + regParam * w) @@ -132,7 +131,7 @@ class SquaredL2Updater extends Updater { (gradient.toBreeze * thisIterStepSize) val norm = brzNorm(brzWeights, 2.0) - (Vectors.fromBreeze(newWeights), 0.5 * regParam * norm * norm) + (Vectors.fromBreeze(brzWeights), 0.5 * regParam * norm * norm) } } 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 643ce63f62eeb..e6d550fe3782b 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 @@ -118,8 +118,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] /** Prepends one to the input vector. */ private def prependOne(vector: Vector): Vector = { - val vectorWithIntercept = vector match { - case dv: BDV[Double] => BDV.vertcat(BDV.ones(1), dv) + val vectorWithIntercept = vector.toBreeze match { + case dv: BDV[Double] => BDV.vertcat(BDV.ones[Double](1), dv) case sv: BSV[Double] => BSV.vertcat(new BSV[Double](Array(0), Array(1.0), 1), sv) case v: Any => throw new IllegalArgumentException("Do not support vector type " + v.getClass) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index b4aafbe8bcaff..13c401aca31ef 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -21,8 +21,7 @@ import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.util.MLUtils - -import org.jblas.DoubleMatrix +import org.apache.spark.mllib.linalg.{Vector, Vectors} /** * Regression model trained using LinearRegression. @@ -31,15 +30,15 @@ import org.jblas.DoubleMatrix * @param intercept Intercept computed for this model. */ class LinearRegressionModel( - override val weights: Array[Double], + override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { override protected def predictPoint( - dataMatrix: DoubleMatrix, - weightMatrix: DoubleMatrix, + dataMatrix: Vector, + weightMatrix: Vector, intercept: Double): Double = { - dataMatrix.dot(weightMatrix) + intercept + weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept } } @@ -69,7 +68,7 @@ class LinearRegressionWithSGD private ( */ def this() = this(1.0, 100, 1.0) - override protected def createModel(weights: Array[Double], intercept: Double) = { + override protected def createModel(weights: Vector, intercept: Double) = { new LinearRegressionModel(weights, intercept) } } @@ -98,7 +97,7 @@ object LinearRegressionWithSGD { numIterations: Int, stepSize: Double, miniBatchFraction: Double, - initialWeights: Array[Double]) + initialWeights: Vector) : LinearRegressionModel = { new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction).run(input, @@ -172,7 +171,7 @@ object LinearRegressionWithSGD { val sc = new SparkContext(args(0), "LinearRegression") val data = MLUtils.loadLabeledData(sc, args(1)) val model = LinearRegressionWithSGD.train(data, args(3).toInt, args(2).toDouble) - println("Weights: " + model.weights.mkString("[", ", ", "]")) + println("Weights: " + model.weights) println("Intercept: " + model.intercept) sc.stop() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index d5371f5c33414..22a0e8b495957 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -21,7 +21,9 @@ import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.{Vectors, Vector} + +import breeze.linalg.{Vector => BV, DenseVector => BDV} /** * Regression model trained using RidgeRegression. @@ -36,10 +38,10 @@ class RidgeRegressionModel( with RegressionModel with Serializable { override protected def predictPoint( - dataMatrix: DoubleMatrix, - weightMatrix: DoubleMatrix, + dataMatrix: Vector, + weightMatrix: Vector, intercept: Double): Double = { - dataMatrix.dot(weightMatrix) + intercept + weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept } } @@ -71,8 +73,8 @@ class RidgeRegressionWithSGD private ( super.setIntercept(false) var yMean = 0.0 - var xColMean: DoubleMatrix = _ - var xColSd: DoubleMatrix = _ + var xColMean: BV[Double] = _ + var xColSd: BV[Double] = _ /** * Construct a RidgeRegression object with default parameters @@ -85,33 +87,33 @@ class RidgeRegressionWithSGD private ( this } - override protected def createModel(weights: Array[Double], intercept: Double) = { - val weightsMat = new DoubleMatrix(weights.length, 1, weights: _*) - val weightsScaled = weightsMat.div(xColSd) - val interceptScaled = yMean - weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0) + override protected def createModel(weights: Vector, intercept: Double) = { + val weightsMat = weights.toBreeze + val weightsScaled = weightsMat :/ xColSd + val interceptScaled = yMean - weightsMat.dot(xColMean :/ xColSd) - new RidgeRegressionModel(weightsScaled.data, interceptScaled) + new RidgeRegressionModel(Vectors.fromBreeze(weightsScaled), interceptScaled) } override def run( input: RDD[LabeledPoint], - initialWeights: Array[Double]) + initialWeights: Vector) : RidgeRegressionModel = { - val nfeatures: Int = input.first().features.length + val nfeatures: Int = input.first().features.size val nexamples: Long = input.count() // To avoid penalizing the intercept, we center and scale the data. val stats = MLUtils.computeStats(input, nfeatures, nexamples) yMean = stats._1 - xColMean = stats._2 - xColSd = stats._3 + xColMean = stats._2.toBreeze + xColSd = stats._3.toBreeze val normalizedData = input.map { point => val yNormalized = point.label - yMean - val featuresMat = new DoubleMatrix(nfeatures, 1, point.features:_*) - val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd) - LabeledPoint(yNormalized, featuresNormalized.toArray) + val featuresMat = point.features.toBreeze + val featuresNormalized = (featuresMat - xColMean) :/ xColSd + LabeledPoint(yNormalized, Vectors.fromBreeze(featuresNormalized)) } super.run(normalizedData, initialWeights) @@ -143,7 +145,7 @@ object RidgeRegressionWithSGD { stepSize: Double, regParam: Double, miniBatchFraction: Double, - initialWeights: Array[Double]) + initialWeights: Vector) : RidgeRegressionModel = { new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run( @@ -220,7 +222,8 @@ object RidgeRegressionWithSGD { val data = MLUtils.loadLabeledData(sc, args(1)) val model = RidgeRegressionWithSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) - println("Weights: " + model.weights.mkString("[", ", ", "]")) + + println("Weights: " + model.weights) println("Intercept: " + model.intercept) sc.stop() 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 08cd9ab05547b..d495d590fd4f6 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 @@ -17,15 +17,13 @@ package org.apache.spark.mllib.util +import breeze.linalg.{Vector => BV, DenseVector => BDV, SparseVector => BSV, + squaredDistance => breezeSquaredDistance} + import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext._ - -import org.jblas.DoubleMatrix - import org.apache.spark.mllib.regression.LabeledPoint - -import breeze.linalg.{Vector => BV, SparseVector => BSV, squaredDistance => breezeSquaredDistance} +import org.apache.spark.mllib.linalg.{Vector, Vectors} /** * Helper methods to load, save and pre-process data used in ML Lib. @@ -54,7 +52,7 @@ object MLUtils { sc.textFile(dir).map { line => val parts = line.split(',') val label = parts(0).toDouble - val features = parts(1).trim().split(' ').map(_.toDouble) + val features = Vectors.dense(parts(1).trim().split(' ').map(_.toDouble)) LabeledPoint(label, features) } } @@ -68,7 +66,7 @@ object MLUtils { * @param dir Directory to save the data. */ def saveLabeledData(data: RDD[LabeledPoint], dir: String) { - val dataStr = data.map(x => x.label + "," + x.features.mkString(" ")) + val dataStr = data.map(x => x.label + "," + x.features.toArray.mkString(" ")) dataStr.saveAsTextFile(dir) } @@ -76,44 +74,51 @@ object MLUtils { * Utility function to compute mean and standard deviation on a given dataset. * * @param data - input data set whose statistics are computed - * @param nfeatures - number of features - * @param nexamples - number of examples in input dataset + * @param numFeatures - number of features + * @param numExamples - number of examples in input dataset * * @return (yMean, xColMean, xColSd) - Tuple consisting of * yMean - mean of the labels * xColMean - Row vector with mean for every column (or feature) of the input data * xColSd - Row vector standard deviation for every column (or feature) of the input data. */ - def computeStats(data: RDD[LabeledPoint], nfeatures: Int, nexamples: Long): - (Double, DoubleMatrix, DoubleMatrix) = { - val yMean: Double = data.map { labeledPoint => labeledPoint.label }.reduce(_ + _) / nexamples + def computeStats(data: RDD[LabeledPoint], numFeatures: Int, numExamples: Long) + : (Double, Vector, Vector) = { - // NOTE: We shuffle X by column here to compute column sum and sum of squares. - val xColSumSq: RDD[(Int, (Double, Double))] = data.flatMap { labeledPoint => - val nCols = labeledPoint.features.length - // Traverse over every column and emit (col, value, value^2) - Iterator.tabulate(nCols) { i => - (i, (labeledPoint.features(i), labeledPoint.features(i)*labeledPoint.features(i))) - } - }.reduceByKey { case(x1, x2) => - (x1._1 + x2._1, x1._2 + x2._2) + val brzData = data.map { case LabeledPoint(label, features) => + (label, features.toBreeze) } - val xColSumsMap = xColSumSq.collectAsMap() - val xColMean = DoubleMatrix.zeros(nfeatures, 1) - val xColSd = DoubleMatrix.zeros(nfeatures, 1) + val aggStats = brzData.aggregate( + (0L, 0.0, BDV.zeros[Double](numFeatures), BDV.zeros[Double](numFeatures)) + )( + seqOp = (c , v) => (c, v) match { + case ((n, sumLabel, sum, sumSq), (label, features)) => + features.activeIterator.foreach { case (i, x) => + sumSq(i) += x * x + } + (n + 1L, sumLabel + label, sum += features, sumSq) + }, + combOp = (c1, c2) => (c1, c2) match { + case ((n1, sumLabel1, sum1, sumSq1), (n2, sumLabel2, sum2, sumSq2)) => + (n1 + n2, sumLabel1 + sumLabel2, sum1 += sum2, sumSq1 += sumSq2) + } + ) + + val (nl, sumLabel, sum, sumSq) = aggStats + require(nl > 0, "Input data is empty.") - // Compute mean and unbiased variance using column sums - var col = 0 - while (col < nfeatures) { - xColMean.put(col, xColSumsMap(col)._1 / nexamples) - val variance = - (xColSumsMap(col)._2 - (math.pow(xColSumsMap(col)._1, 2) / nexamples)) / nexamples - xColSd.put(col, math.sqrt(variance)) - col += 1 + val n = nl.toDouble + val yMean = sumLabel / n + val mean: BDV[Double] = sum / n + val std = new Array[Double](sum.length) + var i = 0 + while (i < numFeatures) { + std(i) = sumSq(i) / n - mean(i) * mean(i) + i += 1 } - (yMean, xColMean, xColSd) + (yMean, Vectors.fromBreeze(mean), Vectors.dense(std)) } /** 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 67dd06cc0f5eb..a1f72bc8536c0 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,14 +17,12 @@ package org.apache.spark.mllib.regression +import org.scalatest.FunSuite import org.jblas.DoubleMatrix -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} - class RidgeRegressionSuite extends FunSuite with LocalSparkContext { def predictionError(predictions: Seq[Double], input: Seq[LabeledPoint]) = { From 18597011768fa857747ab809302c6df351d24cb6 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 26 Mar 2014 15:10:33 -0700 Subject: [PATCH 05/26] passed compile --- .../mllib/api/python/PythonMLLibAPI.scala | 117 +++++++++++++----- .../classification/ClassificationModel.scala | 5 +- .../classification/LogisticRegression.scala | 37 +++--- .../mllib/classification/NaiveBayes.scala | 20 ++- .../spark/mllib/classification/SVM.scala | 36 ++---- .../apache/spark/mllib/regression/Lasso.scala | 50 ++++---- .../mllib/regression/RegressionModel.scala | 5 +- .../mllib/util/LinearDataGenerator.scala | 3 +- .../LogisticRegressionDataGenerator.scala | 3 +- .../spark/mllib/util/SVMDataGenerator.scala | 3 +- .../LogisticRegressionSuite.scala | 5 +- 11 files changed, 165 insertions(+), 119 deletions(-) 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 3449c698da60b..c5729ee17c375 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 @@ -114,12 +114,12 @@ class PythonMLLibAPI extends Serializable { java.util.LinkedList[java.lang.Object] = { val data = dataBytesJRDD.rdd.map(xBytes => { val x = deserializeDoubleVector(xBytes) - LabeledPoint(x(0), x.slice(1, x.length)) + LabeledPoint(x(0), Vectors.dense(x.slice(1, x.length))) }) val initialWeights = deserializeDoubleVector(initialWeightsBA) val model = trainFunc(data, initialWeights) val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(serializeDoubleVector(model.weights)) + ret.add(serializeDoubleVector(model.weights.toArray)) ret.add(model.intercept: java.lang.Double) ret } @@ -127,61 +127,112 @@ class PythonMLLibAPI extends Serializable { /** * Java stub for Python mllib LinearRegressionWithSGD.train() */ - def trainLinearRegressionModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]], - numIterations: Int, stepSize: Double, miniBatchFraction: Double, + def trainLinearRegressionModelWithSGD( + dataBytesJRDD: JavaRDD[Array[Byte]], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double, initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { - trainRegressionModel((data, initialWeights) => - LinearRegressionWithSGD.train(data, numIterations, stepSize, - miniBatchFraction, initialWeights), - dataBytesJRDD, initialWeightsBA) + trainRegressionModel( + (data, initialWeights) => + LinearRegressionWithSGD.train( + data, + numIterations, + stepSize, + miniBatchFraction, + Vectors.dense(initialWeights)), + dataBytesJRDD, + initialWeightsBA) } /** * Java stub for Python mllib LassoWithSGD.train() */ - def trainLassoModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]], numIterations: Int, - stepSize: Double, regParam: Double, miniBatchFraction: Double, + def trainLassoModelWithSGD( + dataBytesJRDD: JavaRDD[Array[Byte]], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { - trainRegressionModel((data, initialWeights) => - LassoWithSGD.train(data, numIterations, stepSize, regParam, - miniBatchFraction, initialWeights), - dataBytesJRDD, initialWeightsBA) + trainRegressionModel( + (data, initialWeights) => + LassoWithSGD.train( + data, + numIterations, + stepSize, + regParam, + miniBatchFraction, + Vectors.dense(initialWeights)), + dataBytesJRDD, + initialWeightsBA) } /** * Java stub for Python mllib RidgeRegressionWithSGD.train() */ - def trainRidgeModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]], numIterations: Int, - stepSize: Double, regParam: Double, miniBatchFraction: Double, + def trainRidgeModelWithSGD( + dataBytesJRDD: JavaRDD[Array[Byte]], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { - trainRegressionModel((data, initialWeights) => - RidgeRegressionWithSGD.train(data, numIterations, stepSize, regParam, - miniBatchFraction, initialWeights), - dataBytesJRDD, initialWeightsBA) + trainRegressionModel( + (data, initialWeights) => + RidgeRegressionWithSGD.train( + data, + numIterations, + stepSize, + regParam, + miniBatchFraction, + Vectors.dense(initialWeights)), + dataBytesJRDD, + initialWeightsBA) } /** * Java stub for Python mllib SVMWithSGD.train() */ - def trainSVMModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]], numIterations: Int, - stepSize: Double, regParam: Double, miniBatchFraction: Double, + def trainSVMModelWithSGD( + dataBytesJRDD: JavaRDD[Array[Byte]], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { - trainRegressionModel((data, initialWeights) => - SVMWithSGD.train(data, numIterations, stepSize, regParam, - miniBatchFraction, initialWeights), - dataBytesJRDD, initialWeightsBA) + trainRegressionModel( + (data, initialWeights) => + SVMWithSGD.train( + data, + numIterations, + stepSize, + regParam, + miniBatchFraction, + Vectors.dense(initialWeights)), + dataBytesJRDD, + initialWeightsBA) } /** * Java stub for Python mllib LogisticRegressionWithSGD.train() */ - def trainLogisticRegressionModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]], - numIterations: Int, stepSize: Double, miniBatchFraction: Double, + def trainLogisticRegressionModelWithSGD( + dataBytesJRDD: JavaRDD[Array[Byte]], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double, initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { - trainRegressionModel((data, initialWeights) => - LogisticRegressionWithSGD.train(data, numIterations, stepSize, - miniBatchFraction, initialWeights), - dataBytesJRDD, initialWeightsBA) + trainRegressionModel( + (data, initialWeights) => + LogisticRegressionWithSGD.train( + data, + numIterations, + stepSize, + miniBatchFraction, + Vectors.dense(initialWeights)), + dataBytesJRDD, + initialWeightsBA) } /** @@ -192,7 +243,7 @@ class PythonMLLibAPI extends Serializable { { val data = dataBytesJRDD.rdd.map(xBytes => { val x = deserializeDoubleVector(xBytes) - LabeledPoint(x(0), x.slice(1, x.length)) + LabeledPoint(x(0), Vectors.dense(x.slice(1, x.length))) }) val model = NaiveBayes.train(data, lambda) val ret = new java.util.LinkedList[java.lang.Object]() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala index 391f5b9b7a7de..2591d89b9e0dc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.classification import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.Vector trait ClassificationModel extends Serializable { /** @@ -26,7 +27,7 @@ trait ClassificationModel extends Serializable { * @param testData RDD representing data points to be predicted * @return RDD[Int] where each entry contains the corresponding prediction */ - def predict(testData: RDD[Array[Double]]): RDD[Double] + def predict(testData: RDD[Vector]): RDD[Double] /** * Predict values for a single data point using the model trained. @@ -34,5 +35,5 @@ trait ClassificationModel extends Serializable { * @param testData array representing a single data point * @return Int prediction from the trained model */ - def predict(testData: Array[Double]): Double + def predict(testData: Vector): Double } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index a481f522761e2..da9995f5879ad 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -25,8 +25,7 @@ import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.mllib.util.DataValidators - -import org.jblas.DoubleMatrix +import org.apache.spark.mllib.linalg.Vector /** * Classification model trained using Logistic Regression. @@ -35,14 +34,14 @@ import org.jblas.DoubleMatrix * @param intercept Intercept computed for this model. */ class LogisticRegressionModel( - override val weights: Array[Double], + override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable { - override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, + override def predictPoint(dataMatrix: Vector, weightMatrix: Vector, intercept: Double) = { - val margin = dataMatrix.mmul(weightMatrix).get(0) + intercept + val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept round(1.0/ (1.0 + math.exp(margin * -1))) } } @@ -73,7 +72,7 @@ class LogisticRegressionWithSGD private ( */ def this() = this(1.0, 100, 0.0, 1.0) - def createModel(weights: Array[Double], intercept: Double) = { + def createModel(weights: Vector, intercept: Double) = { new LogisticRegressionModel(weights, intercept) } } @@ -105,11 +104,9 @@ object LogisticRegressionWithSGD { numIterations: Int, stepSize: Double, miniBatchFraction: Double, - initialWeights: Array[Double]) - : LogisticRegressionModel = - { - new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction).run( - input, initialWeights) + initialWeights: Vector): LogisticRegressionModel = { + new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction) + .run(input, initialWeights) } /** @@ -128,11 +125,9 @@ object LogisticRegressionWithSGD { input: RDD[LabeledPoint], numIterations: Int, stepSize: Double, - miniBatchFraction: Double) - : LogisticRegressionModel = - { - new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction).run( - input) + miniBatchFraction: Double): LogisticRegressionModel = { + new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction) + .run(input) } /** @@ -150,9 +145,7 @@ object LogisticRegressionWithSGD { def train( input: RDD[LabeledPoint], numIterations: Int, - stepSize: Double) - : LogisticRegressionModel = - { + stepSize: Double): LogisticRegressionModel = { train(input, numIterations, stepSize, 1.0) } @@ -168,9 +161,7 @@ object LogisticRegressionWithSGD { */ def train( input: RDD[LabeledPoint], - numIterations: Int) - : LogisticRegressionModel = - { + numIterations: Int): LogisticRegressionModel = { train(input, numIterations, 1.0, 1.0) } @@ -183,7 +174,7 @@ object LogisticRegressionWithSGD { val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadLabeledData(sc, args(1)) val model = LogisticRegressionWithSGD.train(data, args(3).toInt, args(2).toDouble) - println("Weights: " + model.weights.mkString("[", ", ", "]")) + println("Weights: " + model.weights) println("Intercept: " + model.intercept) sc.stop() 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 6539b2f339465..f4228fe5e7522 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 @@ -25,6 +25,7 @@ import org.apache.spark.{SparkContext, Logging} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.mllib.linalg.Vector /** * Model for Naive Bayes Classifiers. @@ -39,9 +40,11 @@ class NaiveBayesModel(val pi: Array[Double], val theta: Array[Array[Double]]) private val _pi = new DoubleMatrix(pi.length, 1, pi: _*) private val _theta = new DoubleMatrix(theta) - def predict(testData: RDD[Array[Double]]): RDD[Double] = testData.map(predict) + override def predict(testData: RDD[Vector]): RDD[Double] = testData.map(predict) - def predict(testData: Array[Double]): Double = { + override def predict(testData: Vector): Double = predict(testData.toArray) + + private def predict(testData: Array[Double]): Double = { val dataMatrix = new DoubleMatrix(testData.length, 1, testData: _*) val result = _pi.add(_theta.mmul(dataMatrix)) result.argmax() @@ -70,9 +73,18 @@ class NaiveBayes private (var lambda: Double) /** * Run the algorithm with the configured parameters on an input RDD of LabeledPoint entries. * - * @param data RDD of (label, array of features) pairs. + * @param data RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. */ def run(data: RDD[LabeledPoint]) = { + runRaw(data.map(v => (v.label, v.features.toArray))) + } + + /** + * Run the algorithm with the configured parameters on an input RDD of LabeledPoint entries. + * + * @param data RDD of (label, array of features) pairs. + */ + private def runRaw(data: RDD[(Double, Array[Double])]) = { // Aggregates all sample points to driver side to get sample count and summed feature vector // for each label. The shape of `zeroCombiner` & `aggregated` is: // @@ -80,7 +92,7 @@ class NaiveBayes private (var lambda: Double) val zeroCombiner = mutable.Map.empty[Int, (Int, DoubleMatrix)] val aggregated = data.aggregate(zeroCombiner)({ (combiner, point) => point match { - case LabeledPoint(label, features) => + case (label, features) => val (count, featuresSum) = combiner.getOrElse(label.toInt, (0, DoubleMatrix.zeros(1))) val fs = new DoubleMatrix(features.length, 1, features: _*) combiner += label.toInt -> (count + 1, featuresSum.addi(fs)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index 6dff29dfb45cc..b854bcab815f0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -23,8 +23,7 @@ import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.mllib.util.DataValidators - -import org.jblas.DoubleMatrix +import org.apache.spark.mllib.linalg.{Vector, Vectors} /** * Model for Support Vector Machines (SVMs). @@ -33,14 +32,14 @@ import org.jblas.DoubleMatrix * @param intercept Intercept computed for this model. */ class SVMModel( - override val weights: Array[Double], + override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable { - override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, + override def predictPoint(dataMatrix: Vector, weightMatrix: Vector, intercept: Double) = { - val margin = dataMatrix.dot(weightMatrix) + intercept + val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept if (margin < 0) 0.0 else 1.0 } } @@ -71,7 +70,7 @@ class SVMWithSGD private ( */ def this() = this(1.0, 100, 1.0, 1.0) - def createModel(weights: Array[Double], intercept: Double) = { + def createModel(weights: Vector, intercept: Double) = { new SVMModel(weights, intercept) } } @@ -103,11 +102,9 @@ object SVMWithSGD { stepSize: Double, regParam: Double, miniBatchFraction: Double, - initialWeights: Array[Double]) - : SVMModel = - { - new SVMWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input, - initialWeights) + initialWeights: Vector): SVMModel = { + new SVMWithSGD(stepSize, numIterations, regParam, miniBatchFraction) + .run(input, initialWeights) } /** @@ -127,9 +124,7 @@ object SVMWithSGD { numIterations: Int, stepSize: Double, regParam: Double, - miniBatchFraction: Double) - : SVMModel = - { + miniBatchFraction: Double): SVMModel = { new SVMWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input) } @@ -149,9 +144,7 @@ object SVMWithSGD { input: RDD[LabeledPoint], numIterations: Int, stepSize: Double, - regParam: Double) - : SVMModel = - { + regParam: Double): SVMModel = { train(input, numIterations, stepSize, regParam, 1.0) } @@ -165,11 +158,7 @@ object SVMWithSGD { * @param numIterations Number of iterations of gradient descent to run. * @return a SVMModel which has the weights and offset from training. */ - def train( - input: RDD[LabeledPoint], - numIterations: Int) - : SVMModel = - { + def train(input: RDD[LabeledPoint], numIterations: Int): SVMModel = { train(input, numIterations, 1.0, 1.0, 1.0) } @@ -181,7 +170,8 @@ object SVMWithSGD { val sc = new SparkContext(args(0), "SVM") val data = MLUtils.loadLabeledData(sc, args(1)) val model = SVMWithSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) - println("Weights: " + model.weights.mkString("[", ", ", "]")) + + println("Weights: " + model.weights) println("Intercept: " + model.intercept) sc.stop() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index e397a573079e5..2b6dbacde22d6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -17,12 +17,13 @@ package org.apache.spark.mllib.regression -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.jblas.DoubleMatrix +import breeze.linalg.{Vector => BV, DenseVector => BDV} /** * Regression model trained using Lasso. @@ -31,16 +32,16 @@ import org.jblas.DoubleMatrix * @param intercept Intercept computed for this model. */ class LassoModel( - override val weights: Array[Double], + override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { override protected def predictPoint( - dataMatrix: DoubleMatrix, - weightMatrix: DoubleMatrix, + dataMatrix: Vector, + weightMatrix: Vector, intercept: Double): Double = { - dataMatrix.dot(weightMatrix) + intercept + weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept } } @@ -71,8 +72,8 @@ class LassoWithSGD private ( super.setIntercept(false) var yMean = 0.0 - var xColMean: DoubleMatrix = _ - var xColSd: DoubleMatrix = _ + var xColMean: BV[Double] = _ + var xColSd: BV[Double] = _ /** * Construct a Lasso object with default parameters @@ -85,33 +86,29 @@ class LassoWithSGD private ( this } - override protected def createModel(weights: Array[Double], intercept: Double) = { - val weightsMat = new DoubleMatrix(weights.length, 1, weights: _*) - val weightsScaled = weightsMat.div(xColSd) - val interceptScaled = yMean - weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0) + override protected def createModel(weights: Vector, intercept: Double) = { + val weightsMat = weights.toBreeze + val weightsScaled = weightsMat :/ xColSd + val interceptScaled = yMean - weightsMat.dot(xColMean :/ xColSd) - new LassoModel(weightsScaled.data, interceptScaled) + new LassoModel(Vectors.fromBreeze(weightsScaled), interceptScaled) } - override def run( - input: RDD[LabeledPoint], - initialWeights: Array[Double]) - : LassoModel = - { - val nfeatures: Int = input.first.features.length + override def run(input: RDD[LabeledPoint], initialWeights: Vector): LassoModel = { + val nfeatures: Int = input.first.features.size val nexamples: Long = input.count() // To avoid penalizing the intercept, we center and scale the data. val stats = MLUtils.computeStats(input, nfeatures, nexamples) yMean = stats._1 - xColMean = stats._2 - xColSd = stats._3 + xColMean = stats._2.toBreeze + xColSd = stats._3.toBreeze val normalizedData = input.map { point => val yNormalized = point.label - yMean - val featuresMat = new DoubleMatrix(nfeatures, 1, point.features:_*) - val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd) - LabeledPoint(yNormalized, featuresNormalized.toArray) + val featuresMat = point.features.toBreeze + val featuresNormalized = (featuresMat - xColMean) :/ xColSd + LabeledPoint(yNormalized, Vectors.fromBreeze(featuresNormalized)) } super.run(normalizedData, initialWeights) @@ -144,7 +141,7 @@ object LassoWithSGD { stepSize: Double, regParam: Double, miniBatchFraction: Double, - initialWeights: Array[Double]) + initialWeights: Vector) : LassoModel = { new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input, @@ -222,7 +219,8 @@ object LassoWithSGD { val sc = new SparkContext(args(0), "Lasso") val data = MLUtils.loadLabeledData(sc, args(1)) val model = LassoWithSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) - println("Weights: " + model.weights.mkString("[", ", ", "]")) + + println("Weights: " + model.weights) println("Intercept: " + model.intercept) sc.stop() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala index 423afc32d665c..5e4b8a345b1c5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.Vector trait RegressionModel extends Serializable { /** @@ -26,7 +27,7 @@ trait RegressionModel extends Serializable { * @param testData RDD representing data points to be predicted * @return RDD[Double] where each entry contains the corresponding prediction */ - def predict(testData: RDD[Array[Double]]): RDD[Double] + def predict(testData: RDD[Vector]): RDD[Double] /** * Predict values for a single data point using the model trained. @@ -34,5 +35,5 @@ trait RegressionModel extends Serializable { * @param testData array representing a single data point * @return Double prediction from the trained model */ - def predict(testData: Array[Double]): Double + def predict(testData: Vector): Double } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 2e03684e62861..81e4eda2a68c4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -24,6 +24,7 @@ import org.jblas.DoubleMatrix import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint /** @@ -74,7 +75,7 @@ object LinearDataGenerator { val y = x.map { xi => new DoubleMatrix(1, xi.length, xi: _*).dot(weightsMat) + intercept + eps * rnd.nextGaussian() } - y.zip(x).map(p => LabeledPoint(p._1, p._2)) + y.zip(x).map(p => LabeledPoint(p._1, Vectors.dense(p._2))) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala index 52c4a71d621a1..61498dcc2be00 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala @@ -22,6 +22,7 @@ import scala.util.Random import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.Vectors /** * Generate test data for LogisticRegression. This class chooses positive labels @@ -54,7 +55,7 @@ object LogisticRegressionDataGenerator { val x = Array.fill[Double](nfeatures) { rnd.nextGaussian() + (y * eps) } - LabeledPoint(y, x) + LabeledPoint(y, Vectors.dense(x)) } data } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index c96c94f70eef7..e300c3dbe1fe0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -23,6 +23,7 @@ import org.jblas.DoubleMatrix import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint /** @@ -58,7 +59,7 @@ object SVMDataGenerator { } val yD = new DoubleMatrix(1, x.length, x: _*).dot(trueWeights) + rnd.nextGaussian() * 0.1 val y = if (yD < 0) 0.0 else 1.0 - LabeledPoint(y, x) + LabeledPoint(y, Vectors.dense(x)) } MLUtils.saveLabeledData(data, outputPath) 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 05322b024d5f6..cce3aea0e2ad8 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 @@ -20,11 +20,10 @@ package org.apache.spark.mllib.classification import scala.util.Random import scala.collection.JavaConversions._ -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers -import org.apache.spark.SparkContext +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.LocalSparkContext @@ -61,7 +60,7 @@ object LogisticRegressionSuite { if (yVal > 0) 1 else 0 } - val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Array(x1(i)))) + val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Vectors.dense(Array(x1(i))))) testData } From 75c83a4697f17db00eb877b2d9fd741ec708ee23 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 26 Mar 2014 15:34:13 -0700 Subject: [PATCH 06/26] passed test compile --- .../org/apache/spark/mllib/linalg/Vectors.scala | 6 ++++++ .../classification/JavaNaiveBayesSuite.java | 13 +++++++------ .../spark/mllib/linalg/JavaVectorsSuite.java | 6 +++--- .../LogisticRegressionSuite.scala | 2 +- .../mllib/classification/NaiveBayesSuite.scala | 4 ++-- .../spark/mllib/classification/SVMSuite.scala | 6 +++--- .../optimization/GradientDescentSuite.scala | 13 ++++++------- .../spark/mllib/regression/LassoSuite.scala | 8 ++------ .../regression/LinearRegressionSuite.scala | 17 ++++++++++------- 9 files changed, 40 insertions(+), 35 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 01c1501548f87..21989d9a025a0 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 @@ -54,6 +54,12 @@ trait Vector extends Serializable { * Converts the instance to a breeze vector. */ private[mllib] def toBreeze: BV[Double] + + /** + * Gets the value of the ith element. + * @param i index + */ + private[mllib] def apply(i: Int): Double = toBreeze(i) } /** diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java index 073ded6f36933..c80b1134ed1b2 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java @@ -19,6 +19,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.regression.LabeledPoint; import org.junit.After; import org.junit.Assert; @@ -45,12 +46,12 @@ public void tearDown() { } private static final List POINTS = Arrays.asList( - new LabeledPoint(0, new double[] {1.0, 0.0, 0.0}), - new LabeledPoint(0, new double[] {2.0, 0.0, 0.0}), - new LabeledPoint(1, new double[] {0.0, 1.0, 0.0}), - new LabeledPoint(1, new double[] {0.0, 2.0, 0.0}), - new LabeledPoint(2, new double[] {0.0, 0.0, 1.0}), - new LabeledPoint(2, new double[] {0.0, 0.0, 2.0}) + new LabeledPoint(0, Vectors.dense(1.0, 0.0, 0.0)), + new LabeledPoint(0, Vectors.dense(2.0, 0.0, 0.0)), + new LabeledPoint(1, Vectors.dense(0.0, 1.0, 0.0)), + new LabeledPoint(1, Vectors.dense(0.0, 2.0, 0.0)), + new LabeledPoint(2, Vectors.dense(0.0, 0.0, 1.0)), + new LabeledPoint(2, Vectors.dense(0.0, 0.0, 2.0)) ); private int validatePrediction(List points, NaiveBayesModel model) { 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 index 2c4d795f96e4e..c6d8425ffc38d 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java @@ -19,10 +19,10 @@ import java.io.Serializable; -import com.google.common.collect.Lists; - import scala.Tuple2; +import com.google.common.collect.Lists; + import org.junit.Test; import static org.junit.Assert.*; @@ -36,7 +36,7 @@ public void denseArrayConstruction() { @Test public void sparseArrayConstruction() { - Vector v = Vectors.sparse(3, Lists.newArrayList( + 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/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index cce3aea0e2ad8..1e03c9df820b0 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 @@ -112,7 +112,7 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Shoul val testData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 42) val initialB = -1.0 - val initialWeights = Array(initialB) + val initialWeights = Vectors.dense(initialB) val testRDD = sc.parallelize(testData, 2) testRDD.cache() 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 9dd6c79ee6ad8..e83e3e73eedc4 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 @@ -19,11 +19,11 @@ package org.apache.spark.mllib.classification import scala.util.Random -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.linalg.Vectors object NaiveBayesSuite { @@ -54,7 +54,7 @@ object NaiveBayesSuite { if (rnd.nextDouble() < _theta(y)(j)) 1 else 0 } - LabeledPoint(y, xi) + LabeledPoint(y, Vectors.dense(xi)) } } } 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 bc7abb568a172..dc35d2483296d 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 @@ -20,7 +20,6 @@ package org.apache.spark.mllib.classification import scala.util.Random import scala.collection.JavaConversions._ -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import org.jblas.DoubleMatrix @@ -28,6 +27,7 @@ import org.jblas.DoubleMatrix 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 object SVMSuite { @@ -54,7 +54,7 @@ object SVMSuite { intercept + 0.01 * rnd.nextGaussian() if (yD < 0) 0.0 else 1.0 } - y.zip(x).map(p => LabeledPoint(p._1, p._2)) + y.zip(x).map(p => LabeledPoint(p._1, Vectors.dense(p._2))) } } @@ -110,7 +110,7 @@ class SVMSuite extends FunSuite with LocalSparkContext { val initialB = -1.0 val initialC = -1.0 - val initialWeights = Array(initialB,initialC) + val initialWeights = Vectors.dense(initialB, initialC) val testRDD = sc.parallelize(testData, 2) testRDD.cache() 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 631d0e2ad9cdb..9ec2749cc4872 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 @@ -20,13 +20,12 @@ package org.apache.spark.mllib.optimization import scala.util.Random import scala.collection.JavaConversions._ -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers -import org.apache.spark.SparkContext import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.linalg.Vectors object GradientDescentSuite { @@ -58,7 +57,7 @@ object GradientDescentSuite { if (yVal > 0) 1 else 0 } - val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Array(x1(i)))) + val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Vectors.dense(Array(x1(i))))) testData } } @@ -83,11 +82,11 @@ class GradientDescentSuite extends FunSuite with LocalSparkContext with ShouldMa // Add a extra variable consisting of all 1.0's for the intercept. val testData = GradientDescentSuite.generateGDInput(A, B, nPoints, 42) val data = testData.map { case LabeledPoint(label, features) => - label -> Array(1.0, features: _*) + label -> Vectors.dense(1.0, features.toArray: _*) } val dataRDD = sc.parallelize(data, 2).cache() - val initialWeightsWithIntercept = Array(1.0, initialWeights: _*) + val initialWeightsWithIntercept = Vectors.dense(0.0, initialWeights: _*) val (_, loss) = GradientDescent.runMiniBatchSGD( dataRDD, @@ -113,13 +112,13 @@ class GradientDescentSuite extends FunSuite with LocalSparkContext with ShouldMa // Add a extra variable consisting of all 1.0's for the intercept. val testData = GradientDescentSuite.generateGDInput(2.0, -1.5, 10000, 42) val data = testData.map { case LabeledPoint(label, features) => - label -> Array(1.0, features: _*) + label -> Vectors.dense(1.0, features.toArray: _*) } val dataRDD = sc.parallelize(data, 2).cache() // Prepare non-zero weights - val initialWeightsWithIntercept = Array(1.0, 0.5) + val initialWeightsWithIntercept = Vectors.dense(1.0, 0.5) val regParam0 = 0 val (newWeights0, loss0) = GradientDescent.runMiniBatchSGD( 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 64e4cbb860f61..bca4a3322484c 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,11 +17,9 @@ package org.apache.spark.mllib.regression - -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite -import org.apache.spark.SparkContext +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} class LassoSuite extends FunSuite with LocalSparkContext { @@ -51,7 +49,6 @@ class LassoSuite extends FunSuite with LocalSparkContext { ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(20) val model = ls.run(testRDD) - val weight0 = model.weights(0) val weight1 = model.weights(1) assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") @@ -79,7 +76,7 @@ class LassoSuite extends FunSuite with LocalSparkContext { val initialB = -1.0 val initialC = -1.0 - val initialWeights = Array(initialB,initialC) + val initialWeights = Vectors.dense(Array(initialB, initialC)) val testRDD = sc.parallelize(testData, 2) testRDD.cache() @@ -88,7 +85,6 @@ class LassoSuite extends FunSuite with LocalSparkContext { ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(20) val model = ls.run(testRDD, initialWeights) - val weight0 = model.weights(0) val weight1 = model.weights(1) assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") 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 5d251bcbf35db..aea3a2d6aa40f 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 @@ -40,11 +40,12 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext { linReg.optimizer.setNumIterations(1000).setStepSize(1.0) val model = linReg.run(testRDD) - assert(model.intercept >= 2.5 && model.intercept <= 3.5) - assert(model.weights.length === 2) - assert(model.weights(0) >= 9.0 && model.weights(0) <= 11.0) - assert(model.weights(1) >= 9.0 && model.weights(1) <= 11.0) + + val weights = model.weights + assert(weights.size === 2) + assert(weights(0) >= 9.0 && weights(0) <= 11.0) + assert(weights(1) >= 9.0 && weights(1) <= 11.0) val validationData = LinearDataGenerator.generateLinearInput( 3.0, Array(10.0, 10.0), 100, 17) @@ -67,9 +68,11 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext { val model = linReg.run(testRDD) assert(model.intercept === 0.0) - assert(model.weights.length === 2) - assert(model.weights(0) >= 9.0 && model.weights(0) <= 11.0) - assert(model.weights(1) >= 9.0 && model.weights(1) <= 11.0) + + val weights = model.weights + assert(weights.size === 2) + assert(weights(0) >= 9.0 && weights(0) <= 11.0) + assert(weights(1) >= 9.0 && weights(1) <= 11.0) val validationData = LinearDataGenerator.generateLinearInput( 0.0, Array(10.0, 10.0), 100, 17) From befa5929b2ecb7a7e966d1d88a8e9f94e0234cd8 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 26 Mar 2014 17:24:53 -0700 Subject: [PATCH 07/26] passed scala/java tests --- .../spark/mllib/optimization/Gradient.scala | 4 +-- .../mllib/optimization/GradientDescent.scala | 24 ++++++++---------- .../spark/mllib/optimization/Optimizer.scala | 2 +- .../spark/mllib/optimization/Updater.scala | 12 ++++++--- .../GeneralizedLinearAlgorithm.scala | 18 +++++++------ .../apache/spark/mllib/regression/Lasso.scala | 25 ++++++------------- .../mllib/regression/LinearRegression.scala | 20 +++++---------- .../org/apache/spark/mllib/util/MLUtils.scala | 8 +++--- .../optimization/GradientDescentSuite.scala | 5 ++-- .../spark/mllib/regression/LassoSuite.scala | 12 ++++----- .../spark/mllib/util/MLUtilsSuite.scala | 16 +++++++++++- 11 files changed, 74 insertions(+), 72 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 3239af9d5df47..9ecffb048907d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -42,7 +42,7 @@ abstract class Gradient extends Serializable { class LogisticGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { val brzData = data.toBreeze - val brzWeights = data.toBreeze + val brzWeights = weights.toBreeze val margin: Double = -1.0 * brzWeights.dot(brzData) val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label val gradient = brzData * gradientMultiplier @@ -67,7 +67,7 @@ class LeastSquaresGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { val brzData = data.toBreeze val brzWeights = weights.toBreeze - val diff: Double = brzWeights.dot(brzData) - label + val diff = brzWeights.dot(brzData) - label val loss = diff * diff val gradient = brzData * (2.0 * diff) 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 ad8868093435e..41752142247fc 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 @@ -17,11 +17,10 @@ package org.apache.spark.mllib.optimization -import org.apache.spark.Logging -import org.apache.spark.rdd.RDD - import scala.collection.mutable.ArrayBuffer +import org.apache.spark.Logging +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.{Vectors, Vector} /** @@ -92,16 +91,15 @@ class GradientDescent(var gradient: Gradient, var updater: Updater) } def optimize(data: RDD[(Double, Vector)], initialWeights: Vector): Vector = { - - val (weights, stochasticLossHistory) = GradientDescent.runMiniBatchSGD( - data, - gradient, - updater, - stepSize, - numIterations, - regParam, - miniBatchFraction, - initialWeights) + val (weights, _) = GradientDescent.runMiniBatchSGD( + data, + gradient, + updater, + stepSize, + numIterations, + regParam, + miniBatchFraction, + initialWeights) weights } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala index a62aecae5dd0d..f9ce908a5f3b0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala @@ -21,7 +21,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vector -trait Optimizer { +trait Optimizer extends Serializable { /** * Solve the provided convex optimization problem. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala index e5d4c4f7d282d..2766c8dbb42a0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala @@ -63,8 +63,12 @@ abstract class Updater extends Serializable { * Uses a step-size decreasing with the square root of the number of iterations. */ class SimpleUpdater extends Updater { - override def compute(weightsOld: Vector, gradient: Vector, - stepSize: Double, iter: Int, regParam: Double): (Vector, Double) = { + override def compute( + weightsOld: Vector, + gradient: Vector, + stepSize: Double, + iter: Int, + regParam: Double): (Vector, Double) = { val thisIterStepSize = stepSize / math.sqrt(iter) val brzWeights = weightsOld.toBreeze - gradient.toBreeze * thisIterStepSize (Vectors.fromBreeze(brzWeights), 0) @@ -101,9 +105,11 @@ class L1Updater extends Updater { val brzWeights = weightsOld.toBreeze - gradient.toBreeze * thisIterStepSize // Apply proximal operator (soft thresholding) val shrinkageVal = regParam * thisIterStepSize - (0 until brzWeights.length).foreach { i => + var i = 0 + while (i < brzWeights.length) { val wi = brzWeights(i) brzWeights(i) = signum(wi) * max(0.0, abs(wi) - shrinkageVal) + i += 1 } (Vectors.fromBreeze(brzWeights), brzNorm(brzWeights, 1.0) * regParam) 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 e6d550fe3782b..797698e4909d8 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 @@ -118,12 +118,12 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] /** Prepends one to the input vector. */ private def prependOne(vector: Vector): Vector = { - val vectorWithIntercept = vector.toBreeze match { + val vector1 = vector.toBreeze match { case dv: BDV[Double] => BDV.vertcat(BDV.ones[Double](1), dv) case sv: BSV[Double] => BSV.vertcat(new BSV[Double](Array(0), Array(1.0), 1), sv) case v: Any => throw new IllegalArgumentException("Do not support vector type " + v.getClass) } - Vectors.fromBreeze(vectorWithIntercept) + Vectors.fromBreeze(vector1) } /** @@ -151,10 +151,14 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] } val weightsWithIntercept = optimizer.optimize(data, initialWeightsWithIntercept) - val brzWeightsWithIntercept = weightsWithIntercept.toBreeze - val intercept = if (addIntercept) brzWeightsWithIntercept(0) else 0.0 - val brzWeights = if (addIntercept) brzWeightsWithIntercept(1 to -1) else brzWeightsWithIntercept - - createModel(Vectors.fromBreeze(brzWeights), intercept) + val intercept = if (addIntercept) weightsWithIntercept(0) else 0.0 + val weights = + if (addIntercept) { + Vectors.dense(weightsWithIntercept.toArray.slice(1, weightsWithIntercept.size)) + } else { + weightsWithIntercept + } + + createModel(weights, intercept) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index 2b6dbacde22d6..4034e79ae924a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -71,9 +71,9 @@ class LassoWithSGD private ( // We don't want to penalize the intercept, so set this to false. super.setIntercept(false) - var yMean = 0.0 - var xColMean: BV[Double] = _ - var xColSd: BV[Double] = _ + private var yMean = 0.0 + private var xColMean: BV[Double] = _ + private var xColSd: BV[Double] = _ /** * Construct a Lasso object with default parameters @@ -141,11 +141,8 @@ object LassoWithSGD { stepSize: Double, regParam: Double, miniBatchFraction: Double, - initialWeights: Vector) - : LassoModel = - { - new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input, - initialWeights) + initialWeights: Vector): LassoModel = { + new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input, initialWeights) } /** @@ -165,9 +162,7 @@ object LassoWithSGD { numIterations: Int, stepSize: Double, regParam: Double, - miniBatchFraction: Double) - : LassoModel = - { + miniBatchFraction: Double): LassoModel = { new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input) } @@ -187,9 +182,7 @@ object LassoWithSGD { input: RDD[LabeledPoint], numIterations: Int, stepSize: Double, - regParam: Double) - : LassoModel = - { + regParam: Double): LassoModel = { train(input, numIterations, stepSize, regParam, 1.0) } @@ -205,9 +198,7 @@ object LassoWithSGD { */ def train( input: RDD[LabeledPoint], - numIterations: Int) - : LassoModel = - { + numIterations: Int): LassoModel = { train(input, numIterations, 1.0, 1.0, 1.0) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 13c401aca31ef..6bc8850f8f6c6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -97,11 +97,9 @@ object LinearRegressionWithSGD { numIterations: Int, stepSize: Double, miniBatchFraction: Double, - initialWeights: Vector) - : LinearRegressionModel = - { - new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction).run(input, - initialWeights) + initialWeights: Vector): LinearRegressionModel = { + new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction) + .run(input, initialWeights) } /** @@ -119,9 +117,7 @@ object LinearRegressionWithSGD { input: RDD[LabeledPoint], numIterations: Int, stepSize: Double, - miniBatchFraction: Double) - : LinearRegressionModel = - { + miniBatchFraction: Double): LinearRegressionModel = { new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction).run(input) } @@ -139,9 +135,7 @@ object LinearRegressionWithSGD { def train( input: RDD[LabeledPoint], numIterations: Int, - stepSize: Double) - : LinearRegressionModel = - { + stepSize: Double): LinearRegressionModel = { train(input, numIterations, stepSize, 1.0) } @@ -157,9 +151,7 @@ object LinearRegressionWithSGD { */ def train( input: RDD[LabeledPoint], - numIterations: Int) - : LinearRegressionModel = - { + numIterations: Int): LinearRegressionModel = { train(input, numIterations, 1.0, 1.0) } 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 d495d590fd4f6..866596ded8d5c 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 @@ -82,13 +82,10 @@ object MLUtils { * xColMean - Row vector with mean for every column (or feature) of the input data * xColSd - Row vector standard deviation for every column (or feature) of the input data. */ - def computeStats(data: RDD[LabeledPoint], numFeatures: Int, numExamples: Long) - : (Double, Vector, Vector) = { - + def computeStats(data: RDD[LabeledPoint], numFeatures: Int, numExamples: Long): (Double, Vector, Vector) = { val brzData = data.map { case LabeledPoint(label, features) => (label, features.toBreeze) } - val aggStats = brzData.aggregate( (0L, 0.0, BDV.zeros[Double](numFeatures), BDV.zeros[Double](numFeatures)) )( @@ -104,9 +101,10 @@ object MLUtils { (n1 + n2, sumLabel1 + sumLabel2, sum1 += sum2, sumSq1 += sumSq2) } ) - val (nl, sumLabel, sum, sumSq) = aggStats + require(nl > 0, "Input data is empty.") + require(nl == numExamples) val n = nl.toDouble val yMean = sumLabel / n 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 9ec2749cc4872..c4b433499a091 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 @@ -57,8 +57,7 @@ object GradientDescentSuite { if (yVal > 0) 1 else 0 } - val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Vectors.dense(Array(x1(i))))) - testData + (0 until nPoints).map(i => LabeledPoint(y(i), Vectors.dense(x1(i)))) } } @@ -86,7 +85,7 @@ class GradientDescentSuite extends FunSuite with LocalSparkContext with ShouldMa } val dataRDD = sc.parallelize(data, 2).cache() - val initialWeightsWithIntercept = Vectors.dense(0.0, initialWeights: _*) + val initialWeightsWithIntercept = Vectors.dense(1.0, initialWeights: _*) val (_, loss) = GradientDescent.runMiniBatchSGD( dataRDD, 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 bca4a3322484c..02e99fac6146f 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 @@ -34,7 +34,7 @@ class LassoSuite extends FunSuite with LocalSparkContext { } test("Lasso local random SGD") { - val nPoints = 10000 + val nPoints = 1000 val A = 2.0 val B = -1.5 @@ -46,7 +46,7 @@ class LassoSuite extends FunSuite with LocalSparkContext { testRDD.cache() val ls = new LassoWithSGD() - ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(20) + ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(40) val model = ls.run(testRDD) val weight0 = model.weights(0) @@ -66,23 +66,23 @@ class LassoSuite extends FunSuite with LocalSparkContext { } test("Lasso local random SGD with initial weights") { - val nPoints = 10000 + val nPoints = 1000 val A = 2.0 val B = -1.5 val C = 1.0e-2 - val testData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 42) + val testData = LinearDataGenerator.generateLinearInput(A, Array[Double](B, C), nPoints, 42) val initialB = -1.0 val initialC = -1.0 - val initialWeights = Vectors.dense(Array(initialB, initialC)) + val initialWeights = Vectors.dense(initialB, initialC) val testRDD = sc.parallelize(testData, 2) testRDD.cache() val ls = new LassoWithSGD() - ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(20) + ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(40) val model = ls.run(testRDD, initialWeights) val weight0 = model.weights(0) 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 60f053b381305..2081fe46b17ef 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 @@ -23,8 +23,10 @@ import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, norm => breezeNor squaredDistance => breezeSquaredDistance} import org.apache.spark.mllib.util.MLUtils._ +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint -class MLUtilsSuite extends FunSuite { +class MLUtilsSuite extends FunSuite with LocalSparkContext { test("epsilon computation") { assert(1.0 + EPSILON > 1.0, s"EPSILON is too small: $EPSILON.") @@ -49,4 +51,16 @@ class MLUtilsSuite extends FunSuite { assert((fastSquaredDist2 - squaredDist) <= precision * squaredDist, s"failed with m = $m") } } + + test("compute stats") { + val data = Seq.fill(3)(Seq( + LabeledPoint(1.0, Vectors.dense(1.0, 2.0, 3.0)), + LabeledPoint(0.0, Vectors.dense(3.0, 4.0, 5.0)) + )).flatten + val rdd = sc.parallelize(data, 2) + val (meanLabel, mean, std) = MLUtils.computeStats(rdd, 3, 6) + assert(meanLabel === 0.5) + assert(mean === Vectors.dense(2.0, 3.0, 4.0)) + assert(std === Vectors.dense(1.0, 1.0, 1.0)) + } } From db808a156d1a298597ae4590987d20c984a14e49 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 26 Mar 2014 17:50:37 -0700 Subject: [PATCH 08/26] update JavaLR example --- .../org/apache/spark/mllib/examples/JavaLR.java | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java index 667c72f379e71..cd8879ff886e2 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java @@ -17,6 +17,7 @@ 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; @@ -24,11 +25,9 @@ import org.apache.spark.mllib.classification.LogisticRegressionWithSGD; import org.apache.spark.mllib.classification.LogisticRegressionModel; +import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.regression.LabeledPoint; -import java.util.Arrays; -import java.util.regex.Pattern; - /** * Logistic regression based classification using ML Lib. */ @@ -47,14 +46,10 @@ public LabeledPoint call(String line) { for (int i = 0; i < tok.length; ++i) { x[i] = Double.parseDouble(tok[i]); } - return new LabeledPoint(y, x); + return new LabeledPoint(y, Vectors.dense(x)); } } - public static void printWeights(double[] a) { - System.out.println(Arrays.toString(a)); - } - public static void main(String[] args) { if (args.length != 4) { System.err.println("Usage: JavaLR "); @@ -80,8 +75,7 @@ public static void main(String[] args) { LogisticRegressionModel model = LogisticRegressionWithSGD.train(points.rdd(), iterations, stepSize); - System.out.print("Final w: "); - printWeights(model.weights()); + System.out.print("Final w: " + model.weights()); System.exit(0); } From e981396be53e9a00552a137e9a47c52536922dbb Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 26 Mar 2014 19:00:08 -0700 Subject: [PATCH 09/26] use axpy in Updater --- .../mllib/optimization/GradientDescent.scala | 17 ++++++++++++----- .../spark/mllib/optimization/Updater.scala | 14 +++++++++----- 2 files changed, 21 insertions(+), 10 deletions(-) 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 41752142247fc..8131925cfc87d 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 @@ -19,6 +19,8 @@ package org.apache.spark.mllib.optimization import scala.collection.mutable.ArrayBuffer +import breeze.linalg.{Vector => BV, DenseVector => BDV} + import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.{Vectors, Vector} @@ -157,11 +159,16 @@ object GradientDescent extends Logging { for (i <- 1 to numIterations) { // 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).map { - case (y, features) => - val (grad, loss) = gradient.compute(features, y, weights) - (grad.toBreeze, loss) - }.reduce((a, b) => (a._1 += b._1, a._2 + b._2)) + val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42 + i) + .aggregate((BDV.zeros[Double](weights.size), 0.0))( + seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) => + val (g, l) = gradient.compute(features, label, weights) + (grad += g.toBreeze, loss + l) + }, + combOp = (c1, c2) => (c1, c2) match { case ((grad1, loss1), (grad2, loss2)) => + (grad1 += grad2, loss1 + loss2) + } + ) /** * NOTE(Xinghao): lossSum is computed using the weights from the previous iteration diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala index 2766c8dbb42a0..3b7754cd7ac28 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.optimization import scala.math._ -import breeze.linalg.{norm => brzNorm} +import breeze.linalg.{norm => brzNorm, axpy => brzAxpy, Vector => BV} import org.apache.spark.mllib.linalg.{Vectors, Vector} @@ -70,7 +70,9 @@ class SimpleUpdater extends Updater { iter: Int, regParam: Double): (Vector, Double) = { val thisIterStepSize = stepSize / math.sqrt(iter) - val brzWeights = weightsOld.toBreeze - gradient.toBreeze * thisIterStepSize + val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector + brzAxpy(-thisIterStepSize, gradient.toBreeze, brzWeights) + (Vectors.fromBreeze(brzWeights), 0) } } @@ -102,7 +104,8 @@ class L1Updater extends Updater { regParam: Double): (Vector, Double) = { val thisIterStepSize = stepSize / math.sqrt(iter) // Take gradient step - val brzWeights = weightsOld.toBreeze - gradient.toBreeze * thisIterStepSize + val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector + brzAxpy(-thisIterStepSize, gradient.toBreeze, brzWeights) // Apply proximal operator (soft thresholding) val shrinkageVal = regParam * thisIterStepSize var i = 0 @@ -133,8 +136,9 @@ class SquaredL2Updater extends Updater { // w' = w - thisIterStepSize * (gradient + regParam * w) // w' = (1 - thisIterStepSize * regParam) * w - thisIterStepSize * gradient val thisIterStepSize = stepSize / math.sqrt(iter) - val brzWeights = weightsOld.toBreeze * (1.0 - thisIterStepSize * regParam) - - (gradient.toBreeze * thisIterStepSize) + val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector + brzWeights :*= (1.0 - thisIterStepSize * regParam) + brzAxpy(-thisIterStepSize, gradient.toBreeze, brzWeights) val norm = brzNorm(brzWeights, 2.0) (Vectors.fromBreeze(brzWeights), 0.5 * regParam * norm * norm) From 44733e1a4bc564c4bd6638d8a9a70a2b007a5d07 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 26 Mar 2014 19:12:32 -0700 Subject: [PATCH 10/26] use in-place gradient computation --- .../spark/mllib/optimization/Gradient.scala | 57 +++++++++++++++++++ .../mllib/optimization/GradientDescent.scala | 7 +-- 2 files changed, 60 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 9ecffb048907d..a174d4e77d33e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -17,6 +17,8 @@ package org.apache.spark.mllib.optimization +import breeze.linalg.{axpy => brzAxpy} + import org.apache.spark.mllib.linalg.{Vectors, Vector} /** @@ -33,6 +35,19 @@ abstract class Gradient extends Serializable { * @return (gradient: Vector, loss: Double) */ def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) + + /** + * Compute the gradient and loss given the features of a single data point, add the gradient to a provided vector to + * avoid creating new objects, and return loss. + * + * @param data features for one data point + * @param label label for this data point + * @param weights weights/coefficients corresponding to features + * @param gradientAddTo gradient will be added to this vector + * + * @return (gradient: Vector, loss: Double) + */ + def compute(data: Vector, label: Double, weights: Vector, gradientAddTo: Vector): Double } /** @@ -55,6 +70,21 @@ class LogisticGradient extends Gradient { (Vectors.fromBreeze(gradient), loss) } + + override def compute(data: Vector, label: Double, weights: Vector, gradientAddTo: Vector): Double = { + val brzData = data.toBreeze + val brzWeights = weights.toBreeze + val margin: Double = -1.0 * brzWeights.dot(brzData) + val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label + + brzAxpy(gradientMultiplier, brzData, gradientAddTo.toBreeze) + + if (label > 0) { + math.log(1 + math.exp(margin)) + } else { + math.log(1 + math.exp(margin)) - margin + } + } } /** @@ -73,6 +103,16 @@ class LeastSquaresGradient extends Gradient { (Vectors.fromBreeze(gradient), loss) } + + override def compute(data: Vector, label: Double, weights: Vector, gradientAddTo: Vector): Double = { + val brzData = data.toBreeze + val brzWeights = weights.toBreeze + val diff = brzWeights.dot(brzData) - label + + brzAxpy(2.0 * diff, brzData, gradientAddTo.toBreeze) + + diff * diff + } } /** @@ -96,4 +136,21 @@ class HingeGradient extends Gradient { (Vectors.dense(new Array[Double](weights.size)), 0.0) } } + + override def compute(data: Vector, label: Double, weights: Vector, gradientAddTo: Vector): Double = { + val brzData = data.toBreeze + val brzWeights = weights.toBreeze + val dotProduct = brzWeights.dot(brzData) + + // Our loss function with {0, 1} labels is max(0, 1 - (2y – 1) (f_w(x))) + // Therefore the gradient is -(2y - 1)*x + val labelScaled = 2 * label - 1.0 + + if (1.0 > labelScaled * dotProduct) { + brzAxpy(-labelScaled, brzData, gradientAddTo.toBreeze) + 1.0 - labelScaled * dotProduct + } else { + 0.0 + } + } } 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 8131925cfc87d..d0777ffd63ff8 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 @@ -162,13 +162,12 @@ object GradientDescent extends Logging { val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42 + i) .aggregate((BDV.zeros[Double](weights.size), 0.0))( seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) => - val (g, l) = gradient.compute(features, label, weights) - (grad += g.toBreeze, loss + l) + val l = gradient.compute(features, label, weights, Vectors.fromBreeze(grad)) + (grad, loss + l) }, combOp = (c1, c2) => (c1, c2) match { case ((grad1, loss1), (grad2, loss2)) => (grad1 += grad2, loss1 + loss2) - } - ) + }) /** * NOTE(Xinghao): lossSum is computed using the weights from the previous iteration From f0fe616669503ba4c3c46397f36ea34dc9e12010 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 26 Mar 2014 20:22:49 -0700 Subject: [PATCH 11/26] add a test for sparse linear regression --- .../regression/LinearRegressionSuite.scala | 34 +++++++++++++++++++ 1 file changed, 34 insertions(+) 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 aea3a2d6aa40f..efffab743d5f4 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,6 +20,7 @@ package org.apache.spark.mllib.regression import org.scalatest.FunSuite import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} +import org.apache.spark.mllib.linalg.Vectors class LinearRegressionSuite extends FunSuite with LocalSparkContext { @@ -84,4 +85,37 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext { // Test prediction on Array. validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } + + // Test if we can correctly learn Y = 10*X1 + 10*X10000 + test("sparse linear regression without intercept") { + val denseRDD = sc.parallelize(LinearDataGenerator.generateLinearInput(0.0, Array(10.0, 10.0), 100, 42), 2) + val sparseRDD = denseRDD.map { case LabeledPoint(label, v) => + val sv = Vectors.sparse(10000, Seq((0, v(0)), (9999, v(1)))) + LabeledPoint(label, sv) + }.cache() + val linReg = new LinearRegressionWithSGD().setIntercept(false) + linReg.optimizer.setNumIterations(1000).setStepSize(1.0) + + val model = linReg.run(sparseRDD) + + assert(model.intercept === 0.0) + + val weights = model.weights + assert(weights.size === 10000) + assert(weights(0) >= 9.0 && weights(0) <= 11.0) + assert(weights(9999) >= 9.0 && weights(9999) <= 11.0) + + val validationData = LinearDataGenerator.generateLinearInput(0.0, Array(10.0, 10.0), 100, 17) + val sparseValidationData = validationData.map { case LabeledPoint(label, v) => + val sv = Vectors.sparse(10000, Seq((0, v(0)), (9999, v(1)))) + LabeledPoint(label, sv) + } + val sparseValidationRDD = sc.parallelize(sparseValidationData, 2) + + // Test prediction on RDD. + validatePrediction(model.predict(sparseValidationRDD.map(_.features)).collect(), sparseValidationData) + + // Test prediction on Array. + validatePrediction(sparseValidationData.map(row => model.predict(row.features)), sparseValidationData) + } } From 78c467164ec6f598230c34754a3c8ffedb6f333b Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 27 Mar 2014 01:27:30 -0700 Subject: [PATCH 12/26] add libSVMFile to MLContext --- .../org/apache/spark/mllib/MLContext.scala | 61 +++++++++++++++++++ .../apache/spark/mllib/MLContextSuite.scala | 51 ++++++++++++++++ 2 files changed, 112 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala b/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala new file mode 100644 index 0000000000000..e678eaaa37810 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala @@ -0,0 +1,61 @@ +/* + * 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 + +import org.apache.spark.SparkContext + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.rdd.RDD + +class MLContext(self: SparkContext) { + /** + * Reads labeled data in the LIBSVM format into an RDD[LabeledPoint]. + * The LIBSVM format is a text-based format used by LIBSVM (http://www.csie.ntu.edu.tw/~cjlin/libsvm/). + * Each line represents a labeled sparse feature vector using the following format: + * {{{label index1:value1 index2:value2 ...}}} + * where the indices are one-based and in ascending order. + * This method parses each line into a [[org.apache.spark.mllib.regression.LabeledPoint]] instance, + * where the feature indices are converted to zero-based. + * + * @param path file or directory path in any Hadoop-supported file system URI + * @param numFeatures number of features + * @param labelParser parser for labels, default: _.toDouble + * @return labeled data stored as an RDD[LabeledPoint] + */ + def libSVMFile( + path: String, + numFeatures: Int, + labelParser: String => Double = _.toDouble): RDD[LabeledPoint] = { + self.textFile(path).map(_.trim).filter(!_.isEmpty).map { line => + val items = line.split(' ') + val label = labelParser(items.head) + val features = Vectors.sparse(numFeatures, items.tail.map { item => + val indexAndValue = item.split(':') + val index = indexAndValue(0).toInt - 1 + val value = indexAndValue(1).toDouble + (index, value) + }) + LabeledPoint(label, features) + } + } +} + +object MLContext { + implicit def sparkContextToMLContext(sc: SparkContext): MLContext = new MLContext(sc) +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala new file mode 100644 index 0000000000000..02697d983588e --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala @@ -0,0 +1,51 @@ +/* + * 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 + +import org.apache.spark.mllib.MLContext._ +import org.apache.spark.mllib.util.LocalSparkContext +import org.scalatest.FunSuite +import com.google.common.io.Files +import java.io.File +import com.google.common.base.Charsets +import org.apache.spark.mllib.linalg.Vectors + +class MLContextSuite extends FunSuite with LocalSparkContext { + test("libSVMFile") { + val lines = + """ + |1 1:1.0 3:2.0 5:3.0 + |0 2:4.0 4:5.0 6:6.0 + """.stripMargin + val tempDir = Files.createTempDir() + val file = new File(tempDir.getPath, "part-00000") + Files.write(lines, file, Charsets.US_ASCII) + val points = sc.libSVMFile(tempDir.toURI.toString, 6).collect() + assert(points.length === 2) + assert(points(0).label === 1.0) + assert(points(0).features === Vectors.sparse(6, Seq((0, 1.0), (2, 2.0), (4, 3.0)))) + assert(points(1).label === 0.0) + assert(points(1).features === Vectors.sparse(6, Seq((1, 4.0), (3, 5.0), (5, 6.0)))) + try { + file.delete() + tempDir.delete() + } catch { + case t: Throwable => + } + } +} From b11659cf4eb4f94868b6d07e8f5dbddae56e00b9 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 27 Mar 2014 02:11:09 -0700 Subject: [PATCH 13/26] style update --- .../org/apache/spark/mllib/MLContext.scala | 4 +-- .../spark/mllib/optimization/Gradient.scala | 34 +++++++++++++------ .../apache/spark/mllib/regression/Lasso.scala | 9 ++--- .../mllib/regression/LinearRegression.scala | 2 +- .../mllib/regression/RidgeRegression.scala | 14 ++++---- .../org/apache/spark/mllib/util/MLUtils.scala | 5 ++- .../apache/spark/mllib/MLContextSuite.scala | 11 +++--- .../classification/NaiveBayesSuite.scala | 2 +- .../regression/LinearRegressionSuite.scala | 11 +++--- 9 files changed, 57 insertions(+), 35 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala b/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala index e678eaaa37810..eefca193ec53e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala @@ -26,11 +26,11 @@ import org.apache.spark.rdd.RDD class MLContext(self: SparkContext) { /** * Reads labeled data in the LIBSVM format into an RDD[LabeledPoint]. - * The LIBSVM format is a text-based format used by LIBSVM (http://www.csie.ntu.edu.tw/~cjlin/libsvm/). + * The LIBSVM format is a text-based format used by LIBSVM and LIBLINEAR. * Each line represents a labeled sparse feature vector using the following format: * {{{label index1:value1 index2:value2 ...}}} * where the indices are one-based and in ascending order. - * This method parses each line into a [[org.apache.spark.mllib.regression.LabeledPoint]] instance, + * This method parses each line into a [[org.apache.spark.mllib.regression.LabeledPoint]], * where the feature indices are converted to zero-based. * * @param path file or directory path in any Hadoop-supported file system URI diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index a174d4e77d33e..20654284965ed 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -37,17 +37,17 @@ abstract class Gradient extends Serializable { def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) /** - * Compute the gradient and loss given the features of a single data point, add the gradient to a provided vector to - * avoid creating new objects, and return loss. + * Compute the gradient and loss given the features of a single data point, + * add the gradient to a provided vector to avoid creating new objects, and return loss. * * @param data features for one data point * @param label label for this data point * @param weights weights/coefficients corresponding to features - * @param gradientAddTo gradient will be added to this vector + * @param cumGradient the computed gradient will be added to this vector * - * @return (gradient: Vector, loss: Double) + * @return loss */ - def compute(data: Vector, label: Double, weights: Vector, gradientAddTo: Vector): Double + def compute(data: Vector, label: Double, weights: Vector, cumGradient: Vector): Double } /** @@ -71,13 +71,17 @@ class LogisticGradient extends Gradient { (Vectors.fromBreeze(gradient), loss) } - override def compute(data: Vector, label: Double, weights: Vector, gradientAddTo: Vector): Double = { + override def compute( + data: Vector, + label: Double, + weights: Vector, + cumGradient: Vector): Double = { val brzData = data.toBreeze val brzWeights = weights.toBreeze val margin: Double = -1.0 * brzWeights.dot(brzData) val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label - brzAxpy(gradientMultiplier, brzData, gradientAddTo.toBreeze) + brzAxpy(gradientMultiplier, brzData, cumGradient.toBreeze) if (label > 0) { math.log(1 + math.exp(margin)) @@ -104,12 +108,16 @@ class LeastSquaresGradient extends Gradient { (Vectors.fromBreeze(gradient), loss) } - override def compute(data: Vector, label: Double, weights: Vector, gradientAddTo: Vector): Double = { + override def compute( + data: Vector, + label: Double, + weights: Vector, + cumGradient: Vector): Double = { val brzData = data.toBreeze val brzWeights = weights.toBreeze val diff = brzWeights.dot(brzData) - label - brzAxpy(2.0 * diff, brzData, gradientAddTo.toBreeze) + brzAxpy(2.0 * diff, brzData, cumGradient.toBreeze) diff * diff } @@ -137,7 +145,11 @@ class HingeGradient extends Gradient { } } - override def compute(data: Vector, label: Double, weights: Vector, gradientAddTo: Vector): Double = { + override def compute( + data: Vector, + label: Double, + weights: Vector, + cumGradient: Vector): Double = { val brzData = data.toBreeze val brzWeights = weights.toBreeze val dotProduct = brzWeights.dot(brzData) @@ -147,7 +159,7 @@ class HingeGradient extends Gradient { val labelScaled = 2 * label - 1.0 if (1.0 > labelScaled * dotProduct) { - brzAxpy(-labelScaled, brzData, gradientAddTo.toBreeze) + brzAxpy(-labelScaled, brzData, cumGradient.toBreeze) 1.0 - labelScaled * dotProduct } else { 0.0 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index 4034e79ae924a..911e0c0178cab 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -17,13 +17,13 @@ package org.apache.spark.mllib.regression +import breeze.linalg.{Vector => BV} + import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.mllib.linalg.{Vector, Vectors} - -import breeze.linalg.{Vector => BV, DenseVector => BDV} /** * Regression model trained using Lasso. @@ -142,7 +142,8 @@ object LassoWithSGD { regParam: Double, miniBatchFraction: Double, initialWeights: Vector): LassoModel = { - new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input, initialWeights) + new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction) + .run(input, initialWeights) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 6bc8850f8f6c6..9ed927994e795 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -19,9 +19,9 @@ package org.apache.spark.mllib.regression import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.mllib.linalg.{Vector, Vectors} /** * Regression model trained using LinearRegression. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index 22a0e8b495957..0f10e48af02e4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -17,14 +17,14 @@ package org.apache.spark.mllib.regression +import breeze.linalg.{Vector => BV} + import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.mllib.linalg.{Vectors, Vector} -import breeze.linalg.{Vector => BV, DenseVector => BDV} - /** * Regression model trained using RidgeRegression. * @@ -72,9 +72,9 @@ class RidgeRegressionWithSGD private ( // We don't want to penalize the intercept in RidgeRegression, so set this to false. super.setIntercept(false) - var yMean = 0.0 - var xColMean: BV[Double] = _ - var xColSd: BV[Double] = _ + private var yMean = 0.0 + private var xColMean: BV[Double] = _ + private var xColSd: BV[Double] = _ /** * Construct a RidgeRegression object with default parameters @@ -214,8 +214,8 @@ object RidgeRegressionWithSGD { def main(args: Array[String]) { if (args.length != 5) { - println("Usage: RidgeRegression " + - " ") + println("Usage: RidgeRegression " + + " ") System.exit(1) } val sc = new SparkContext(args(0), "RidgeRegression") 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 866596ded8d5c..68c56dd6a6f1d 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 @@ -82,7 +82,10 @@ object MLUtils { * xColMean - Row vector with mean for every column (or feature) of the input data * xColSd - Row vector standard deviation for every column (or feature) of the input data. */ - def computeStats(data: RDD[LabeledPoint], numFeatures: Int, numExamples: Long): (Double, Vector, Vector) = { + def computeStats( + data: RDD[LabeledPoint], + numFeatures: Int, + numExamples: Long): (Double, Vector, Vector) = { val brzData = data.map { case LabeledPoint(label, features) => (label, features.toBreeze) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala index 02697d983588e..6762f8c479e98 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala @@ -17,13 +17,16 @@ package org.apache.spark.mllib -import org.apache.spark.mllib.MLContext._ -import org.apache.spark.mllib.util.LocalSparkContext -import org.scalatest.FunSuite -import com.google.common.io.Files import java.io.File + +import org.scalatest.FunSuite + import com.google.common.base.Charsets +import com.google.common.io.Files + +import org.apache.spark.mllib.MLContext._ import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.LocalSparkContext class MLContextSuite extends FunSuite with LocalSparkContext { test("libSVMFile") { 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 e83e3e73eedc4..516895d04222d 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 @@ -21,9 +21,9 @@ import scala.util.Random 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.linalg.Vectors object NaiveBayesSuite { 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 efffab743d5f4..2f7d30708ce17 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 @@ -19,8 +19,8 @@ package org.apache.spark.mllib.regression 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.{LinearDataGenerator, LocalSparkContext} class LinearRegressionSuite extends FunSuite with LocalSparkContext { @@ -88,7 +88,8 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext { // Test if we can correctly learn Y = 10*X1 + 10*X10000 test("sparse linear regression without intercept") { - val denseRDD = sc.parallelize(LinearDataGenerator.generateLinearInput(0.0, Array(10.0, 10.0), 100, 42), 2) + val denseRDD = sc.parallelize( + LinearDataGenerator.generateLinearInput(0.0, Array(10.0, 10.0), 100, 42), 2) val sparseRDD = denseRDD.map { case LabeledPoint(label, v) => val sv = Vectors.sparse(10000, Seq((0, v(0)), (9999, v(1)))) LabeledPoint(label, sv) @@ -113,9 +114,11 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext { val sparseValidationRDD = sc.parallelize(sparseValidationData, 2) // Test prediction on RDD. - validatePrediction(model.predict(sparseValidationRDD.map(_.features)).collect(), sparseValidationData) + validatePrediction( + model.predict(sparseValidationRDD.map(_.features)).collect(), sparseValidationData) // Test prediction on Array. - validatePrediction(sparseValidationData.map(row => model.predict(row.features)), sparseValidationData) + validatePrediction( + sparseValidationData.map(row => model.predict(row.features)), sparseValidationData) } } From 0f8759b3a8bd6d795e914e43224e7f0594c8f7f9 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 30 Mar 2014 14:31:34 -0700 Subject: [PATCH 14/26] minor updates to NB --- .../apache/spark/mllib/classification/NaiveBayes.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) 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 f4228fe5e7522..924ab43f26e06 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 @@ -20,8 +20,10 @@ package org.apache.spark.mllib.classification import scala.collection.mutable import org.jblas.DoubleMatrix +import breeze.linalg.{Vector => BV} import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.SparkContext._ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD import org.apache.spark.mllib.util.MLUtils @@ -76,7 +78,13 @@ class NaiveBayes private (var lambda: Double) * @param data RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. */ def run(data: RDD[LabeledPoint]) = { - runRaw(data.map(v => (v.label, v.features.toArray))) + val agg = data.map(p => (p.label, p.features)).combineByKey[(Long, BV[Double])]( + createCombiner = (v: Vector) => (1L, v.toBreeze.toDenseVector), + mergeValue = (c: (Long, BV[Double]), v: Vector) => (c._1 + 1L, c._2 += v.toBreeze), + mergeCombiners = (c1: (Long, BV[Double]), c2: (Long, BV[Double])) => + (c1._1 + c2._1, c1._2 += c2._2) + ).collect() + val numLabels = agg.size } /** From 3432e842860d81bda39787a80279f4340257fc42 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 30 Mar 2014 17:15:30 -0700 Subject: [PATCH 15/26] update NaiveBayes to support sparse data --- .../mllib/api/python/PythonMLLibAPI.scala | 1 + .../mllib/classification/NaiveBayes.scala | 122 ++++++++---------- python/pyspark/mllib/classification.py | 12 +- 3 files changed, 61 insertions(+), 74 deletions(-) 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 c5729ee17c375..757aea32ef94e 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 @@ -247,6 +247,7 @@ class PythonMLLibAPI extends Serializable { }) val model = NaiveBayes.train(data, lambda) val ret = new java.util.LinkedList[java.lang.Object]() + ret.add(serializeDoubleVector(model.labels)) ret.add(serializeDoubleVector(model.pi)) ret.add(serializeDoubleMatrix(model.theta)) ret 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 924ab43f26e06..fb6567ac2e431 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 @@ -17,17 +17,14 @@ package org.apache.spark.mllib.classification -import scala.collection.mutable +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum} -import org.jblas.DoubleMatrix -import breeze.linalg.{Vector => BV} - -import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.{Logging, SparkContext} import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.rdd.RDD import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.rdd.RDD /** * Model for Naive Bayes Classifiers. @@ -35,21 +32,28 @@ import org.apache.spark.mllib.linalg.Vector * @param pi Log of class priors, whose dimension is C. * @param theta Log of class conditional probabilities, whose dimension is CxD. */ -class NaiveBayesModel(val pi: Array[Double], val theta: Array[Array[Double]]) - extends ClassificationModel with Serializable { - - // Create a column vector that can be used for predictions - private val _pi = new DoubleMatrix(pi.length, 1, pi: _*) - private val _theta = new DoubleMatrix(theta) +class NaiveBayesModel( + val labels: Array[Double], + val pi: Array[Double], + val theta: Array[Array[Double]]) extends ClassificationModel with Serializable { + + private val brzPi = new BDV[Double](pi) + private val brzTheta = new BDM[Double](theta.length, theta(0).length) + + var i = 0 + while (i < theta.length) { + var j = 0 + while (j < theta(i).length) { + brzTheta(i, j) = theta(i)(j) + j += 1 + } + i += 1 + } override def predict(testData: RDD[Vector]): RDD[Double] = testData.map(predict) - override def predict(testData: Vector): Double = predict(testData.toArray) - - private def predict(testData: Array[Double]): Double = { - val dataMatrix = new DoubleMatrix(testData.length, 1, testData: _*) - val result = _pi.add(_theta.mmul(dataMatrix)) - result.argmax() + override def predict(testData: Vector): Double = { + labels(brzArgmax(brzPi + brzTheta * testData.toBreeze)) } } @@ -61,9 +65,8 @@ class NaiveBayesModel(val pi: Array[Double], val theta: Array[Array[Double]]) * document classification. By making every vector a 0-1 vector, it can also be used as * Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). */ -class NaiveBayes private (var lambda: Double) - extends Serializable with Logging -{ +class NaiveBayes private (var lambda: Double) extends Serializable with Logging { + def this() = this(1.0) /** Set the smoothing parameter. Default: 1.0. */ @@ -78,57 +81,37 @@ class NaiveBayes private (var lambda: Double) * @param data RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. */ def run(data: RDD[LabeledPoint]) = { - val agg = data.map(p => (p.label, p.features)).combineByKey[(Long, BV[Double])]( + // Aggregates term frequencies per label. + val aggregated = data.map(p => (p.label, p.features)).combineByKey[(Long, BDV[Double])]( createCombiner = (v: Vector) => (1L, v.toBreeze.toDenseVector), - mergeValue = (c: (Long, BV[Double]), v: Vector) => (c._1 + 1L, c._2 += v.toBreeze), - mergeCombiners = (c1: (Long, BV[Double]), c2: (Long, BV[Double])) => + mergeValue = (c: (Long, BDV[Double]), v: Vector) => (c._1 + 1L, c._2 += v.toBreeze), + mergeCombiners = (c1: (Long, BDV[Double]), c2: (Long, BDV[Double])) => (c1._1 + c2._1, c1._2 += c2._2) ).collect() - val numLabels = agg.size - } - - /** - * Run the algorithm with the configured parameters on an input RDD of LabeledPoint entries. - * - * @param data RDD of (label, array of features) pairs. - */ - private def runRaw(data: RDD[(Double, Array[Double])]) = { - // Aggregates all sample points to driver side to get sample count and summed feature vector - // for each label. The shape of `zeroCombiner` & `aggregated` is: - // - // label: Int -> (count: Int, featuresSum: DoubleMatrix) - val zeroCombiner = mutable.Map.empty[Int, (Int, DoubleMatrix)] - val aggregated = data.aggregate(zeroCombiner)({ (combiner, point) => - point match { - case (label, features) => - val (count, featuresSum) = combiner.getOrElse(label.toInt, (0, DoubleMatrix.zeros(1))) - val fs = new DoubleMatrix(features.length, 1, features: _*) - combiner += label.toInt -> (count + 1, featuresSum.addi(fs)) - } - }, { (lhs, rhs) => - for ((label, (c, fs)) <- rhs) { - val (count, featuresSum) = lhs.getOrElse(label, (0, DoubleMatrix.zeros(1))) - lhs(label) = (count + c, featuresSum.addi(fs)) + val numLabels = aggregated.length + var numExamples = 0L + aggregated.foreach { case (_, (n, _)) => + numExamples += n + } + val numFeatures = aggregated.head match { case (_, (_, v)) => v.size } + val labels = new Array[Double](numLabels) + val pi = new Array[Double](numLabels) + val theta = Array.fill(numLabels)(new Array[Double](numFeatures)) + val piLogDenom = math.log(numExamples + numLabels * lambda) + var i = 0 + aggregated.foreach { case (label, (n, sum)) => + labels(i) = label + val thetaLogDenom = math.log(brzSum(sum) + numFeatures * lambda) + pi(i) = math.log(n + lambda) - piLogDenom + var j = 0 + while (j < numFeatures) { + theta(i)(j) = math.log(sum(j) + lambda) - thetaLogDenom + j += 1 } - lhs - }) - - // Kinds of label - val C = aggregated.size - // Total sample count - val N = aggregated.values.map(_._1).sum - - val pi = new Array[Double](C) - val theta = new Array[Array[Double]](C) - val piLogDenom = math.log(N + C * lambda) - - for ((label, (count, fs)) <- aggregated) { - val thetaLogDenom = math.log(fs.sum() + fs.length * lambda) - pi(label) = math.log(count + lambda) - piLogDenom - theta(label) = fs.toArray.map(f => math.log(f + lambda) - thetaLogDenom) + i += 1 } - new NaiveBayesModel(pi, theta) + new NaiveBayesModel(labels, pi, theta) } } @@ -178,8 +161,9 @@ object NaiveBayes { } else { NaiveBayes.train(data, args(2).toDouble) } - println("Pi: " + model.pi.mkString("[", ", ", "]")) - println("Theta:\n" + model.theta.map(_.mkString("[", ", ", "]")).mkString("[", "\n ", "]")) + + println("Pi\n: " + model.pi) + println("Theta:\n" + model.theta) sc.stop() } diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 19b90dfd6e167..d2f9cdb3f4298 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -87,18 +87,19 @@ class NaiveBayesModel(object): >>> data = array([0.0, 0.0, 1.0, 0.0, 0.0, 2.0, 1.0, 1.0, 0.0]).reshape(3,3) >>> model = NaiveBayes.train(sc.parallelize(data)) >>> model.predict(array([0.0, 1.0])) - 0 + 0.0 >>> model.predict(array([1.0, 0.0])) - 1 + 1.0 """ - def __init__(self, pi, theta): + def __init__(self, labels, pi, theta): + self.labels = labels self.pi = pi self.theta = theta def predict(self, x): """Return the most likely class for a data vector x""" - return numpy.argmax(self.pi + dot(x, self.theta)) + return self.labels[numpy.argmax(self.pi + dot(x, self.theta))] class NaiveBayes(object): @classmethod @@ -122,7 +123,8 @@ def train(cls, data, lambda_=1.0): ans = sc._jvm.PythonMLLibAPI().trainNaiveBayes(dataBytes._jrdd, lambda_) return NaiveBayesModel( _deserialize_double_vector(ans[0]), - _deserialize_double_matrix(ans[1])) + _deserialize_double_vector(ans[1]), + _deserialize_double_matrix(ans[2])) def _test(): From 6f59eed4aec6f0032ec24a9dd2c2f0d0e0e037ee Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 30 Mar 2014 18:22:57 -0700 Subject: [PATCH 16/26] update libSVMFile to determine number of features automatically --- .../org/apache/spark/mllib/MLContext.scala | 27 ++++++++++++++----- .../apache/spark/mllib/MLContextSuite.scala | 21 ++++++++++----- 2 files changed, 35 insertions(+), 13 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala b/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala index eefca193ec53e..fb4d458cd8a09 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala @@ -34,24 +34,37 @@ class MLContext(self: SparkContext) { * where the feature indices are converted to zero-based. * * @param path file or directory path in any Hadoop-supported file system URI - * @param numFeatures number of features - * @param labelParser parser for labels, default: _.toDouble + * @param numFeatures number of features, it will be determined from input + * if a non-positive value is given + *@param labelParser parser for labels, default: _.toDouble * @return labeled data stored as an RDD[LabeledPoint] */ def libSVMFile( path: String, numFeatures: Int, labelParser: String => Double = _.toDouble): RDD[LabeledPoint] = { - self.textFile(path).map(_.trim).filter(!_.isEmpty).map { line => - val items = line.split(' ') + val parsed = self.textFile(path).map(_.trim).filter(!_.isEmpty).map(_.split(' ')) + // Determine number of features. + val d = if (numFeatures > 0) { + numFeatures + } else { + parsed.map { items => + if (items.length > 1) { + items.last.split(':')(0).toInt + } else { + 0 + } + }.reduce(math.max) + } + parsed.map { items => val label = labelParser(items.head) - val features = Vectors.sparse(numFeatures, items.tail.map { item => + val (indices, values) = items.tail.map { item => val indexAndValue = item.split(':') val index = indexAndValue(0).toInt - 1 val value = indexAndValue(1).toDouble (index, value) - }) - LabeledPoint(label, features) + }.unzip + LabeledPoint(label, Vectors.sparse(d, indices.toArray, values.toArray)) } } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala index 6762f8c479e98..743102b54fa9e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala @@ -33,17 +33,26 @@ class MLContextSuite extends FunSuite with LocalSparkContext { val lines = """ |1 1:1.0 3:2.0 5:3.0 + |0 |0 2:4.0 4:5.0 6:6.0 """.stripMargin val tempDir = Files.createTempDir() val file = new File(tempDir.getPath, "part-00000") Files.write(lines, file, Charsets.US_ASCII) - val points = sc.libSVMFile(tempDir.toURI.toString, 6).collect() - assert(points.length === 2) - assert(points(0).label === 1.0) - assert(points(0).features === Vectors.sparse(6, Seq((0, 1.0), (2, 2.0), (4, 3.0)))) - assert(points(1).label === 0.0) - assert(points(1).features === Vectors.sparse(6, Seq((1, 4.0), (3, 5.0), (5, 6.0)))) + + val pointsWithNumFeatures = sc.libSVMFile(tempDir.toURI.toString, 6).collect() + val pointsWithoutNumFeatures = sc.libSVMFile(tempDir.toURI.toString, 0).collect() + + for (points <- Seq(pointsWithNumFeatures, pointsWithoutNumFeatures)) { + assert(points.length === 3) + assert(points(0).label === 1.0) + assert(points(0).features === Vectors.sparse(6, Seq((0, 1.0), (2, 2.0), (4, 3.0)))) + assert(points(1).label == 0.0) + assert(points(1).features == Vectors.sparse(6, Seq())) + assert(points(2).label === 0.0) + assert(points(2).features === Vectors.sparse(6, Seq((1, 4.0), (3, 5.0), (5, 6.0)))) + } + try { file.delete() tempDir.delete() From d0885526e16c3d5b798d92bf98873cfd67f15301 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 30 Mar 2014 18:39:09 -0700 Subject: [PATCH 17/26] use static constructor for MLContext --- .../org/apache/spark/mllib/MLContext.scala | 23 +++++++++++++------ .../apache/spark/mllib/MLContextSuite.scala | 7 +++--- 2 files changed, 20 insertions(+), 10 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala b/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala index fb4d458cd8a09..3cd09b0d48113 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala @@ -23,7 +23,12 @@ import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD -class MLContext(self: SparkContext) { +/** + * Provides methods related to machine learning on top of [[org.apache.spark.SparkContext]]. + * + * @param sparkContext a [[org.apache.spark.SparkContext]] instance + */ +class MLContext(val sparkContext: SparkContext) { /** * Reads labeled data in the LIBSVM format into an RDD[LabeledPoint]. * The LIBSVM format is a text-based format used by LIBSVM and LIBLINEAR. @@ -34,16 +39,16 @@ class MLContext(self: SparkContext) { * where the feature indices are converted to zero-based. * * @param path file or directory path in any Hadoop-supported file system URI - * @param numFeatures number of features, it will be determined from input - * if a non-positive value is given - *@param labelParser parser for labels, default: _.toDouble + * @param numFeatures number of features, which will be determined from the input data if a + * non-positive value is given. The default value is 0. + * @param labelParser parser for labels, default: _.toDouble * @return labeled data stored as an RDD[LabeledPoint] */ def libSVMFile( path: String, - numFeatures: Int, + numFeatures: Int = 0, labelParser: String => Double = _.toDouble): RDD[LabeledPoint] = { - val parsed = self.textFile(path).map(_.trim).filter(!_.isEmpty).map(_.split(' ')) + val parsed = sparkContext.textFile(path).map(_.trim).filter(!_.isEmpty).map(_.split(' ')) // Determine number of features. val d = if (numFeatures > 0) { numFeatures @@ -70,5 +75,9 @@ class MLContext(self: SparkContext) { } object MLContext { - implicit def sparkContextToMLContext(sc: SparkContext): MLContext = new MLContext(sc) + /** + * Creates an [[org.apache.spark.mllib.MLContext]] instance from + * an [[org.apache.spark.SparkContext]] instance. + */ + def apply(sc: SparkContext): MLContext = new MLContext(sc) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala index 743102b54fa9e..05be434590c48 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala @@ -24,7 +24,6 @@ import org.scalatest.FunSuite import com.google.common.base.Charsets import com.google.common.io.Files -import org.apache.spark.mllib.MLContext._ import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.LocalSparkContext @@ -40,8 +39,10 @@ class MLContextSuite extends FunSuite with LocalSparkContext { val file = new File(tempDir.getPath, "part-00000") Files.write(lines, file, Charsets.US_ASCII) - val pointsWithNumFeatures = sc.libSVMFile(tempDir.toURI.toString, 6).collect() - val pointsWithoutNumFeatures = sc.libSVMFile(tempDir.toURI.toString, 0).collect() + val mlc = MLContext(sc) + + val pointsWithNumFeatures = mlc.libSVMFile(tempDir.toURI.toString, 6).collect() + val pointsWithoutNumFeatures = mlc.libSVMFile(tempDir.toURI.toString, 0).collect() for (points <- Seq(pointsWithNumFeatures, pointsWithoutNumFeatures)) { assert(points.length === 3) From f04fe8a4430f624d9ac48b1bd645e911ce467a99 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 30 Mar 2014 21:15:53 -0700 Subject: [PATCH 18/26] remove normalization from RidgeRegression and update tests --- .../mllib/regression/RidgeRegression.scala | 57 +++---------------- .../regression/JavaRidgeRegressionSuite.java | 38 ++++++------- .../regression/RidgeRegressionSuite.scala | 24 ++++---- 3 files changed, 36 insertions(+), 83 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index 0f10e48af02e4..1f17d2107f940 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -17,13 +17,11 @@ package org.apache.spark.mllib.regression -import breeze.linalg.{Vector => BV} - import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.mllib.linalg.{Vectors, Vector} +import org.apache.spark.mllib.linalg.Vector /** * Regression model trained using RidgeRegression. @@ -58,8 +56,7 @@ class RidgeRegressionWithSGD private ( var numIterations: Int, var regParam: Double, var miniBatchFraction: Double) - extends GeneralizedLinearAlgorithm[RidgeRegressionModel] - with Serializable { + extends GeneralizedLinearAlgorithm[RidgeRegressionModel] with Serializable { val gradient = new LeastSquaresGradient() val updater = new SquaredL2Updater() @@ -72,10 +69,6 @@ class RidgeRegressionWithSGD private ( // We don't want to penalize the intercept in RidgeRegression, so set this to false. super.setIntercept(false) - private var yMean = 0.0 - private var xColMean: BV[Double] = _ - private var xColSd: BV[Double] = _ - /** * Construct a RidgeRegression object with default parameters */ @@ -88,35 +81,7 @@ class RidgeRegressionWithSGD private ( } override protected def createModel(weights: Vector, intercept: Double) = { - val weightsMat = weights.toBreeze - val weightsScaled = weightsMat :/ xColSd - val interceptScaled = yMean - weightsMat.dot(xColMean :/ xColSd) - - new RidgeRegressionModel(Vectors.fromBreeze(weightsScaled), interceptScaled) - } - - override def run( - input: RDD[LabeledPoint], - initialWeights: Vector) - : RidgeRegressionModel = - { - val nfeatures: Int = input.first().features.size - val nexamples: Long = input.count() - - // To avoid penalizing the intercept, we center and scale the data. - val stats = MLUtils.computeStats(input, nfeatures, nexamples) - yMean = stats._1 - xColMean = stats._2.toBreeze - xColSd = stats._3.toBreeze - - val normalizedData = input.map { point => - val yNormalized = point.label - yMean - val featuresMat = point.features.toBreeze - val featuresNormalized = (featuresMat - xColMean) :/ xColSd - LabeledPoint(yNormalized, Vectors.fromBreeze(featuresNormalized)) - } - - super.run(normalizedData, initialWeights) + new RidgeRegressionModel(weights, intercept) } } @@ -145,9 +110,7 @@ object RidgeRegressionWithSGD { stepSize: Double, regParam: Double, miniBatchFraction: Double, - initialWeights: Vector) - : RidgeRegressionModel = - { + initialWeights: Vector): RidgeRegressionModel = { new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run( input, initialWeights) } @@ -168,9 +131,7 @@ object RidgeRegressionWithSGD { numIterations: Int, stepSize: Double, regParam: Double, - miniBatchFraction: Double) - : RidgeRegressionModel = - { + miniBatchFraction: Double): RidgeRegressionModel = { new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input) } @@ -189,9 +150,7 @@ object RidgeRegressionWithSGD { input: RDD[LabeledPoint], numIterations: Int, stepSize: Double, - regParam: Double) - : RidgeRegressionModel = - { + regParam: Double): RidgeRegressionModel = { train(input, numIterations, stepSize, regParam, 1.0) } @@ -206,9 +165,7 @@ object RidgeRegressionWithSGD { */ def train( input: RDD[LabeledPoint], - numIterations: Int) - : RidgeRegressionModel = - { + numIterations: Int): RidgeRegressionModel = { train(input, numIterations, 1.0, 1.0, 1.0) } diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java index 2fdd5fc8fdca6..03714ae7e4d00 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java @@ -55,30 +55,27 @@ public void tearDown() { return errorSum / validationData.size(); } - List generateRidgeData(int numPoints, int nfeatures, double eps) { + List generateRidgeData(int numPoints, int numFeatures, double std) { org.jblas.util.Random.seed(42); // Pick weights as random values distributed uniformly in [-0.5, 0.5] - DoubleMatrix w = DoubleMatrix.rand(nfeatures, 1).subi(0.5); - // Set first two weights to eps - w.put(0, 0, eps); - w.put(1, 0, eps); - return LinearDataGenerator.generateLinearInputAsList(0.0, w.data, numPoints, 42, eps); + DoubleMatrix w = DoubleMatrix.rand(numFeatures, 1).subi(0.5); + return LinearDataGenerator.generateLinearInputAsList(0.0, w.data, numPoints, 42, std); } @Test public void runRidgeRegressionUsingConstructor() { - int nexamples = 200; - int nfeatures = 20; - double eps = 10.0; - List data = generateRidgeData(2*nexamples, nfeatures, eps); + int numExamples = 50; + int numFeatures = 20; + List data = generateRidgeData(2*numExamples, numFeatures, 10.0); - JavaRDD testRDD = sc.parallelize(data.subList(0, nexamples)); - List validationData = data.subList(nexamples, 2*nexamples); + JavaRDD testRDD = sc.parallelize(data.subList(0, numExamples)); + List validationData = data.subList(numExamples, 2 * numExamples); RidgeRegressionWithSGD ridgeSGDImpl = new RidgeRegressionWithSGD(); - ridgeSGDImpl.optimizer().setStepSize(1.0) - .setRegParam(0.0) - .setNumIterations(200); + ridgeSGDImpl.optimizer() + .setStepSize(1.0) + .setRegParam(0.0) + .setNumIterations(200); RidgeRegressionModel model = ridgeSGDImpl.run(testRDD.rdd()); double unRegularizedErr = predictionError(validationData, model); @@ -91,13 +88,12 @@ public void runRidgeRegressionUsingConstructor() { @Test public void runRidgeRegressionUsingStaticMethods() { - int nexamples = 200; - int nfeatures = 20; - double eps = 10.0; - List data = generateRidgeData(2*nexamples, nfeatures, eps); + int numExamples = 50; + int numFeatures = 20; + List data = generateRidgeData(2 * numExamples, numFeatures, 10.0); - JavaRDD testRDD = sc.parallelize(data.subList(0, nexamples)); - List validationData = data.subList(nexamples, 2*nexamples); + JavaRDD testRDD = sc.parallelize(data.subList(0, numExamples)); + List validationData = data.subList(numExamples, 2 * numExamples); RidgeRegressionModel model = RidgeRegressionWithSGD.train(testRDD.rdd(), 200, 1.0, 0.0); double unRegularizedErr = predictionError(validationData, model); 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 a1f72bc8536c0..67768e17fbe6d 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 @@ -31,22 +31,22 @@ class RidgeRegressionSuite extends FunSuite with LocalSparkContext { }.reduceLeft(_ + _) / predictions.size } - test("regularization with skewed weights") { - val nexamples = 200 - val nfeatures = 20 - val eps = 10 + test("ridge regression can help avoid overfitting") { + + // For small number of examples and large variance of error distribution, + // ridge regression should give smaller generalization error that linear regression. + + val numExamples = 50 + val numFeatures = 20 org.jblas.util.Random.seed(42) // Pick weights as random values distributed uniformly in [-0.5, 0.5] - val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) - // Set first two weights to eps - w.put(0, 0, eps) - w.put(1, 0, eps) + val w = DoubleMatrix.rand(numFeatures, 1).subi(0.5) // Use half of data for training and other half for validation - val data = LinearDataGenerator.generateLinearInput(3.0, w.toArray, 2*nexamples, 42, eps) - val testData = data.take(nexamples) - val validationData = data.takeRight(nexamples) + val data = LinearDataGenerator.generateLinearInput(3.0, w.toArray, 2 * numExamples, 42, 10.0) + val testData = data.take(numExamples) + val validationData = data.takeRight(numExamples) val testRDD = sc.parallelize(testData, 2).cache() val validationRDD = sc.parallelize(validationData, 2).cache() @@ -68,7 +68,7 @@ class RidgeRegressionSuite extends FunSuite with LocalSparkContext { val ridgeErr = predictionError( ridgeModel.predict(validationRDD.map(_.features)).collect(), validationData) - // Ridge CV-error should be lower than linear regression + // Ridge validation error should be lower than linear regression. assert(ridgeErr < linearErr, "ridgeError (" + ridgeErr + ") was not less than linearError(" + linearErr + ")") } From 4ca5b1bcb8b2e004fdb3e4a87c5e23fc35f515c3 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 30 Mar 2014 21:37:01 -0700 Subject: [PATCH 19/26] remove normalization from Lasso and update tests --- .../apache/spark/mllib/regression/Lasso.scala | 39 ++----------------- .../mllib/regression/JavaLassoSuite.java | 4 +- .../spark/mllib/regression/LassoSuite.scala | 38 +++++++++++------- 3 files changed, 31 insertions(+), 50 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index 911e0c0178cab..25920d0dc976e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -17,13 +17,11 @@ package org.apache.spark.mllib.regression -import breeze.linalg.{Vector => BV} - import org.apache.spark.SparkContext -import org.apache.spark.rdd.RDD -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD /** * Regression model trained using Lasso. @@ -58,8 +56,7 @@ class LassoWithSGD private ( var numIterations: Int, var regParam: Double, var miniBatchFraction: Double) - extends GeneralizedLinearAlgorithm[LassoModel] - with Serializable { + extends GeneralizedLinearAlgorithm[LassoModel] with Serializable { val gradient = new LeastSquaresGradient() val updater = new L1Updater() @@ -71,10 +68,6 @@ class LassoWithSGD private ( // We don't want to penalize the intercept, so set this to false. super.setIntercept(false) - private var yMean = 0.0 - private var xColMean: BV[Double] = _ - private var xColSd: BV[Double] = _ - /** * Construct a Lasso object with default parameters */ @@ -87,31 +80,7 @@ class LassoWithSGD private ( } override protected def createModel(weights: Vector, intercept: Double) = { - val weightsMat = weights.toBreeze - val weightsScaled = weightsMat :/ xColSd - val interceptScaled = yMean - weightsMat.dot(xColMean :/ xColSd) - - new LassoModel(Vectors.fromBreeze(weightsScaled), interceptScaled) - } - - override def run(input: RDD[LabeledPoint], initialWeights: Vector): LassoModel = { - val nfeatures: Int = input.first.features.size - val nexamples: Long = input.count() - - // To avoid penalizing the intercept, we center and scale the data. - val stats = MLUtils.computeStats(input, nfeatures, nexamples) - yMean = stats._1 - xColMean = stats._2.toBreeze - xColSd = stats._3.toBreeze - - val normalizedData = input.map { point => - val yNormalized = point.label - yMean - val featuresMat = point.features.toBreeze - val featuresNormalized = (featuresMat - xColMean) :/ xColSd - LabeledPoint(yNormalized, Vectors.fromBreeze(featuresNormalized)) - } - - super.run(normalizedData, initialWeights) + new LassoModel(weights, intercept) } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java index f44b25cd44d19..f725924a2d971 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java @@ -59,7 +59,7 @@ int validatePrediction(List validationData, LassoModel model) { @Test public void runLassoUsingConstructor() { int nPoints = 10000; - double A = 2.0; + double A = 0.0; double[] weights = {-1.5, 1.0e-2}; JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, @@ -80,7 +80,7 @@ public void runLassoUsingConstructor() { @Test public void runLassoUsingStaticMethods() { int nPoints = 10000; - double A = 2.0; + double A = 0.0; double[] weights = {-1.5, 1.0e-2}; JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, 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 02e99fac6146f..bfa42959c8ead 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 @@ -40,10 +40,11 @@ class LassoSuite extends FunSuite with LocalSparkContext { val B = -1.5 val C = 1.0e-2 - val testData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 42) - - val testRDD = sc.parallelize(testData, 2) - testRDD.cache() + val testData = LinearDataGenerator.generateLinearInput(A, Array[Double](B, C), nPoints, 42) + .map { case LabeledPoint(label, features) => + LabeledPoint(label, Vectors.dense(1.0 +: features.toArray)) + } + val testRDD = sc.parallelize(testData, 2).cache() val ls = new LassoWithSGD() ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(40) @@ -51,11 +52,15 @@ class LassoSuite extends FunSuite with LocalSparkContext { val model = ls.run(testRDD) val weight0 = model.weights(0) val weight1 = model.weights(1) - assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") - assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") - assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]") + val weight2 = model.weights(2) + assert(weight0 >= 1.9 && weight0 <= 2.1, weight0 + " not in [1.9, 2.1]") + assert(weight1 >= -1.60 && weight1 <= -1.40, weight1 + " not in [-1.6, -1.4]") + assert(weight2 >= -1.0e-3 && weight2 <= 1.0e-3, weight2 + " not in [-0.001, 0.001]") val validationData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 17) + .map { case LabeledPoint(label, features) => + LabeledPoint(label, Vectors.dense(1.0 +: features.toArray)) + } val validationRDD = sc.parallelize(validationData, 2) // Test prediction on RDD. @@ -73,13 +78,16 @@ class LassoSuite extends FunSuite with LocalSparkContext { val C = 1.0e-2 val testData = LinearDataGenerator.generateLinearInput(A, Array[Double](B, C), nPoints, 42) + .map { case LabeledPoint(label, features) => + LabeledPoint(label, Vectors.dense(1.0 +: features.toArray)) + } + val initialA = -1.0 val initialB = -1.0 val initialC = -1.0 - val initialWeights = Vectors.dense(initialB, initialC) + val initialWeights = Vectors.dense(initialA, initialB, initialC) - val testRDD = sc.parallelize(testData, 2) - testRDD.cache() + val testRDD = sc.parallelize(testData, 2).cache() val ls = new LassoWithSGD() ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(40) @@ -87,11 +95,15 @@ class LassoSuite extends FunSuite with LocalSparkContext { val model = ls.run(testRDD, initialWeights) val weight0 = model.weights(0) val weight1 = model.weights(1) - assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") - assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") - assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]") + val weight2 = model.weights(2) + assert(weight0 >= 1.9 && weight0 <= 2.1, weight0 + " not in [1.9, 2.1]") + assert(weight1 >= -1.60 && weight1 <= -1.40, weight1 + " not in [-1.6, -1.4]") + assert(weight2 >= -1.0e-3 && weight2 <= 1.0e-3, weight2 + " not in [-0.001, 0.001]") val validationData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 17) + .map { case LabeledPoint(label, features) => + LabeledPoint(label, Vectors.dense(1.0 +: features.toArray)) + } val validationRDD = sc.parallelize(validationData,2) // Test prediction on RDD. From b01df546f528e443e799e18a7b5b465dde18d831 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 31 Mar 2014 00:53:31 -0700 Subject: [PATCH 20/26] allow to change or clear threshold in LR and SVM add more comments to MLUtils.fastSquaredDistance --- .../mllib/api/python/PythonMLLibAPI.scala | 36 +++++++++----- .../classification/ClassificationModel.scala | 10 ++-- .../classification/LogisticRegression.scala | 49 +++++++++++++------ .../spark/mllib/classification/SVM.scala | 35 ++++++++++--- .../spark/mllib/clustering/KMeans.scala | 3 +- .../org/apache/spark/mllib/util/MLUtils.scala | 12 +++++ .../mllib/classification/JavaSVMSuite.java | 3 -- .../spark/mllib/classification/SVMSuite.scala | 4 +- 8 files changed, 108 insertions(+), 44 deletions(-) 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 757aea32ef94e..2df5b0d02b699 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 @@ -110,8 +110,8 @@ class PythonMLLibAPI extends Serializable { private def trainRegressionModel( trainFunc: (RDD[LabeledPoint], Array[Double]) => GeneralizedLinearModel, - dataBytesJRDD: JavaRDD[Array[Byte]], initialWeightsBA: Array[Byte]): - java.util.LinkedList[java.lang.Object] = { + dataBytesJRDD: JavaRDD[Array[Byte]], + initialWeightsBA: Array[Byte]): java.util.LinkedList[java.lang.Object] = { val data = dataBytesJRDD.rdd.map(xBytes => { val x = deserializeDoubleVector(xBytes) LabeledPoint(x(0), Vectors.dense(x.slice(1, x.length))) @@ -238,9 +238,9 @@ class PythonMLLibAPI extends Serializable { /** * Java stub for NaiveBayes.train() */ - def trainNaiveBayes(dataBytesJRDD: JavaRDD[Array[Byte]], lambda: Double) - : java.util.List[java.lang.Object] = - { + def trainNaiveBayes( + dataBytesJRDD: JavaRDD[Array[Byte]], + lambda: Double): java.util.List[java.lang.Object] = { val data = dataBytesJRDD.rdd.map(xBytes => { val x = deserializeDoubleVector(xBytes) LabeledPoint(x(0), Vectors.dense(x.slice(1, x.length))) @@ -256,9 +256,12 @@ class PythonMLLibAPI extends Serializable { /** * Java stub for Python mllib KMeans.train() */ - def trainKMeansModel(dataBytesJRDD: JavaRDD[Array[Byte]], k: Int, - maxIterations: Int, runs: Int, initializationMode: String): - java.util.List[java.lang.Object] = { + 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 => Vectors.dense(deserializeDoubleVector(xBytes))) val model = KMeans.train(data, k, maxIterations, runs, initializationMode) val ret = new java.util.LinkedList[java.lang.Object]() @@ -311,8 +314,12 @@ class PythonMLLibAPI extends Serializable { * needs to be taken in the Python code to ensure it gets freed on exit; see * the Py4J documentation. */ - def trainALSModel(ratingsBytesJRDD: JavaRDD[Array[Byte]], rank: Int, - iterations: Int, lambda: Double, blocks: Int): MatrixFactorizationModel = { + def trainALSModel( + ratingsBytesJRDD: JavaRDD[Array[Byte]], + rank: Int, + iterations: Int, + lambda: Double, + blocks: Int): MatrixFactorizationModel = { val ratings = ratingsBytesJRDD.rdd.map(unpackRating) ALS.train(ratings, rank, iterations, lambda, blocks) } @@ -323,8 +330,13 @@ class PythonMLLibAPI extends Serializable { * Extra care needs to be taken in the Python code to ensure it gets freed on * exit; see the Py4J documentation. */ - def trainImplicitALSModel(ratingsBytesJRDD: JavaRDD[Array[Byte]], rank: Int, - iterations: Int, lambda: Double, blocks: Int, alpha: Double): MatrixFactorizationModel = { + def trainImplicitALSModel( + ratingsBytesJRDD: JavaRDD[Array[Byte]], + rank: Int, + iterations: Int, + lambda: Double, + blocks: Int, + alpha: Double): MatrixFactorizationModel = { val ratings = ratingsBytesJRDD.rdd.map(unpackRating) ALS.trainImplicit(ratings, rank, iterations, lambda, blocks, alpha) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala index 2591d89b9e0dc..bd10e2e9e10e2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala @@ -17,15 +17,19 @@ package org.apache.spark.mllib.classification -import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.rdd.RDD +/** + * Represents a classification model that predicts to which of a set of categories an example + * belongs. The categories are represented by double values: 0.0, 1.0, 2.0, etc. + */ trait ClassificationModel extends Serializable { /** * Predict values for the given data set using the model trained. * * @param testData RDD representing data points to be predicted - * @return RDD[Int] where each entry contains the corresponding prediction + * @return an RDD[Double] where each entry contains the corresponding prediction */ def predict(testData: RDD[Vector]): RDD[Double] @@ -33,7 +37,7 @@ trait ClassificationModel extends Serializable { * Predict values for a single data point using the model trained. * * @param testData array representing a single data point - * @return Int prediction from the trained model + * @return predicted category from the trained model */ def predict(testData: Vector): Double } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index da9995f5879ad..798f3a5c94740 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -17,15 +17,12 @@ package org.apache.spark.mllib.classification -import scala.math.round - import org.apache.spark.SparkContext -import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.mllib.util.DataValidators -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.util.{DataValidators, MLUtils} +import org.apache.spark.rdd.RDD /** * Classification model trained using Logistic Regression. @@ -36,13 +33,36 @@ import org.apache.spark.mllib.linalg.Vector class LogisticRegressionModel( override val weights: Vector, override val intercept: Double) - extends GeneralizedLinearModel(weights, intercept) - with ClassificationModel with Serializable { + extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable { + + private var threshold: Option[Double] = Some(0.5) + + /** + * Sets the threshold that separates positive predictions from negative predictions. An example + * with prediction score greater than or equal to this threshold is identified as an positive, + * and negative otherwise. The default value is 0.5. + */ + def setThreshold(threshold: Double): this.type = { + this.threshold = Some(threshold) + this + } + + /** + * Clears the threshold so that `predict` will output raw prediction scores. + */ + def clearThreshold(): this.type = { + threshold = None + this + } override def predictPoint(dataMatrix: Vector, weightMatrix: Vector, intercept: Double) = { val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept - round(1.0/ (1.0 + math.exp(margin * -1))) + val score = 1.0/ (1.0 + math.exp(-margin)) + threshold match { + case Some(t) => if (score < t) 0.0 else 1.0 + case None => score + } } } @@ -55,16 +75,15 @@ class LogisticRegressionWithSGD private ( var numIterations: Int, var regParam: Double, var miniBatchFraction: Double) - extends GeneralizedLinearAlgorithm[LogisticRegressionModel] - with Serializable { + extends GeneralizedLinearAlgorithm[LogisticRegressionModel] with Serializable { val gradient = new LogisticGradient() val updater = new SimpleUpdater() override val optimizer = new GradientDescent(gradient, updater) - .setStepSize(stepSize) - .setNumIterations(numIterations) - .setRegParam(regParam) - .setMiniBatchFraction(miniBatchFraction) + .setStepSize(stepSize) + .setNumIterations(numIterations) + .setRegParam(regParam) + .setMiniBatchFraction(miniBatchFraction) override val validators = List(DataValidators.classificationLabels) /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index b854bcab815f0..e31a08899f8bc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -18,12 +18,11 @@ package org.apache.spark.mllib.classification import org.apache.spark.SparkContext -import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.mllib.util.DataValidators -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.util.{DataValidators, MLUtils} +import org.apache.spark.rdd.RDD /** * Model for Support Vector Machines (SVMs). @@ -34,13 +33,35 @@ import org.apache.spark.mllib.linalg.{Vector, Vectors} class SVMModel( override val weights: Vector, override val intercept: Double) - extends GeneralizedLinearModel(weights, intercept) - with ClassificationModel with Serializable { + extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable { + + private var threshold: Option[Double] = Some(0.0) + + /** + * Sets the threshold that separates positive predictions from negative predictions. An example + * with prediction score greater than or equal to this threshold is identified as an positive, + * and negative otherwise. The default value is 0.0. + */ + def setThreshold(threshold: Double): this.type = { + this.threshold = Some(threshold) + this + } + + /** + * Clears the threshold so that `predict` will output raw prediction scores. + */ + def clearThreshold(): this.type = { + threshold = None + this + } override def predictPoint(dataMatrix: Vector, weightMatrix: Vector, intercept: Double) = { val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept - if (margin < 0) 0.0 else 1.0 + threshold match { + case Some(t) => if (margin < 0) 0.0 else 1.0 + case None => margin + } } } 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 b412738e3f00a..a78503df3134d 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 @@ -42,8 +42,7 @@ class KMeans private ( var runs: Int, var initializationMode: String, var initializationSteps: Int, - var epsilon: Double) - extends Serializable with Logging { + var epsilon: Double) 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. */ 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 5d21ed8e6983c..4c47eb5c7506c 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 @@ -150,6 +150,18 @@ object MLUtils { val sumSquaredNorm = norm1 * norm1 + norm2 * norm2 val normDiff = norm1 - norm2 var sqDist = 0.0 + /* + * The relative error is + *
+     * EPSILON * ( \|a\|_2^2 + \|b\\_2^2 + 2 |a^T b|) / ( \|a - b\|_2^2 ),
+     * 
+ * which is bounded by + *
+     * 2.0 * EPSILON * ( \|a\|_2^2 + \|b\|_2^2 ) / ( (\|a\|_2 - \|b\|_2)^2 ).
+     * 
+ * The bound doesn't need the inner product, so we can use it as a sufficient condition to + * check quickly whether the inner product approach is accurate. + */ val precisionBound1 = 2.0 * EPSILON * sumSquaredNorm / (normDiff * normDiff + EPSILON) if (precisionBound1 < precision) { sqDist = sumSquaredNorm - 2.0 * v1.dot(v2) diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java index 117e5eaa8b78e..4701a5e545020 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.classification; - import java.io.Serializable; import java.util.List; @@ -28,7 +27,6 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; - import org.apache.spark.mllib.regression.LabeledPoint; public class JavaSVMSuite implements Serializable { @@ -94,5 +92,4 @@ public void runSVMUsingStaticMethods() { int numAccurate = validatePrediction(validationData, model); Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); } - } 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 dc35d2483296d..dfacbfeee6fb4 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 @@ -150,10 +150,10 @@ class SVMSuite extends FunSuite with LocalSparkContext { } intercept[SparkException] { - val model = SVMWithSGD.train(testRDDInvalid, 100) + SVMWithSGD.train(testRDDInvalid, 100) } // Turning off data validation should not throw an exception - val noValidationModel = new SVMWithSGD().setValidateData(false).run(testRDDInvalid) + new SVMWithSGD().setValidateData(false).run(testRDDInvalid) } } From b9b7ef7244bdf7fe68a6abd5b739c2f943ee5e92 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 31 Mar 2014 12:16:28 -0700 Subject: [PATCH 21/26] change default value of addIntercept to false --- .../spark/mllib/regression/GeneralizedLinearAlgorithm.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 7f6e0cec0177a..a063ae9d64bec 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 @@ -71,7 +71,7 @@ abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double } /** - * GeneralizedLinearAlgorithm implements methods to train a Genearalized Linear Model (GLM). + * GeneralizedLinearAlgorithm implements methods to train a Generalized Linear Model (GLM). * This class should be extended with an Optimizer to create a new GLM. */ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] @@ -81,7 +81,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] val optimizer: Optimizer - protected var addIntercept: Boolean = true + /** Whether to add intercept (default: false). */ + protected var addIntercept: Boolean = false protected var validateData: Boolean = true From 7c1bc0193402a3031030e90a04410d86331d49f0 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 31 Mar 2014 13:20:44 -0700 Subject: [PATCH 22/26] add a TODO to NB --- .../spark/mllib/classification/NaiveBayes.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 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 fb6567ac2e431..e956185319a69 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 @@ -82,6 +82,8 @@ class NaiveBayes private (var lambda: Double) extends Serializable with Logging */ def run(data: RDD[LabeledPoint]) = { // Aggregates term frequencies per label. + // TODO: Calling combineByKey and collect creates two stages, we can implement something + // TODO: similar to reduceByKeyLocally to save one stage. val aggregated = data.map(p => (p.label, p.features)).combineByKey[(Long, BDV[Double])]( createCombiner = (v: Vector) => (1L, v.toBreeze.toDenseVector), mergeValue = (c: (Long, BDV[Double]), v: Vector) => (c._1 + 1L, c._2 += v.toBreeze), @@ -89,23 +91,23 @@ class NaiveBayes private (var lambda: Double) extends Serializable with Logging (c1._1 + c2._1, c1._2 += c2._2) ).collect() val numLabels = aggregated.length - var numExamples = 0L + var numDocuments = 0L aggregated.foreach { case (_, (n, _)) => - numExamples += n + numDocuments += n } val numFeatures = aggregated.head match { case (_, (_, v)) => v.size } val labels = new Array[Double](numLabels) val pi = new Array[Double](numLabels) val theta = Array.fill(numLabels)(new Array[Double](numFeatures)) - val piLogDenom = math.log(numExamples + numLabels * lambda) + val piLogDenom = math.log(numDocuments + numLabels * lambda) var i = 0 - aggregated.foreach { case (label, (n, sum)) => + aggregated.foreach { case (label, (n, sumTermFreqs)) => labels(i) = label - val thetaLogDenom = math.log(brzSum(sum) + numFeatures * lambda) + val thetaLogDenom = math.log(brzSum(sumTermFreqs) + numFeatures * lambda) pi(i) = math.log(n + lambda) - piLogDenom var j = 0 while (j < numFeatures) { - theta(i)(j) = math.log(sum(j) + lambda) - thetaLogDenom + theta(i)(j) = math.log(sumTermFreqs(j) + lambda) - thetaLogDenom j += 1 } i += 1 From da25e249a2dd292349b1ae5baf13584f78f0d7b6 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 31 Mar 2014 14:21:28 -0700 Subject: [PATCH 23/26] revert the change to default addIntercept because it might change the behavior of existing code without warning --- .../spark/mllib/regression/GeneralizedLinearAlgorithm.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 a063ae9d64bec..80dc0f12ff84f 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 @@ -81,8 +81,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] val optimizer: Optimizer - /** Whether to add intercept (default: false). */ - protected var addIntercept: Boolean = false + /** Whether to add intercept (default: true). */ + protected var addIntercept: Boolean = true protected var validateData: Boolean = true From f7da54bb97c945fbd8b2a40417032915c9787a44 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 31 Mar 2014 17:43:02 -0700 Subject: [PATCH 24/26] add minSplits to libSVMFile --- .../src/main/scala/org/apache/spark/mllib/MLContext.scala | 7 ++++++- .../test/scala/org/apache/spark/mllib/MLContextSuite.scala | 4 ++-- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala b/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala index 3cd09b0d48113..00988bc480dc8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala @@ -39,6 +39,7 @@ class MLContext(val sparkContext: SparkContext) { * where the feature indices are converted to zero-based. * * @param path file or directory path in any Hadoop-supported file system URI + * @param minSplits min number of partitions, default: sparkContext.defaultMinSplits * @param numFeatures number of features, which will be determined from the input data if a * non-positive value is given. The default value is 0. * @param labelParser parser for labels, default: _.toDouble @@ -46,9 +47,13 @@ class MLContext(val sparkContext: SparkContext) { */ def libSVMFile( path: String, + minSplits: Int = sparkContext.defaultMinSplits, numFeatures: Int = 0, labelParser: String => Double = _.toDouble): RDD[LabeledPoint] = { - val parsed = sparkContext.textFile(path).map(_.trim).filter(!_.isEmpty).map(_.split(' ')) + val parsed = sparkContext.textFile(path, minSplits) + .map(_.trim) + .filter(!_.isEmpty) + .map(_.split(' ')) // Determine number of features. val d = if (numFeatures > 0) { numFeatures diff --git a/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala index 05be434590c48..6313978d546b9 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala @@ -41,8 +41,8 @@ class MLContextSuite extends FunSuite with LocalSparkContext { val mlc = MLContext(sc) - val pointsWithNumFeatures = mlc.libSVMFile(tempDir.toURI.toString, 6).collect() - val pointsWithoutNumFeatures = mlc.libSVMFile(tempDir.toURI.toString, 0).collect() + val pointsWithNumFeatures = mlc.libSVMFile(tempDir.toURI.toString, numFeatures = 6).collect() + val pointsWithoutNumFeatures = mlc.libSVMFile(tempDir.toURI.toString).collect() for (points <- Seq(pointsWithNumFeatures, pointsWithoutNumFeatures)) { assert(points.length === 3) From c26c4fcc300cf48f5fdbae125ec1c7a0e2959202 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 2 Apr 2014 00:19:38 -0700 Subject: [PATCH 25/26] update DecisionTree to use RDD[Vector] --- .../scala/org/apache/spark/mllib/linalg/Vectors.scala | 2 ++ .../scala/org/apache/spark/mllib/tree/DecisionTree.scala | 9 +++++---- .../spark/mllib/tree/model/DecisionTreeModel.scala | 5 +++-- .../scala/org/apache/spark/mllib/tree/model/Node.scala | 7 ++++--- .../org/apache/spark/mllib/tree/DecisionTreeSuite.scala | 9 +++++---- 5 files changed, 19 insertions(+), 13 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 21989d9a025a0..2cea58cd3fd22 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 @@ -151,6 +151,8 @@ class DenseVector(val values: Array[Double]) extends Vector { override def toArray: Array[Double] = values private[mllib] override def toBreeze: BV[Double] = new BDV[Double](values) + + override def apply(i: Int) = values(i) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 33205b919db8f..dee9594a9dd79 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -30,6 +30,7 @@ import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Impurity, Variance} import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom +import org.apache.spark.mllib.linalg.{Vector, Vectors} /** * A class that implements a decision tree algorithm for classification and regression. It @@ -295,7 +296,7 @@ object DecisionTree extends Serializable with Logging { val numNodes = scala.math.pow(2, level).toInt logDebug("numNodes = " + numNodes) // Find the number of features by looking at the first sample. - val numFeatures = input.first().features.length + val numFeatures = input.first().features.size logDebug("numFeatures = " + numFeatures) val numBins = bins(0).length logDebug("numBins = " + numBins) @@ -902,7 +903,7 @@ object DecisionTree extends Serializable with Logging { val count = input.count() // Find the number of features by looking at the first sample - val numFeatures = input.take(1)(0).features.length + val numFeatures = input.take(1)(0).features.size val maxBins = strategy.maxBins val numBins = if (maxBins <= count) maxBins else count.toInt @@ -1116,7 +1117,7 @@ object DecisionTree extends Serializable with Logging { sc.textFile(dir).map { line => val parts = line.trim().split(",") val label = parts(0).toDouble - val features = parts.slice(1,parts.length).map(_.toDouble) + val features = Vectors.dense(parts.slice(1,parts.length).map(_.toDouble)) LabeledPoint(label, features) } } @@ -1127,7 +1128,7 @@ object DecisionTree extends Serializable with Logging { */ private def accuracyScore(model: DecisionTreeModel, data: RDD[LabeledPoint], threshold: Double = 0.5): Double = { - def predictedValue(features: Array[Double]) = { + def predictedValue(features: Vector) = { if (model.predict(features) < threshold) 0.0 else 1.0 } val correctCount = data.filter(y => predictedValue(y.features) == y.label).count() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index a8bbf21daec01..a6dca84a2ce09 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.tree.model import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.Vector /** * Model to store the decision tree parameters @@ -33,7 +34,7 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable * @param features array representing a single data point * @return Double prediction from the trained model */ - def predict(features: Array[Double]): Double = { + def predict(features: Vector): Double = { topNode.predictIfLeaf(features) } @@ -43,7 +44,7 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable * @param features RDD representing data points to be predicted * @return RDD[Int] where each entry contains the corresponding prediction */ - def predict(features: RDD[Array[Double]]): RDD[Double] = { + def predict(features: RDD[Vector]): RDD[Double] = { features.map(x => predict(x)) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index ea4693c5c2f4e..aac3f9ce308f7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.tree.model import org.apache.spark.Logging import org.apache.spark.mllib.tree.configuration.FeatureType._ +import org.apache.spark.mllib.linalg.Vector /** * Node in a decision tree @@ -54,8 +55,8 @@ class Node ( logDebug("stats = " + stats) logDebug("predict = " + predict) if (!isLeaf) { - val leftNodeIndex = id*2 + 1 - val rightNodeIndex = id*2 + 2 + val leftNodeIndex = id * 2 + 1 + val rightNodeIndex = id * 2 + 2 leftNode = Some(nodes(leftNodeIndex)) rightNode = Some(nodes(rightNodeIndex)) leftNode.get.build(nodes) @@ -68,7 +69,7 @@ class Node ( * @param feature feature value * @return predicted value */ - def predictIfLeaf(feature: Array[Double]) : Double = { + def predictIfLeaf(feature: Vector) : Double = { if (isLeaf) { predict } else{ diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 4349c7000a0ae..350130c914f26 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.mllib.tree.model.Filter import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ +import org.apache.spark.mllib.linalg.Vectors class DecisionTreeSuite extends FunSuite with BeforeAndAfterAll { @@ -396,7 +397,7 @@ object DecisionTreeSuite { def generateOrderedLabeledPointsWithLabel0(): Array[LabeledPoint] = { val arr = new Array[LabeledPoint](1000) for (i <- 0 until 1000){ - val lp = new LabeledPoint(0.0,Array(i.toDouble,1000.0-i)) + val lp = new LabeledPoint(0.0, Vectors.dense(i.toDouble, 1000.0 - i)) arr(i) = lp } arr @@ -405,7 +406,7 @@ object DecisionTreeSuite { def generateOrderedLabeledPointsWithLabel1(): Array[LabeledPoint] = { val arr = new Array[LabeledPoint](1000) for (i <- 0 until 1000){ - val lp = new LabeledPoint(1.0,Array(i.toDouble,999.0-i)) + val lp = new LabeledPoint(1.0, Vectors.dense(i.toDouble, 999.0 - i)) arr(i) = lp } arr @@ -415,9 +416,9 @@ object DecisionTreeSuite { val arr = new Array[LabeledPoint](1000) for (i <- 0 until 1000){ if (i < 600){ - arr(i) = new LabeledPoint(1.0,Array(0.0,1.0)) + arr(i) = new LabeledPoint(1.0, Vectors.dense(0.0, 1.0)) } else { - arr(i) = new LabeledPoint(0.0,Array(1.0,0.0)) + arr(i) = new LabeledPoint(0.0, Vectors.dense(1.0, 0.0)) } } arr From eb6e79321548fc6e806105d731b12d87ac01e3de Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 2 Apr 2014 01:04:21 -0700 Subject: [PATCH 26/26] move libSVMFile to MLUtils and rename to loadLibSVMData --- .../org/apache/spark/mllib/MLContext.scala | 88 --------------- .../org/apache/spark/mllib/util/MLUtils.scala | 101 ++++++++++++++++++ .../apache/spark/mllib/MLContextSuite.scala | 64 ----------- .../spark/mllib/util/MLUtilsSuite.scala | 45 +++++++- 4 files changed, 145 insertions(+), 153 deletions(-) delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala delete mode 100644 mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala b/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala deleted file mode 100644 index 00988bc480dc8..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/MLContext.scala +++ /dev/null @@ -1,88 +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.mllib - -import org.apache.spark.SparkContext - -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.rdd.RDD - -/** - * Provides methods related to machine learning on top of [[org.apache.spark.SparkContext]]. - * - * @param sparkContext a [[org.apache.spark.SparkContext]] instance - */ -class MLContext(val sparkContext: SparkContext) { - /** - * Reads labeled data in the LIBSVM format into an RDD[LabeledPoint]. - * The LIBSVM format is a text-based format used by LIBSVM and LIBLINEAR. - * Each line represents a labeled sparse feature vector using the following format: - * {{{label index1:value1 index2:value2 ...}}} - * where the indices are one-based and in ascending order. - * This method parses each line into a [[org.apache.spark.mllib.regression.LabeledPoint]], - * where the feature indices are converted to zero-based. - * - * @param path file or directory path in any Hadoop-supported file system URI - * @param minSplits min number of partitions, default: sparkContext.defaultMinSplits - * @param numFeatures number of features, which will be determined from the input data if a - * non-positive value is given. The default value is 0. - * @param labelParser parser for labels, default: _.toDouble - * @return labeled data stored as an RDD[LabeledPoint] - */ - def libSVMFile( - path: String, - minSplits: Int = sparkContext.defaultMinSplits, - numFeatures: Int = 0, - labelParser: String => Double = _.toDouble): RDD[LabeledPoint] = { - val parsed = sparkContext.textFile(path, minSplits) - .map(_.trim) - .filter(!_.isEmpty) - .map(_.split(' ')) - // Determine number of features. - val d = if (numFeatures > 0) { - numFeatures - } else { - parsed.map { items => - if (items.length > 1) { - items.last.split(':')(0).toInt - } else { - 0 - } - }.reduce(math.max) - } - parsed.map { items => - val label = labelParser(items.head) - val (indices, values) = items.tail.map { item => - val indexAndValue = item.split(':') - val index = indexAndValue(0).toInt - 1 - val value = indexAndValue(1).toDouble - (index, value) - }.unzip - LabeledPoint(label, Vectors.sparse(d, indices.toArray, values.toArray)) - } - } -} - -object MLContext { - /** - * Creates an [[org.apache.spark.mllib.MLContext]] instance from - * an [[org.apache.spark.SparkContext]] instance. - */ - def apply(sc: SparkContext): MLContext = new MLContext(sc) -} 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 4c47eb5c7506c..cb85e433bfc73 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 @@ -38,6 +38,107 @@ object MLUtils { eps } + /** + * Multiclass label parser, which parses a string into double. + */ + val multiclassLabelParser: String => Double = _.toDouble + + /** + * Binary label parser, which outputs 1.0 (positive) if the value is greater than 0.5, + * or 0.0 (negative) otherwise. + */ + val binaryLabelParser: String => Double = label => if (label.toDouble > 0.5) 1.0 else 0.0 + + /** + * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint]. + * The LIBSVM format is a text-based format used by LIBSVM and LIBLINEAR. + * Each line represents a labeled sparse feature vector using the following format: + * {{{label index1:value1 index2:value2 ...}}} + * where the indices are one-based and in ascending order. + * This method parses each line into a [[org.apache.spark.mllib.regression.LabeledPoint]], + * where the feature indices are converted to zero-based. + * + * @param sc Spark context + * @param path file or directory path in any Hadoop-supported file system URI + * @param labelParser parser for labels, default: 1.0 if label > 0.5 or 0.0 otherwise + * @param numFeatures number of features, which will be determined from the input data if a + * negative value is given. The default value is -1. + * @param minSplits min number of partitions, default: sc.defaultMinSplits + * @return labeled data stored as an RDD[LabeledPoint] + */ + def loadLibSVMData( + sc: SparkContext, + path: String, + labelParser: String => Double, + numFeatures: Int, + minSplits: Int): RDD[LabeledPoint] = { + val parsed = sc.textFile(path, minSplits) + .map(_.trim) + .filter(!_.isEmpty) + .map(_.split(' ')) + // Determine number of features. + val d = if (numFeatures >= 0) { + numFeatures + } else { + parsed.map { items => + if (items.length > 1) { + items.last.split(':')(0).toInt + } else { + 0 + } + }.reduce(math.max) + } + parsed.map { items => + val label = labelParser(items.head) + val (indices, values) = items.tail.map { item => + val indexAndValue = item.split(':') + val index = indexAndValue(0).toInt - 1 + val value = indexAndValue(1).toDouble + (index, value) + }.unzip + LabeledPoint(label, Vectors.sparse(d, indices.toArray, values.toArray)) + } + } + + // Convenient methods for calling from Java. + + /** + * Loads binary labeled data in the LIBSVM format into an RDD[LabeledPoint], + * with number of features determined automatically and the default number of partitions. + */ + def loadLibSVMData(sc: SparkContext, path: String): RDD[LabeledPoint] = + loadLibSVMData(sc, path, binaryLabelParser, -1, sc.defaultMinSplits) + + /** + * Loads binary labeled data in the LIBSVM format into an RDD[LabeledPoint], + * with number of features specified explicitly and the default number of partitions. + */ + def loadLibSVMData(sc: SparkContext, path: String, numFeatures: Int): RDD[LabeledPoint] = + loadLibSVMData(sc, path, binaryLabelParser, numFeatures, sc.defaultMinSplits) + + /** + * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], + * with the given label parser, number of features determined automatically, + * and the default number of partitions. + */ + def loadLibSVMData( + sc: SparkContext, + path: String, + labelParser: String => Double): RDD[LabeledPoint] = + loadLibSVMData(sc, path, labelParser, -1, sc.defaultMinSplits) + + /** + * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], + * with the given label parser, number of features specified explicitly, + * and the default number of partitions. + */ + def loadLibSVMData( + sc: SparkContext, + path: String, + labelParser: String => Double, + numFeatures: Int): RDD[LabeledPoint] = + loadLibSVMData(sc, path, labelParser, numFeatures, sc.defaultMinSplits) + /** * Load labeled data from a file. The data format used here is * , ... diff --git a/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala deleted file mode 100644 index 6313978d546b9..0000000000000 --- a/mllib/src/test/scala/org/apache/spark/mllib/MLContextSuite.scala +++ /dev/null @@ -1,64 +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.mllib - -import java.io.File - -import org.scalatest.FunSuite - -import com.google.common.base.Charsets -import com.google.common.io.Files - -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.LocalSparkContext - -class MLContextSuite extends FunSuite with LocalSparkContext { - test("libSVMFile") { - val lines = - """ - |1 1:1.0 3:2.0 5:3.0 - |0 - |0 2:4.0 4:5.0 6:6.0 - """.stripMargin - val tempDir = Files.createTempDir() - val file = new File(tempDir.getPath, "part-00000") - Files.write(lines, file, Charsets.US_ASCII) - - val mlc = MLContext(sc) - - val pointsWithNumFeatures = mlc.libSVMFile(tempDir.toURI.toString, numFeatures = 6).collect() - val pointsWithoutNumFeatures = mlc.libSVMFile(tempDir.toURI.toString).collect() - - for (points <- Seq(pointsWithNumFeatures, pointsWithoutNumFeatures)) { - assert(points.length === 3) - assert(points(0).label === 1.0) - assert(points(0).features === Vectors.sparse(6, Seq((0, 1.0), (2, 2.0), (4, 3.0)))) - assert(points(1).label == 0.0) - assert(points(1).features == Vectors.sparse(6, Seq())) - assert(points(2).label === 0.0) - assert(points(2).features === Vectors.sparse(6, Seq((1, 4.0), (3, 5.0), (5, 6.0)))) - } - - try { - file.delete() - tempDir.delete() - } catch { - case t: Throwable => - } - } -} 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 2081fe46b17ef..27d41c7869aa0 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 @@ -17,14 +17,18 @@ package org.apache.spark.mllib.util +import java.io.File + import org.scalatest.FunSuite import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, norm => breezeNorm, squaredDistance => breezeSquaredDistance} +import com.google.common.base.Charsets +import com.google.common.io.Files -import org.apache.spark.mllib.util.MLUtils._ import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.MLUtils._ class MLUtilsSuite extends FunSuite with LocalSparkContext { @@ -63,4 +67,43 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { assert(mean === Vectors.dense(2.0, 3.0, 4.0)) assert(std === Vectors.dense(1.0, 1.0, 1.0)) } + + test("loadLibSVMData") { + val lines = + """ + |+1 1:1.0 3:2.0 5:3.0 + |-1 + |-1 2:4.0 4:5.0 6:6.0 + """.stripMargin + val tempDir = Files.createTempDir() + val file = new File(tempDir.getPath, "part-00000") + Files.write(lines, file, Charsets.US_ASCII) + val path = tempDir.toURI.toString + + val pointsWithNumFeatures = MLUtils.loadLibSVMData(sc, path, 6).collect() + val pointsWithoutNumFeatures = MLUtils.loadLibSVMData(sc, path).collect() + + for (points <- Seq(pointsWithNumFeatures, pointsWithoutNumFeatures)) { + assert(points.length === 3) + assert(points(0).label === 1.0) + assert(points(0).features === Vectors.sparse(6, Seq((0, 1.0), (2, 2.0), (4, 3.0)))) + assert(points(1).label == 0.0) + assert(points(1).features == Vectors.sparse(6, Seq())) + assert(points(2).label === 0.0) + assert(points(2).features === Vectors.sparse(6, Seq((1, 4.0), (3, 5.0), (5, 6.0)))) + } + + val multiclassPoints = MLUtils.loadLibSVMData(sc, path, MLUtils.multiclassLabelParser).collect() + assert(multiclassPoints.length === 3) + assert(multiclassPoints(0).label === 1.0) + assert(multiclassPoints(1).label === -1.0) + assert(multiclassPoints(2).label === -1.0) + + try { + file.delete() + tempDir.delete() + } catch { + case t: Throwable => + } + } }