From 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 31 Aug 2013 19:27:07 -0700 Subject: Initial work to rename package to org.apache.spark --- mllib/pom.xml | 6 +- .../mllib/classification/ClassificationModel.scala | 21 + .../mllib/classification/LogisticRegression.scala | 188 +++++++++ .../apache/spark/mllib/classification/SVM.scala | 187 +++++++++ .../org/apache/spark/mllib/clustering/KMeans.scala | 335 +++++++++++++++ .../spark/mllib/clustering/KMeansModel.scala | 44 ++ .../spark/mllib/clustering/LocalKMeans.scala | 105 +++++ .../apache/spark/mllib/optimization/Gradient.scala | 98 +++++ .../spark/mllib/optimization/GradientDescent.scala | 166 ++++++++ .../spark/mllib/optimization/Optimizer.scala | 29 ++ .../apache/spark/mllib/optimization/Updater.scala | 99 +++++ .../apache/spark/mllib/recommendation/ALS.scala | 453 +++++++++++++++++++++ .../recommendation/MatrixFactorizationModel.scala | 49 +++ .../regression/GeneralizedLinearAlgorithm.scala | 159 ++++++++ .../spark/mllib/regression/LabeledPoint.scala | 26 ++ .../org/apache/spark/mllib/regression/Lasso.scala | 210 ++++++++++ .../spark/mllib/regression/LinearRegression.scala | 167 ++++++++ .../spark/mllib/regression/RegressionModel.scala | 38 ++ .../spark/mllib/regression/RidgeRegression.scala | 213 ++++++++++ .../apache/spark/mllib/util/DataValidators.scala | 42 ++ .../spark/mllib/util/KMeansDataGenerator.scala | 84 ++++ .../spark/mllib/util/LinearDataGenerator.scala | 132 ++++++ .../util/LogisticRegressionDataGenerator.scala | 81 ++++ .../apache/spark/mllib/util/MFDataGenerator.scala | 113 +++++ .../org/apache/spark/mllib/util/MLUtils.scala | 122 ++++++ .../apache/spark/mllib/util/SVMDataGenerator.scala | 50 +++ .../mllib/classification/ClassificationModel.scala | 21 - .../mllib/classification/LogisticRegression.scala | 188 --------- .../scala/spark/mllib/classification/SVM.scala | 187 --------- .../main/scala/spark/mllib/clustering/KMeans.scala | 335 --------------- .../scala/spark/mllib/clustering/KMeansModel.scala | 44 -- .../scala/spark/mllib/clustering/LocalKMeans.scala | 105 ----- .../scala/spark/mllib/optimization/Gradient.scala | 98 ----- .../spark/mllib/optimization/GradientDescent.scala | 166 -------- .../scala/spark/mllib/optimization/Optimizer.scala | 29 -- .../scala/spark/mllib/optimization/Updater.scala | 99 ----- .../scala/spark/mllib/recommendation/ALS.scala | 453 --------------------- .../recommendation/MatrixFactorizationModel.scala | 49 --- .../regression/GeneralizedLinearAlgorithm.scala | 159 -------- .../spark/mllib/regression/LabeledPoint.scala | 26 -- .../main/scala/spark/mllib/regression/Lasso.scala | 210 ---------- .../spark/mllib/regression/LinearRegression.scala | 167 -------- .../spark/mllib/regression/RegressionModel.scala | 38 -- .../spark/mllib/regression/RidgeRegression.scala | 213 ---------- .../scala/spark/mllib/util/DataValidators.scala | 42 -- .../spark/mllib/util/KMeansDataGenerator.scala | 84 ---- .../spark/mllib/util/LinearDataGenerator.scala | 132 ------ .../util/LogisticRegressionDataGenerator.scala | 81 ---- .../scala/spark/mllib/util/MFDataGenerator.scala | 113 ----- .../src/main/scala/spark/mllib/util/MLUtils.scala | 122 ------ .../scala/spark/mllib/util/SVMDataGenerator.scala | 50 --- .../JavaLogisticRegressionSuite.java | 98 +++++ .../spark/mllib/classification/JavaSVMSuite.java | 98 +++++ .../spark/mllib/clustering/JavaKMeansSuite.java | 115 ++++++ .../spark/mllib/recommendation/JavaALSSuite.java | 110 +++++ .../spark/mllib/regression/JavaLassoSuite.java | 97 +++++ .../regression/JavaLinearRegressionSuite.java | 94 +++++ .../mllib/regression/JavaRidgeRegressionSuite.java | 110 +++++ .../JavaLogisticRegressionSuite.java | 98 ----- .../spark/mllib/classification/JavaSVMSuite.java | 98 ----- .../spark/mllib/clustering/JavaKMeansSuite.java | 115 ------ .../spark/mllib/recommendation/JavaALSSuite.java | 110 ----- .../spark/mllib/regression/JavaLassoSuite.java | 97 ----- .../regression/JavaLinearRegressionSuite.java | 94 ----- .../mllib/regression/JavaRidgeRegressionSuite.java | 110 ----- .../classification/LogisticRegressionSuite.scala | 150 +++++++ .../spark/mllib/classification/SVMSuite.scala | 169 ++++++++ .../spark/mllib/clustering/KMeansSuite.scala | 173 ++++++++ .../spark/mllib/recommendation/ALSSuite.scala | 125 ++++++ .../apache/spark/mllib/regression/LassoSuite.scala | 121 ++++++ .../mllib/regression/LinearRegressionSuite.scala | 72 ++++ .../mllib/regression/RidgeRegressionSuite.scala | 90 ++++ .../classification/LogisticRegressionSuite.scala | 150 ------- .../spark/mllib/classification/SVMSuite.scala | 169 -------- .../scala/spark/mllib/clustering/KMeansSuite.scala | 173 -------- .../spark/mllib/recommendation/ALSSuite.scala | 125 ------ .../scala/spark/mllib/regression/LassoSuite.scala | 121 ------ .../mllib/regression/LinearRegressionSuite.scala | 72 ---- .../mllib/regression/RidgeRegressionSuite.scala | 90 ---- 79 files changed, 4836 insertions(+), 4836 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala delete mode 100644 mllib/src/main/scala/spark/mllib/classification/ClassificationModel.scala delete mode 100644 mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala delete mode 100644 mllib/src/main/scala/spark/mllib/classification/SVM.scala delete mode 100644 mllib/src/main/scala/spark/mllib/clustering/KMeans.scala delete mode 100644 mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala delete mode 100644 mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala delete mode 100644 mllib/src/main/scala/spark/mllib/optimization/Gradient.scala delete mode 100644 mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala delete mode 100644 mllib/src/main/scala/spark/mllib/optimization/Optimizer.scala delete mode 100644 mllib/src/main/scala/spark/mllib/optimization/Updater.scala delete mode 100644 mllib/src/main/scala/spark/mllib/recommendation/ALS.scala delete mode 100644 mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala delete mode 100644 mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala delete mode 100644 mllib/src/main/scala/spark/mllib/regression/LabeledPoint.scala delete mode 100644 mllib/src/main/scala/spark/mllib/regression/Lasso.scala delete mode 100644 mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala delete mode 100644 mllib/src/main/scala/spark/mllib/regression/RegressionModel.scala delete mode 100644 mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala delete mode 100644 mllib/src/main/scala/spark/mllib/util/DataValidators.scala delete mode 100644 mllib/src/main/scala/spark/mllib/util/KMeansDataGenerator.scala delete mode 100644 mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala delete mode 100644 mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala delete mode 100644 mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala delete mode 100644 mllib/src/main/scala/spark/mllib/util/MLUtils.scala delete mode 100644 mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala create mode 100644 mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java create mode 100644 mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java create mode 100644 mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java create mode 100644 mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java create mode 100644 mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java create mode 100644 mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java create mode 100644 mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java delete mode 100644 mllib/src/test/java/spark/mllib/classification/JavaLogisticRegressionSuite.java delete mode 100644 mllib/src/test/java/spark/mllib/classification/JavaSVMSuite.java delete mode 100644 mllib/src/test/java/spark/mllib/clustering/JavaKMeansSuite.java delete mode 100644 mllib/src/test/java/spark/mllib/recommendation/JavaALSSuite.java delete mode 100644 mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java delete mode 100644 mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java delete mode 100644 mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala delete mode 100644 mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala delete mode 100644 mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala delete mode 100644 mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala delete mode 100644 mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala delete mode 100644 mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala delete mode 100644 mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala delete mode 100644 mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala (limited to 'mllib') diff --git a/mllib/pom.xml b/mllib/pom.xml index ab31d5734e..2d5d3c00d1 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -19,13 +19,13 @@ 4.0.0 - org.spark-project + org.apache.spark spark-parent 0.8.0-SNAPSHOT ../pom.xml - org.spark-project + org.apache.spark spark-mllib jar Spark Project ML Library @@ -33,7 +33,7 @@ - org.spark-project + org.apache.spark spark-core ${project.version} 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 new file mode 100644 index 0000000000..4f4a7f5296 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala @@ -0,0 +1,21 @@ +package org.apache.spark.mllib.classification + +import org.apache.spark.RDD + +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 + */ + def predict(testData: RDD[Array[Double]]): RDD[Double] + + /** + * 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 + */ + def predict(testData: Array[Double]): 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 new file mode 100644 index 0000000000..91bb50c829 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -0,0 +1,188 @@ +/* + * 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.classification + +import scala.math.round + +import org.apache.spark.{Logging, RDD, SparkContext} +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 + +/** + * Classification model trained using Logistic Regression. + * + * @param weights Weights computed for every feature. + * @param intercept Intercept computed for this model. + */ +class LogisticRegressionModel( + override val weights: Array[Double], + override val intercept: Double) + extends GeneralizedLinearModel(weights, intercept) + with ClassificationModel with Serializable { + + override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, + intercept: Double) = { + val margin = dataMatrix.mmul(weightMatrix).get(0) + intercept + round(1.0/ (1.0 + math.exp(margin * -1))) + } +} + +/** + * Train a classification model for Logistic Regression using Stochastic Gradient Descent. + * NOTE: Labels used in Logistic Regression should be {0, 1} + */ +class LogisticRegressionWithSGD private ( + var stepSize: Double, + var numIterations: Int, + var regParam: Double, + var miniBatchFraction: Double) + 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) + override val validators = List(DataValidators.classificationLabels) + + /** + * Construct a LogisticRegression object with default parameters + */ + def this() = this(1.0, 100, 0.0, 1.0) + + def createModel(weights: Array[Double], intercept: Double) = { + new LogisticRegressionModel(weights, intercept) + } +} + +/** + * Top-level methods for calling Logistic Regression. + * NOTE: Labels used in Logistic Regression should be {0, 1} + */ +object LogisticRegressionWithSGD { + // NOTE(shivaram): We use multiple train methods instead of default arguments to support + // Java programs. + + /** + * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed + * number of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in + * gradient descent are initialized using the initial weights provided. + * NOTE: Labels used in Logistic Regression should be {0, 1} + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param miniBatchFraction Fraction of data to be used per iteration. + * @param initialWeights Initial set of weights to be used. Array should be equal in size to + * the number of features in the data. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double, + initialWeights: Array[Double]) + : LogisticRegressionModel = + { + new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction).run( + input, initialWeights) + } + + /** + * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed + * number of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. + * NOTE: Labels used in Logistic Regression should be {0, 1} + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + + * @param miniBatchFraction Fraction of data to be used per iteration. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double) + : LogisticRegressionModel = + { + new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction).run( + input) + } + + /** + * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed + * number of iterations of gradient descent using the specified step size. We use the entire data + * set to update the gradient in each iteration. + * NOTE: Labels used in Logistic Regression should be {0, 1} + * + * @param input RDD of (label, array of features) pairs. + * @param stepSize Step size to be used for each iteration of Gradient Descent. + + * @param numIterations Number of iterations of gradient descent to run. + * @return a LogisticRegressionModel which has the weights and offset from training. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double) + : LogisticRegressionModel = + { + train(input, numIterations, stepSize, 1.0) + } + + /** + * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed + * number of iterations of gradient descent using a step size of 1.0. We use the entire data set + * to update the gradient in each iteration. + * NOTE: Labels used in Logistic Regression should be {0, 1} + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @return a LogisticRegressionModel which has the weights and offset from training. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int) + : LogisticRegressionModel = + { + train(input, numIterations, 1.0, 1.0) + } + + def main(args: Array[String]) { + if (args.length != 4) { + println("Usage: LogisticRegression " + + "") + System.exit(1) + } + 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) + + sc.stop() + } +} 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 new file mode 100644 index 0000000000..c92c7cc3f3 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -0,0 +1,187 @@ +/* + * 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.classification + +import scala.math.signum + +import org.apache.spark.{Logging, RDD, SparkContext} +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 + +/** + * Model built using SVM. + * + * @param weights Weights computed for every feature. + * @param intercept Intercept computed for this model. + */ +class SVMModel( + override val weights: Array[Double], + override val intercept: Double) + extends GeneralizedLinearModel(weights, intercept) + with ClassificationModel with Serializable { + + override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, + intercept: Double) = { + val margin = dataMatrix.dot(weightMatrix) + intercept + if (margin < 0) 0.0 else 1.0 + } +} + +/** + * Train an SVM using Stochastic Gradient Descent. + * NOTE: Labels used in SVM should be {0, 1} + */ +class SVMWithSGD private ( + var stepSize: Double, + var numIterations: Int, + var regParam: Double, + var miniBatchFraction: Double) + extends GeneralizedLinearAlgorithm[SVMModel] with Serializable { + + val gradient = new HingeGradient() + val updater = new SquaredL2Updater() + override val optimizer = new GradientDescent(gradient, updater) + .setStepSize(stepSize) + .setNumIterations(numIterations) + .setRegParam(regParam) + .setMiniBatchFraction(miniBatchFraction) + + override val validators = List(DataValidators.classificationLabels) + + /** + * Construct a SVM object with default parameters + */ + def this() = this(1.0, 100, 1.0, 1.0) + + def createModel(weights: Array[Double], intercept: Double) = { + new SVMModel(weights, intercept) + } +} + +/** + * Top-level methods for calling SVM. NOTE: Labels used in SVM should be {0, 1} + */ +object SVMWithSGD { + + /** + * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in + * gradient descent are initialized using the initial weights provided. + * NOTE: Labels used in SVM should be {0, 1} + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + * @param initialWeights Initial set of weights to be used. Array should be equal in size to + * the number of features in the data. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, + initialWeights: Array[Double]) + : SVMModel = + { + new SVMWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input, + initialWeights) + } + + /** + * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. + * NOTE: Labels used in SVM should be {0, 1} + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double) + : SVMModel = + { + new SVMWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input) + } + + /** + * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. We use the entire data set to + * update the gradient in each iteration. + * NOTE: Labels used in SVM should be {0, 1} + * + * @param input RDD of (label, array of features) pairs. + * @param stepSize Step size to be used for each iteration of Gradient Descent. + * @param regParam Regularization parameter. + * @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, + stepSize: Double, + regParam: Double) + : SVMModel = + { + train(input, numIterations, stepSize, regParam, 1.0) + } + + /** + * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using a step size of 1.0. We use the entire data set to + * update the gradient in each iteration. + * NOTE: Labels used in SVM should be {0, 1} + * + * @param input RDD of (label, array of features) pairs. + * @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 = + { + train(input, numIterations, 1.0, 1.0, 1.0) + } + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: SVM ") + System.exit(1) + } + 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) + + sc.stop() + } +} 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 new file mode 100644 index 0000000000..2c3db099fa --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -0,0 +1,335 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.clustering + +import scala.collection.mutable.ArrayBuffer +import scala.util.Random + +import org.apache.spark.{SparkContext, RDD} +import org.apache.spark.SparkContext._ +import org.apache.spark.Logging +import org.apache.spark.mllib.util.MLUtils + +import org.jblas.DoubleMatrix + + +/** + * K-means clustering with support for multiple parallel runs and a k-means++ like initialization + * mode (the k-means|| algorithm by Bahmani et al). When multiple concurrent runs are requested, + * they are executed together with joint passes over the data for efficiency. + * + * This is an iterative algorithm that will make multiple passes over the data, so any RDDs given + * to it should be cached by the user. + */ +class KMeans private ( + var k: Int, + var maxIterations: Int, + var runs: Int, + var initializationMode: String, + var initializationSteps: Int, + var epsilon: Double) + extends Serializable with Logging +{ + private type ClusterCenters = Array[Array[Double]] + + def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 5, 1e-4) + + /** Set the number of clusters to create (k). Default: 2. */ + def setK(k: Int): KMeans = { + this.k = k + this + } + + /** Set maximum number of iterations to run. Default: 20. */ + def setMaxIterations(maxIterations: Int): KMeans = { + this.maxIterations = maxIterations + this + } + + /** + * Set the initialization algorithm. This can be either "random" to choose random points as + * initial cluster centers, or "k-means||" to use a parallel variant of k-means++ + * (Bahmani et al., Scalable K-Means++, VLDB 2012). Default: k-means||. + */ + def setInitializationMode(initializationMode: String): KMeans = { + if (initializationMode != KMeans.RANDOM && initializationMode != KMeans.K_MEANS_PARALLEL) { + throw new IllegalArgumentException("Invalid initialization mode: " + initializationMode) + } + this.initializationMode = initializationMode + this + } + + /** + * Set the number of runs of the algorithm to execute in parallel. We initialize the algorithm + * this many times with random starting conditions (configured by the initialization mode), then + * return the best clustering found over any run. Default: 1. + */ + def setRuns(runs: Int): KMeans = { + if (runs <= 0) { + throw new IllegalArgumentException("Number of runs must be positive") + } + this.runs = runs + this + } + + /** + * Set the number of steps for the k-means|| initialization mode. This is an advanced + * setting -- the default of 5 is almost always enough. Default: 5. + */ + def setInitializationSteps(initializationSteps: Int): KMeans = { + if (initializationSteps <= 0) { + throw new IllegalArgumentException("Number of initialization steps must be positive") + } + this.initializationSteps = initializationSteps + this + } + + /** + * Set the distance threshold within which we've consider centers to have converged. + * If all centers move less than this Euclidean distance, we stop iterating one run. + */ + def setEpsilon(epsilon: Double): KMeans = { + this.epsilon = epsilon + this + } + + /** + * Train a K-means model on the given set of points; `data` should be cached for high + * performance, because this is an iterative algorithm. + */ + def run(data: RDD[Array[Double]]): KMeansModel = { + // TODO: check whether data is persistent; this needs RDD.storageLevel to be publicly readable + + val sc = data.sparkContext + + val centers = if (initializationMode == KMeans.RANDOM) { + initRandom(data) + } else { + initKMeansParallel(data) + } + + val active = Array.fill(runs)(true) + val costs = Array.fill(runs)(0.0) + + var activeRuns = new ArrayBuffer[Int] ++ (0 until runs) + var iteration = 0 + + // Execute iterations of Lloyd's algorithm until all runs have converged + while (iteration < maxIterations && !activeRuns.isEmpty) { + type WeightedPoint = (DoubleMatrix, Long) + def mergeContribs(p1: WeightedPoint, p2: WeightedPoint): WeightedPoint = { + (p1._1.addi(p2._1), p1._2 + p2._2) + } + + val activeCenters = activeRuns.map(r => centers(r)).toArray + val costAccums = activeRuns.map(_ => sc.accumulator(0.0)) + + // Find the sum and count of points mapping to each center + val totalContribs = data.mapPartitions { points => + val runs = activeCenters.length + val k = activeCenters(0).length + val dims = activeCenters(0)(0).length + + val sums = Array.fill(runs, k)(new DoubleMatrix(dims)) + val counts = Array.fill(runs, k)(0L) + + for (point <- points; (centers, runIndex) <- activeCenters.zipWithIndex) { + val (bestCenter, cost) = KMeans.findClosest(centers, point) + costAccums(runIndex) += cost + sums(runIndex)(bestCenter).addi(new DoubleMatrix(point)) + counts(runIndex)(bestCenter) += 1 + } + + val contribs = for (i <- 0 until runs; j <- 0 until k) yield { + ((i, j), (sums(i)(j), counts(i)(j))) + } + contribs.iterator + }.reduceByKey(mergeContribs).collectAsMap() + + // Update the cluster centers and costs for each active run + for ((run, i) <- activeRuns.zipWithIndex) { + var changed = false + for (j <- 0 until k) { + val (sum, count) = totalContribs((i, j)) + if (count != 0) { + val newCenter = sum.divi(count).data + if (MLUtils.squaredDistance(newCenter, centers(run)(j)) > epsilon * epsilon) { + changed = true + } + centers(run)(j) = newCenter + } + } + if (!changed) { + active(run) = false + logInfo("Run " + run + " finished in " + (iteration + 1) + " iterations") + } + costs(run) = costAccums(i).value + } + + activeRuns = activeRuns.filter(active(_)) + iteration += 1 + } + + val bestRun = costs.zipWithIndex.min._2 + new KMeansModel(centers(bestRun)) + } + + /** + * Initialize `runs` sets of cluster centers at random. + */ + private def initRandom(data: RDD[Array[Double]]): Array[ClusterCenters] = { + // Sample all the cluster centers in one pass to avoid repeated scans + val sample = data.takeSample(true, runs * k, new Random().nextInt()).toSeq + Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k).toArray) + } + + /** + * Initialize `runs` sets of cluster centers using the k-means|| algorithm by Bahmani et al. + * (Bahmani et al., Scalable K-Means++, VLDB 2012). This is a variant of k-means++ that tries + * to find with dissimilar cluster centers by starting with a random center and then doing + * passes where more centers are chosen with probability proportional to their squared distance + * to the current cluster set. It results in a provable approximation to an optimal clustering. + * + * The original paper can be found at http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf. + */ + private def initKMeansParallel(data: RDD[Array[Double]]): Array[ClusterCenters] = { + // Initialize each run's center to a random point + val seed = new Random().nextInt() + val sample = data.takeSample(true, runs, seed).toSeq + val centers = Array.tabulate(runs)(r => ArrayBuffer(sample(r))) + + // On each step, sample 2 * k points on average for each run with probability proportional + // to their squared distance from that run's current centers + for (step <- 0 until initializationSteps) { + val centerArrays = centers.map(_.toArray) + val sumCosts = data.flatMap { point => + for (r <- 0 until runs) yield (r, KMeans.pointCost(centerArrays(r), point)) + }.reduceByKey(_ + _).collectAsMap() + val chosen = data.mapPartitionsWithIndex { (index, points) => + val rand = new Random(seed ^ (step << 16) ^ index) + for { + p <- points + r <- 0 until runs + if rand.nextDouble() < KMeans.pointCost(centerArrays(r), p) * 2 * k / sumCosts(r) + } yield (r, p) + }.collect() + for ((r, p) <- chosen) { + centers(r) += p + } + } + + // Finally, we might have a set of more than k candidate centers for each run; weigh each + // candidate by the number of points in the dataset mapping to it and run a local k-means++ + // on the weighted centers to pick just k of them + val centerArrays = centers.map(_.toArray) + val weightMap = data.flatMap { p => + for (r <- 0 until runs) yield ((r, KMeans.findClosest(centerArrays(r), p)._1), 1.0) + }.reduceByKey(_ + _).collectAsMap() + val finalCenters = (0 until runs).map { r => + val myCenters = centers(r).toArray + val myWeights = (0 until myCenters.length).map(i => weightMap.getOrElse((r, i), 0.0)).toArray + LocalKMeans.kMeansPlusPlus(r, myCenters, myWeights, k, 30) + } + + finalCenters.toArray + } +} + + +/** + * Top-level methods for calling K-means clustering. + */ +object KMeans { + // Initialization mode names + val RANDOM = "random" + val K_MEANS_PARALLEL = "k-means||" + + def train( + data: RDD[Array[Double]], + k: Int, + maxIterations: Int, + runs: Int, + initializationMode: String) + : KMeansModel = + { + new KMeans().setK(k) + .setMaxIterations(maxIterations) + .setRuns(runs) + .setInitializationMode(initializationMode) + .run(data) + } + + def train(data: RDD[Array[Double]], k: Int, maxIterations: Int, runs: Int): KMeansModel = { + train(data, k, maxIterations, runs, K_MEANS_PARALLEL) + } + + def train(data: RDD[Array[Double]], k: Int, maxIterations: Int): KMeansModel = { + train(data, k, maxIterations, 1, K_MEANS_PARALLEL) + } + + /** + * Return the index of the closest point in `centers` to `point`, as well as its distance. + */ + private[mllib] def findClosest(centers: Array[Array[Double]], point: Array[Double]) + : (Int, Double) = + { + var bestDistance = Double.PositiveInfinity + var bestIndex = 0 + for (i <- 0 until centers.length) { + val distance = MLUtils.squaredDistance(point, centers(i)) + if (distance < bestDistance) { + bestDistance = distance + bestIndex = i + } + } + (bestIndex, bestDistance) + } + + /** + * Return the K-means cost of a given point against the given cluster centers. + */ + private[mllib] def pointCost(centers: Array[Array[Double]], point: Array[Double]): Double = { + var bestDistance = Double.PositiveInfinity + for (i <- 0 until centers.length) { + val distance = MLUtils.squaredDistance(point, centers(i)) + if (distance < bestDistance) { + bestDistance = distance + } + } + bestDistance + } + + def main(args: Array[String]) { + if (args.length < 4) { + println("Usage: KMeans []") + System.exit(1) + } + val (master, inputFile, k, iters) = (args(0), args(1), args(2).toInt, args(3).toInt) + val runs = if (args.length >= 5) args(4).toInt else 1 + val sc = new SparkContext(master, "KMeans") + val data = sc.textFile(inputFile).map(line => line.split(' ').map(_.toDouble)).cache() + val model = KMeans.train(data, k, iters, runs) + val cost = model.computeCost(data) + println("Cluster centers:") + for (c <- model.clusterCenters) { + println(" " + c.mkString(" ")) + } + println("Cost: " + cost) + System.exit(0) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala new file mode 100644 index 0000000000..d1fe5d138d --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.clustering + +import org.apache.spark.RDD +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.util.MLUtils + + +/** + * A clustering model for K-means. Each point belongs to the cluster with the closest center. + */ +class KMeansModel(val clusterCenters: Array[Array[Double]]) extends Serializable { + /** Total number of clusters. */ + def k: Int = clusterCenters.length + + /** Return the cluster index that a given point belongs to. */ + def predict(point: Array[Double]): Int = { + KMeans.findClosest(clusterCenters, point)._1 + } + + /** + * Return the K-means cost (sum of squared distances of points to their nearest center) for this + * model on the given data. + */ + def computeCost(data: RDD[Array[Double]]): Double = { + data.map(p => KMeans.pointCost(clusterCenters, p)).sum + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala new file mode 100644 index 0000000000..baf8251d8f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.clustering + +import scala.util.Random + +import org.jblas.{DoubleMatrix, SimpleBlas} + +/** + * An utility object to run K-means locally. This is private to the ML package because it's used + * in the initialization of KMeans but not meant to be publicly exposed. + */ +private[mllib] object LocalKMeans { + /** + * Run K-means++ on the weighted point set `points`. This first does the K-means++ + * initialization procedure and then roudns of Lloyd's algorithm. + */ + def kMeansPlusPlus( + seed: Int, + points: Array[Array[Double]], + weights: Array[Double], + k: Int, + maxIterations: Int) + : Array[Array[Double]] = + { + val rand = new Random(seed) + val dimensions = points(0).length + val centers = new Array[Array[Double]](k) + + // Initialize centers by sampling using the k-means++ procedure + centers(0) = pickWeighted(rand, points, weights) + for (i <- 1 until k) { + // Pick the next center with a probability proportional to cost under current centers + val curCenters = centers.slice(0, i) + val sum = points.zip(weights).map { case (p, w) => + w * KMeans.pointCost(curCenters, p) + }.sum + val r = rand.nextDouble() * sum + var cumulativeScore = 0.0 + var j = 0 + while (j < points.length && cumulativeScore < r) { + cumulativeScore += weights(j) * KMeans.pointCost(curCenters, points(j)) + j += 1 + } + centers(i) = points(j-1) + } + + // Run up to maxIterations iterations of Lloyd's algorithm + val oldClosest = Array.fill(points.length)(-1) + var iteration = 0 + var moved = true + while (moved && iteration < maxIterations) { + moved = false + val sums = Array.fill(k)(new DoubleMatrix(dimensions)) + val counts = Array.fill(k)(0.0) + for ((p, i) <- points.zipWithIndex) { + val index = KMeans.findClosest(centers, p)._1 + SimpleBlas.axpy(weights(i), new DoubleMatrix(p), sums(index)) + counts(index) += weights(i) + if (index != oldClosest(i)) { + moved = true + oldClosest(i) = index + } + } + // Update centers + for (i <- 0 until k) { + if (counts(i) == 0.0) { + // Assign center to a random point + centers(i) = points(rand.nextInt(points.length)) + } else { + centers(i) = sums(i).divi(counts(i)).data + } + } + iteration += 1 + } + + centers + } + + private def pickWeighted[T](rand: Random, data: Array[T], weights: Array[Double]): T = { + val r = rand.nextDouble() * weights.sum + var i = 0 + var curWeight = 0.0 + while (i < data.length && curWeight < r) { + curWeight += weights(i) + i += 1 + } + data(i - 1) + } +} 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 new file mode 100644 index 0000000000..749e7364f4 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -0,0 +1,98 @@ +/* + * 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.optimization + +import org.jblas.DoubleMatrix + +/** + * Class used to compute the gradient for a loss function, given a single data point. + */ +abstract class Gradient extends Serializable { + /** + * Compute the gradient and loss given features of a single data point. + * + * @param data - Feature values for one data point. Column matrix of size nx1 + * where n 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. + * + */ + def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): + (DoubleMatrix, Double) +} + +/** + * Compute gradient and loss for a logistic loss function. + */ +class LogisticGradient extends Gradient { + override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): + (DoubleMatrix, Double) = { + val margin: Double = -1.0 * data.dot(weights) + val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label + + val gradient = data.mul(gradientMultiplier) + val loss = + if (margin > 0) { + math.log(1 + math.exp(0 - margin)) + } else { + math.log(1 + math.exp(margin)) - margin + } + + (gradient, loss) + } +} + +/** + * Compute gradient and loss for a Least-squared loss function. + */ +class SquaredGradient extends Gradient { + override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): + (DoubleMatrix, Double) = { + val diff: Double = data.dot(weights) - label + + val loss = 0.5 * diff * diff + val gradient = data.mul(diff) + + (gradient, loss) + } +} + +/** + * Compute gradient and loss for a Hinge loss function. + * 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) + + // 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) + } else { + (DoubleMatrix.zeros(1, weights.length), 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 new file mode 100644 index 0000000000..b62c9b3340 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -0,0 +1,166 @@ +/* + * 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.optimization + +import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.SparkContext._ + +import org.jblas.DoubleMatrix + +import scala.collection.mutable.ArrayBuffer + +/** + * Class used to solve an optimization problem using Gradient Descent. + * @param gradient Gradient function to be used. + * @param updater Updater to be used to update weights after every iteration. + */ +class GradientDescent(var gradient: Gradient, var updater: Updater) extends Optimizer { + + private var stepSize: Double = 1.0 + private var numIterations: Int = 100 + private var regParam: Double = 0.0 + private var miniBatchFraction: Double = 1.0 + + /** + * Set the step size per-iteration of SGD. Default 1.0. + */ + def setStepSize(step: Double): this.type = { + this.stepSize = step + this + } + + /** + * Set fraction of data to be used for each SGD iteration. Default 1.0. + */ + def setMiniBatchFraction(fraction: Double): this.type = { + this.miniBatchFraction = fraction + this + } + + /** + * Set the number of iterations for SGD. Default 100. + */ + def setNumIterations(iters: Int): this.type = { + this.numIterations = iters + this + } + + /** + * Set the regularization parameter used for SGD. Default 0.0. + */ + def setRegParam(regParam: Double): this.type = { + this.regParam = regParam + this + } + + /** + * Set the gradient function to be used for SGD. + */ + def setGradient(gradient: Gradient): this.type = { + this.gradient = gradient + this + } + + + /** + * Set the updater function to be used for SGD. + */ + def setUpdater(updater: Updater): this.type = { + this.updater = updater + this + } + + def optimize(data: RDD[(Double, Array[Double])], initialWeights: Array[Double]) + : Array[Double] = { + + val (weights, stochasticLossHistory) = GradientDescent.runMiniBatchSGD( + data, + gradient, + updater, + stepSize, + numIterations, + regParam, + miniBatchFraction, + initialWeights) + weights + } + +} + +// Top-level method to run gradient descent. +object GradientDescent extends Logging { + /** + * Run gradient descent in parallel using mini batches. + * + * @param data - Input data for SGD. RDD of form (label, [feature values]). + * @param gradient - Gradient object that will be used to compute the gradient. + * @param updater - Updater object that will be used to update the model. + * @param stepSize - stepSize to be used during update. + * @param numIterations - number of iterations that SGD should be run. + * @param regParam - regularization parameter + * @param miniBatchFraction - fraction of the input data set that should be used for + * one iteration of SGD. Default value 1.0. + * + * @return A tuple containing two elements. The first element is a column matrix containing + * weights for every feature, and the second element is an array containing the stochastic + * loss computed for every iteration. + */ + def runMiniBatchSGD( + data: RDD[(Double, Array[Double])], + gradient: Gradient, + updater: Updater, + stepSize: Double, + numIterations: Int, + regParam: Double, + miniBatchFraction: Double, + initialWeights: Array[Double]) : (Array[Double], Array[Double]) = { + + val stochasticLossHistory = new ArrayBuffer[Double](numIterations) + + val nexamples: Long = data.count() + val miniBatchSize = nexamples * miniBatchFraction + + // Initialize weights as a column vector + var weights = new DoubleMatrix(initialWeights.length, 1, initialWeights:_*) + var regVal = 0.0 + + for (i <- 1 to numIterations) { + 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)) + + /** + * NOTE(Xinghao): lossSum is computed using the weights from the previous iteration + * and regVal is the regularization value computed in the previous iteration as well. + */ + stochasticLossHistory.append(lossSum / miniBatchSize + regVal) + val update = updater.compute( + weights, gradientSum.div(miniBatchSize), stepSize, i, regParam) + weights = update._1 + regVal = update._2 + } + + logInfo("GradientDescent finished. Last 10 stochastic losses %s".format( + stochasticLossHistory.takeRight(10).mkString(", "))) + + (weights.toArray, stochasticLossHistory.toArray) + } +} 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 new file mode 100644 index 0000000000..50059d385d --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala @@ -0,0 +1,29 @@ +/* + * 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.optimization + +import org.apache.spark.RDD + +trait Optimizer { + + /** + * Solve the provided convex optimization problem. + */ + def optimize(data: RDD[(Double, Array[Double])], initialWeights: Array[Double]): Array[Double] + +} 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 new file mode 100644 index 0000000000..4c51f4f881 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala @@ -0,0 +1,99 @@ +/* + * 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.optimization + +import scala.math._ +import org.jblas.DoubleMatrix + +/** + * Class used to update weights used in Gradient Descent. + */ +abstract class Updater extends Serializable { + /** + * Compute an updated value for weights given the gradient, stepSize, iteration number and + * regularization parameter. Also returns the regularization value computed using the + * *updated* weights. + * + * @param weightsOld - Column matrix of size nx1 where n is the number of features. + * @param gradient - Column matrix of size nx1 where n is the number of features. + * @param stepSize - step size across iterations + * @param iter - Iteration number + * @param regParam - Regularization parameter + * + * @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) +} + +/** + * A simple updater that adaptively adjusts the learning rate the + * square root of the number of iterations. Does not perform any regularization. + */ +class SimpleUpdater extends Updater { + override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, + stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { + val thisIterStepSize = stepSize / math.sqrt(iter) + val normGradient = gradient.mul(thisIterStepSize) + (weightsOld.sub(normGradient), 0) + } +} + +/** + * Updater that adjusts learning rate and performs L1 regularization. + * + * The corresponding proximal operator used is the soft-thresholding function. + * That is, each weight component is shrunk towards 0 by shrinkageVal. + * + * If w > shrinkageVal, set weight component to w-shrinkageVal. + * If w < -shrinkageVal, set weight component to w+shrinkageVal. + * If -shrinkageVal < w < shrinkageVal, set weight component to 0. + * + * 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) = { + val thisIterStepSize = stepSize / math.sqrt(iter) + val normGradient = gradient.mul(thisIterStepSize) + // Take gradient step + val newWeights = weightsOld.sub(normGradient) + // 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)) + } + (newWeights, newWeights.norm1 * regParam) + } +} + +/** + * Updater that adjusts the learning rate and performs L2 regularization + */ +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 normGradient = gradient.mul(thisIterStepSize) + val newWeights = weightsOld.sub(normGradient).div(2.0 * thisIterStepSize * regParam + 1.0) + (newWeights, pow(newWeights.norm2, 2.0) * regParam) + } +} + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala new file mode 100644 index 0000000000..218217acfe --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -0,0 +1,453 @@ +/* + * 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.recommendation + +import scala.collection.mutable.{ArrayBuffer, BitSet} +import scala.util.Random +import scala.util.Sorting + +import org.apache.spark.{HashPartitioner, Partitioner, SparkContext, RDD} +import org.apache.spark.storage.StorageLevel +import org.apache.spark.KryoRegistrator +import org.apache.spark.SparkContext._ + +import com.esotericsoftware.kryo.Kryo +import org.jblas.{DoubleMatrix, SimpleBlas, Solve} + + +/** + * Out-link information for a user or product block. This includes the original user/product IDs + * of the elements within this block, and the list of destination blocks that each user or + * product will need to send its feature vector to. + */ +private[recommendation] case class OutLinkBlock(elementIds: Array[Int], shouldSend: Array[BitSet]) + + +/** + * In-link information for a user (or product) block. This includes the original user/product IDs + * of the elements within this block, as well as an array of indices and ratings that specify + * which user in the block will be rated by which products from each product block (or vice-versa). + * Specifically, if this InLinkBlock is for users, ratingsForBlock(b)(i) will contain two arrays, + * indices and ratings, for the i'th product that will be sent to us by product block b (call this + * P). These arrays represent the users that product P had ratings for (by their index in this + * block), as well as the corresponding rating for each one. We can thus use this information when + * we get product block b's message to update the corresponding users. + */ +private[recommendation] case class InLinkBlock( + elementIds: Array[Int], ratingsForBlock: Array[Array[(Array[Int], Array[Double])]]) + + +/** + * A more compact class to represent a rating than Tuple3[Int, Int, Double]. + */ +case class Rating(val user: Int, val product: Int, val rating: Double) + +/** + * Alternating Least Squares matrix factorization. + * + * This is a blocked implementation of the ALS factorization algorithm that groups the two sets + * of factors (referred to as "users" and "products") into blocks and reduces communication by only + * sending one copy of each user vector to each product block on each iteration, and only for the + * product blocks that need that user's feature vector. This is achieved by precomputing some + * information about the ratings matrix to determine the "out-links" of each user (which blocks of + * products it will contribute to) and "in-link" information for each product (which of the feature + * vectors it receives from each user block it will depend on). This allows us to send only an + * array of feature vectors between each user block and product block, and have the product block + * find the users' ratings and update the products based on these messages. + */ +class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var lambda: Double) + extends Serializable +{ + def this() = this(-1, 10, 10, 0.01) + + /** + * Set the number of blocks to parallelize the computation into; pass -1 for an auto-configured + * number of blocks. Default: -1. + */ + def setBlocks(numBlocks: Int): ALS = { + this.numBlocks = numBlocks + this + } + + /** Set the rank of the feature matrices computed (number of features). Default: 10. */ + def setRank(rank: Int): ALS = { + this.rank = rank + this + } + + /** Set the number of iterations to run. Default: 10. */ + def setIterations(iterations: Int): ALS = { + this.iterations = iterations + this + } + + /** Set the regularization parameter, lambda. Default: 0.01. */ + def setLambda(lambda: Double): ALS = { + this.lambda = lambda + this + } + + /** + * Run ALS with the configured parameters on an input RDD of (user, product, rating) triples. + * Returns a MatrixFactorizationModel with feature vectors for each user and product. + */ + def run(ratings: RDD[Rating]): MatrixFactorizationModel = { + val numBlocks = if (this.numBlocks == -1) { + math.max(ratings.context.defaultParallelism, ratings.partitions.size / 2) + } else { + this.numBlocks + } + + val partitioner = new HashPartitioner(numBlocks) + + val ratingsByUserBlock = ratings.map{ rating => (rating.user % numBlocks, rating) } + val ratingsByProductBlock = ratings.map{ rating => + (rating.product % numBlocks, Rating(rating.product, rating.user, rating.rating)) + } + + val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock) + val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) + + // Initialize user and product factors randomly, but use a deterministic seed for each partition + // so that fault recovery works + val seedGen = new Random() + val seed1 = seedGen.nextInt() + val seed2 = seedGen.nextInt() + // Hash an integer to propagate random bits at all positions, similar to java.util.HashTable + def hash(x: Int): Int = { + val r = x ^ (x >>> 20) ^ (x >>> 12) + r ^ (r >>> 7) ^ (r >>> 4) + } + var users = userOutLinks.mapPartitionsWithIndex { (index, itr) => + val rand = new Random(hash(seed1 ^ index)) + itr.map { case (x, y) => + (x, y.elementIds.map(_ => randomFactor(rank, rand))) + } + } + var products = productOutLinks.mapPartitionsWithIndex { (index, itr) => + val rand = new Random(hash(seed2 ^ index)) + itr.map { case (x, y) => + (x, y.elementIds.map(_ => randomFactor(rank, rand))) + } + } + + for (iter <- 0 until iterations) { + // perform ALS update + products = updateFeatures(users, userOutLinks, productInLinks, partitioner, rank, lambda) + users = updateFeatures(products, productOutLinks, userInLinks, partitioner, rank, lambda) + } + + // Flatten and cache the two final RDDs to un-block them + val usersOut = users.join(userOutLinks).flatMap { case (b, (factors, outLinkBlock)) => + for (i <- 0 until factors.length) yield (outLinkBlock.elementIds(i), factors(i)) + } + val productsOut = products.join(productOutLinks).flatMap { case (b, (factors, outLinkBlock)) => + for (i <- 0 until factors.length) yield (outLinkBlock.elementIds(i), factors(i)) + } + + usersOut.persist() + productsOut.persist() + + new MatrixFactorizationModel(rank, usersOut, productsOut) + } + + /** + * Make the out-links table for a block of the users (or products) dataset given the list of + * (user, product, rating) values for the users in that block (or the opposite for products). + */ + private def makeOutLinkBlock(numBlocks: Int, ratings: Array[Rating]): OutLinkBlock = { + val userIds = ratings.map(_.user).distinct.sorted + val numUsers = userIds.length + val userIdToPos = userIds.zipWithIndex.toMap + val shouldSend = Array.fill(numUsers)(new BitSet(numBlocks)) + for (r <- ratings) { + shouldSend(userIdToPos(r.user))(r.product % numBlocks) = true + } + OutLinkBlock(userIds, shouldSend) + } + + /** + * Make the in-links table for a block of the users (or products) dataset given a list of + * (user, product, rating) values for the users in that block (or the opposite for products). + */ + private def makeInLinkBlock(numBlocks: Int, ratings: Array[Rating]): InLinkBlock = { + val userIds = ratings.map(_.user).distinct.sorted + val numUsers = userIds.length + val userIdToPos = userIds.zipWithIndex.toMap + // Split out our ratings by product block + val blockRatings = Array.fill(numBlocks)(new ArrayBuffer[Rating]) + for (r <- ratings) { + blockRatings(r.product % numBlocks) += r + } + val ratingsForBlock = new Array[Array[(Array[Int], Array[Double])]](numBlocks) + for (productBlock <- 0 until numBlocks) { + // Create an array of (product, Seq(Rating)) ratings + val groupedRatings = blockRatings(productBlock).groupBy(_.product).toArray + // Sort them by product ID + val ordering = new Ordering[(Int, ArrayBuffer[Rating])] { + def compare(a: (Int, ArrayBuffer[Rating]), b: (Int, ArrayBuffer[Rating])): Int = a._1 - b._1 + } + Sorting.quickSort(groupedRatings)(ordering) + // Translate the user IDs to indices based on userIdToPos + ratingsForBlock(productBlock) = groupedRatings.map { case (p, rs) => + (rs.view.map(r => userIdToPos(r.user)).toArray, rs.view.map(_.rating).toArray) + } + } + InLinkBlock(userIds, ratingsForBlock) + } + + /** + * Make RDDs of InLinkBlocks and OutLinkBlocks given an RDD of (blockId, (u, p, r)) values for + * the users (or (blockId, (p, u, r)) for the products). We create these simultaneously to avoid + * having to shuffle the (blockId, (u, p, r)) RDD twice, or to cache it. + */ + private def makeLinkRDDs(numBlocks: Int, ratings: RDD[(Int, Rating)]) + : (RDD[(Int, InLinkBlock)], RDD[(Int, OutLinkBlock)]) = + { + val grouped = ratings.partitionBy(new HashPartitioner(numBlocks)) + val links = grouped.mapPartitionsWithIndex((blockId, elements) => { + val ratings = elements.map{_._2}.toArray + val inLinkBlock = makeInLinkBlock(numBlocks, ratings) + val outLinkBlock = makeOutLinkBlock(numBlocks, ratings) + Iterator.single((blockId, (inLinkBlock, outLinkBlock))) + }, true) + links.persist(StorageLevel.MEMORY_AND_DISK) + (links.mapValues(_._1), links.mapValues(_._2)) + } + + /** + * Make a random factor vector with the given random. + */ + private def randomFactor(rank: Int, rand: Random): Array[Double] = { + Array.fill(rank)(rand.nextDouble) + } + + /** + * Compute the user feature vectors given the current products (or vice-versa). This first joins + * the products with their out-links to generate a set of messages to each destination block + * (specifically, the features for the products that user block cares about), then groups these + * by destination and joins them with the in-link info to figure out how to update each user. + * It returns an RDD of new feature vectors for each user block. + */ + private def updateFeatures( + products: RDD[(Int, Array[Array[Double]])], + productOutLinks: RDD[(Int, OutLinkBlock)], + userInLinks: RDD[(Int, InLinkBlock)], + partitioner: Partitioner, + rank: Int, + lambda: Double) + : RDD[(Int, Array[Array[Double]])] = + { + val numBlocks = products.partitions.size + productOutLinks.join(products).flatMap { case (bid, (outLinkBlock, factors)) => + val toSend = Array.fill(numBlocks)(new ArrayBuffer[Array[Double]]) + for (p <- 0 until outLinkBlock.elementIds.length; userBlock <- 0 until numBlocks) { + if (outLinkBlock.shouldSend(p)(userBlock)) { + toSend(userBlock) += factors(p) + } + } + toSend.zipWithIndex.map{ case (buf, idx) => (idx, (bid, buf.toArray)) } + }.groupByKey(partitioner) + .join(userInLinks) + .mapValues{ case (messages, inLinkBlock) => updateBlock(messages, inLinkBlock, rank, lambda) } + } + + /** + * Compute the new feature vectors for a block of the users matrix given the list of factors + * it received from each product and its InLinkBlock. + */ + def updateBlock(messages: Seq[(Int, Array[Array[Double]])], inLinkBlock: InLinkBlock, + rank: Int, lambda: Double) + : Array[Array[Double]] = + { + // Sort the incoming block factor messages by block ID and make them an array + val blockFactors = messages.sortBy(_._1).map(_._2).toArray // Array[Array[Double]] + val numBlocks = blockFactors.length + val numUsers = inLinkBlock.elementIds.length + + // We'll sum up the XtXes using vectors that represent only the lower-triangular part, since + // the matrices are symmetric + val triangleSize = rank * (rank + 1) / 2 + val userXtX = Array.fill(numUsers)(DoubleMatrix.zeros(triangleSize)) + val userXy = Array.fill(numUsers)(DoubleMatrix.zeros(rank)) + + // Some temp variables to avoid memory allocation + val tempXtX = DoubleMatrix.zeros(triangleSize) + val fullXtX = DoubleMatrix.zeros(rank, rank) + + // Compute the XtX and Xy values for each user by adding products it rated in each product block + for (productBlock <- 0 until numBlocks) { + for (p <- 0 until blockFactors(productBlock).length) { + val x = new DoubleMatrix(blockFactors(productBlock)(p)) + fillXtX(x, tempXtX) + val (us, rs) = inLinkBlock.ratingsForBlock(productBlock)(p) + for (i <- 0 until us.length) { + userXtX(us(i)).addi(tempXtX) + SimpleBlas.axpy(rs(i), x, userXy(us(i))) + } + } + } + + // Solve the least-squares problem for each user and return the new feature vectors + userXtX.zipWithIndex.map{ case (triangularXtX, index) => + // Compute the full XtX matrix from the lower-triangular part we got above + fillFullMatrix(triangularXtX, fullXtX) + // Add regularization + (0 until rank).foreach(i => fullXtX.data(i*rank + i) += lambda) + // Solve the resulting matrix, which is symmetric and positive-definite + Solve.solvePositive(fullXtX, userXy(index)).data + } + } + + /** + * Set xtxDest to the lower-triangular part of x transpose * x. For efficiency in summing + * these matrices, we store xtxDest as only rank * (rank+1) / 2 values, namely the values + * at (0,0), (1,0), (1,1), (2,0), (2,1), (2,2), etc in that order. + */ + private def fillXtX(x: DoubleMatrix, xtxDest: DoubleMatrix) { + var i = 0 + var pos = 0 + while (i < x.length) { + var j = 0 + while (j <= i) { + xtxDest.data(pos) = x.data(i) * x.data(j) + pos += 1 + j += 1 + } + i += 1 + } + } + + /** + * Given a triangular matrix in the order of fillXtX above, compute the full symmetric square + * matrix that it represents, storing it into destMatrix. + */ + private def fillFullMatrix(triangularMatrix: DoubleMatrix, destMatrix: DoubleMatrix) { + val rank = destMatrix.rows + var i = 0 + var pos = 0 + while (i < rank) { + var j = 0 + while (j <= i) { + destMatrix.data(i*rank + j) = triangularMatrix.data(pos) + destMatrix.data(j*rank + i) = triangularMatrix.data(pos) + pos += 1 + j += 1 + } + i += 1 + } + } +} + + +/** + * Top-level methods for calling Alternating Least Squares (ALS) matrix factorizaton. + */ +object ALS { + /** + * Train a matrix factorization model given an RDD of ratings given by users to some products, + * in the form of (userID, productID, rating) pairs. We approximate the ratings matrix as the + * product of two lower-rank matrices of a given rank (number of features). To solve for these + * features, we run a given number of iterations of ALS. This is done using a level of + * parallelism given by `blocks`. + * + * @param ratings RDD of (userID, productID, rating) pairs + * @param rank number of features to use + * @param iterations number of iterations of ALS (recommended: 10-20) + * @param lambda regularization factor (recommended: 0.01) + * @param blocks level of parallelism to split computation into + */ + def train( + ratings: RDD[Rating], + rank: Int, + iterations: Int, + lambda: Double, + blocks: Int) + : MatrixFactorizationModel = + { + new ALS(blocks, rank, iterations, lambda).run(ratings) + } + + /** + * Train a matrix factorization model given an RDD of ratings given by users to some products, + * in the form of (userID, productID, rating) pairs. We approximate the ratings matrix as the + * product of two lower-rank matrices of a given rank (number of features). To solve for these + * features, we run a given number of iterations of ALS. The level of parallelism is determined + * automatically based on the number of partitions in `ratings`. + * + * @param ratings RDD of (userID, productID, rating) pairs + * @param rank number of features to use + * @param iterations number of iterations of ALS (recommended: 10-20) + * @param lambda regularization factor (recommended: 0.01) + */ + def train(ratings: RDD[Rating], rank: Int, iterations: Int, lambda: Double) + : MatrixFactorizationModel = + { + train(ratings, rank, iterations, lambda, -1) + } + + /** + * Train a matrix factorization model given an RDD of ratings given by users to some products, + * in the form of (userID, productID, rating) pairs. We approximate the ratings matrix as the + * product of two lower-rank matrices of a given rank (number of features). To solve for these + * features, we run a given number of iterations of ALS. The level of parallelism is determined + * automatically based on the number of partitions in `ratings`. + * + * @param ratings RDD of (userID, productID, rating) pairs + * @param rank number of features to use + * @param iterations number of iterations of ALS (recommended: 10-20) + */ + def train(ratings: RDD[Rating], rank: Int, iterations: Int) + : MatrixFactorizationModel = + { + train(ratings, rank, iterations, 0.01, -1) + } + + private class ALSRegistrator extends KryoRegistrator { + override def registerClasses(kryo: Kryo) { + kryo.register(classOf[Rating]) + } + } + + def main(args: Array[String]) { + if (args.length != 5 && args.length != 6) { + println("Usage: ALS []") + System.exit(1) + } + val (master, ratingsFile, rank, iters, outputDir) = + (args(0), args(1), args(2).toInt, args(3).toInt, args(4)) + val blocks = if (args.length == 6) args(5).toInt else -1 + System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer") + System.setProperty("spark.kryo.registrator", classOf[ALSRegistrator].getName) + System.setProperty("spark.kryo.referenceTracking", "false") + System.setProperty("spark.kryoserializer.buffer.mb", "8") + System.setProperty("spark.locality.wait", "10000") + val sc = new SparkContext(master, "ALS") + val ratings = sc.textFile(ratingsFile).map { line => + val fields = line.split(',') + Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble) + } + val model = ALS.train(ratings, rank, iters, 0.01, blocks) + model.userFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") } + .saveAsTextFile(outputDir + "/userFeatures") + model.productFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") } + .saveAsTextFile(outputDir + "/productFeatures") + println("Final user/product features written to " + outputDir) + System.exit(0) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala new file mode 100644 index 0000000000..ae9fe48aec --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -0,0 +1,49 @@ +/* + * 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.recommendation + +import org.apache.spark.RDD +import org.apache.spark.SparkContext._ + +import org.jblas._ + +/** + * Model representing the result of matrix factorization. + * + * @param rank Rank for the features in this model. + * @param userFeatures RDD of tuples where each tuple represents the userId and + * the features computed for this user. + * @param productFeatures RDD of tuples where each tuple represents the productId + * and the features computed for this product. + */ +class MatrixFactorizationModel( + val rank: Int, + val userFeatures: RDD[(Int, Array[Double])], + val productFeatures: RDD[(Int, Array[Double])]) + extends Serializable +{ + /** Predict the rating of one user for one product. */ + def predict(user: Int, product: Int): Double = { + val userVector = new DoubleMatrix(userFeatures.lookup(user).head) + val productVector = new DoubleMatrix(productFeatures.lookup(product).head) + userVector.dot(productVector) + } + + // TODO: Figure out what good bulk prediction methods would look like. + // Probably want a way to get the top users for a product or vice-versa. +} 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 new file mode 100644 index 0000000000..06015110ac --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -0,0 +1,159 @@ +/* + * 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.regression + +import org.apache.spark.{Logging, RDD, SparkException} +import org.apache.spark.mllib.optimization._ + +import org.jblas.DoubleMatrix + +/** + * GeneralizedLinearModel (GLM) represents a model trained using + * GeneralizedLinearAlgorithm. GLMs consist of a weight vector and + * an intercept. + * + * @param weights Weights computed for every feature. + * @param intercept Intercept computed for this model. + */ +abstract class GeneralizedLinearModel(val weights: Array[Double], 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. + * + * @param dataMatrix Row vector containing the features for this data point + * @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 + + /** + * Predict values for the given data set using the model trained. + * + * @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] = { + // A small optimization to avoid serializing the entire model. Only the weightsMatrix + // and intercept is needed. + val localWeights = weightsMatrix + val localIntercept = intercept + + testData.map { x => + val dataMatrix = new DoubleMatrix(1, x.length, x:_*) + predictPoint(dataMatrix, localWeights, localIntercept) + } + } + + /** + * Predict values for a single data point using the model trained. + * + * @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) + } +} + +/** + * GeneralizedLinearAlgorithm implements methods to train a Genearalized Linear Model (GLM). + * This class should be extended with an Optimizer to create a new GLM. + */ +abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] + extends Logging with Serializable { + + protected val validators: Seq[RDD[LabeledPoint] => Boolean] = List() + + val optimizer: Optimizer + + protected var addIntercept: Boolean = true + + protected var validateData: Boolean = true + + /** + * Create a model given the weights and intercept + */ + protected def createModel(weights: Array[Double], intercept: Double): M + + /** + * Set if the algorithm should add an intercept. Default true. + */ + def setIntercept(addIntercept: Boolean): this.type = { + this.addIntercept = addIntercept + this + } + + /** + * Set if the algorithm should validate data before training. Default true. + */ + def setValidateData(validateData: Boolean): this.type = { + this.validateData = validateData + this + } + + /** + * 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 = Array.fill(nfeatures)(1.0) + run(input, initialWeights) + } + + /** + * 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 = { + + // Check the data properties before running the optimizer + if (validateData && !validators.forall(func => func(input))) { + throw new SparkException("Input validation failed.") + } + + // Add a extra variable consisting of all 1.0's for the intercept. + val data = if (addIntercept) { + input.map(labeledPoint => (labeledPoint.label, Array(1.0, labeledPoint.features:_*))) + } else { + input.map(labeledPoint => (labeledPoint.label, labeledPoint.features)) + } + + val initialWeightsWithIntercept = if (addIntercept) { + Array(1.0, initialWeights:_*) + } else { + initialWeights + } + + val weights = optimizer.optimize(data, initialWeightsWithIntercept) + val intercept = weights(0) + val weightsScaled = weights.tail + + val model = createModel(weightsScaled, 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 new file mode 100644 index 0000000000..63240e24dc --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala @@ -0,0 +1,26 @@ +/* + * 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.regression + +/** + * 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(val label: Double, val features: Array[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 new file mode 100644 index 0000000000..df3beb1959 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -0,0 +1,210 @@ +/* + * 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.regression + +import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.util.MLUtils + +import org.jblas.DoubleMatrix + +/** + * Regression model trained using Lasso. + * + * @param weights Weights computed for every feature. + * @param intercept Intercept computed for this model. + */ +class LassoModel( + 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) = { + dataMatrix.dot(weightMatrix) + intercept + } +} + +/** + * Train a regression model with L1-regularization using Stochastic Gradient Descent. + */ +class LassoWithSGD private ( + var stepSize: Double, + var numIterations: Int, + var regParam: Double, + var miniBatchFraction: Double) + extends GeneralizedLinearAlgorithm[LassoModel] + with Serializable { + + val gradient = new SquaredGradient() + val updater = new L1Updater() + @transient val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) + .setNumIterations(numIterations) + .setRegParam(regParam) + .setMiniBatchFraction(miniBatchFraction) + + // We don't want to penalize the intercept, so set this to false. + setIntercept(false) + + var yMean = 0.0 + var xColMean: DoubleMatrix = _ + var xColSd: DoubleMatrix = _ + + /** + * Construct a Lasso object with default parameters + */ + 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):_*) + val weightsScaled = weightsMat.div(xColSd) + val interceptScaled = yMean - (weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0)) + + new LassoModel(weightsScaled.data, interceptScaled) + } + + override def run( + input: RDD[LabeledPoint], + initialWeights: Array[Double]) + : LassoModel = + { + val nfeatures: Int = input.first.features.length + 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 + + 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) + } + + super.run(normalizedData, initialWeights) + } +} + +/** + * Top-level methods for calling Lasso. + */ +object LassoWithSGD { + + /** + * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in + * gradient descent are initialized using the initial weights provided. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + * @param initialWeights Initial set of weights to be used. Array should be equal in size to + * the number of features in the data. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, + initialWeights: Array[Double]) + : LassoModel = + { + new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input, + initialWeights) + } + + /** + * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double) + : LassoModel = + { + new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input) + } + + /** + * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. We use the entire data set to + * update the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param stepSize Step size to be used for each iteration of Gradient Descent. + * @param regParam Regularization parameter. + * @param numIterations Number of iterations of gradient descent to run. + * @return a LassoModel which has the weights and offset from training. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + regParam: Double) + : LassoModel = + { + train(input, numIterations, stepSize, regParam, 1.0) + } + + /** + * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using a step size of 1.0. We use the entire data set to + * update the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @return a LassoModel which has the weights and offset from training. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int) + : LassoModel = + { + train(input, numIterations, 1.0, 1.0, 1.0) + } + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: Lasso ") + System.exit(1) + } + 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) + + sc.stop() + } +} 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 new file mode 100644 index 0000000000..71f968471c --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -0,0 +1,167 @@ +/* + * 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.regression + +import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.util.MLUtils + +import org.jblas.DoubleMatrix + +/** + * Regression model trained using LinearRegression. + * + * @param weights Weights computed for every feature. + * @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) = { + dataMatrix.dot(weightMatrix) + intercept + } +} + +/** + * Train a regression model with no regularization using Stochastic Gradient Descent. + */ +class LinearRegressionWithSGD private ( + var stepSize: Double, + var numIterations: Int, + var miniBatchFraction: Double) + extends GeneralizedLinearAlgorithm[LinearRegressionModel] + with Serializable { + + val gradient = new SquaredGradient() + val updater = new SimpleUpdater() + val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) + .setNumIterations(numIterations) + .setMiniBatchFraction(miniBatchFraction) + + /** + * Construct a LinearRegression object with default parameters + */ + def this() = this(1.0, 100, 1.0) + + def createModel(weights: Array[Double], intercept: Double) = { + new LinearRegressionModel(weights, intercept) + } +} + +/** + * Top-level methods for calling LinearRegression. + */ +object LinearRegressionWithSGD { + + /** + * Train a Linear Regression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in + * gradient descent are initialized using the initial weights provided. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param miniBatchFraction Fraction of data to be used per iteration. + * @param initialWeights Initial set of weights to be used. Array should be equal in size to + * the number of features in the data. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double, + initialWeights: Array[Double]) + : LinearRegressionModel = + { + new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction).run(input, + initialWeights) + } + + /** + * Train a LinearRegression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param miniBatchFraction Fraction of data to be used per iteration. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double) + : LinearRegressionModel = + { + new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction).run(input) + } + + /** + * Train a LinearRegression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. We use the entire data set to + * update the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param stepSize Step size to be used for each iteration of Gradient Descent. + * @param numIterations Number of iterations of gradient descent to run. + * @return a LinearRegressionModel which has the weights and offset from training. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double) + : LinearRegressionModel = + { + train(input, numIterations, stepSize, 1.0) + } + + /** + * Train a LinearRegression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using a step size of 1.0. We use the entire data set to + * update the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @return a LinearRegressionModel which has the weights and offset from training. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int) + : LinearRegressionModel = + { + train(input, numIterations, 1.0, 1.0) + } + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: LinearRegression ") + System.exit(1) + } + 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) + + 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 new file mode 100644 index 0000000000..8dd325efc0 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala @@ -0,0 +1,38 @@ +/* + * 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.regression + +import org.apache.spark.RDD + +trait RegressionModel extends Serializable { + /** + * Predict values for the given data set using the model trained. + * + * @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] + + /** + * Predict values for a single data point using the model trained. + * + * @param testData array representing a single data point + * @return Double prediction from the trained model + */ + def predict(testData: Array[Double]): Double +} 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 new file mode 100644 index 0000000000..228ab9e4e8 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -0,0 +1,213 @@ +/* + * 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.regression + +import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.util.MLUtils + +import org.jblas.DoubleMatrix + +/** + * Regression model trained using RidgeRegression. + * + * @param weights Weights computed for every feature. + * @param intercept Intercept computed for this model. + */ +class RidgeRegressionModel( + 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) = { + dataMatrix.dot(weightMatrix) + intercept + } +} + +/** + * Train a regression model with L2-regularization using Stochastic Gradient Descent. + */ +class RidgeRegressionWithSGD private ( + var stepSize: Double, + var numIterations: Int, + var regParam: Double, + var miniBatchFraction: Double) + extends GeneralizedLinearAlgorithm[RidgeRegressionModel] + with Serializable { + + val gradient = new SquaredGradient() + val updater = new SquaredL2Updater() + + @transient val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) + .setNumIterations(numIterations) + .setRegParam(regParam) + .setMiniBatchFraction(miniBatchFraction) + + // We don't want to penalize the intercept in RidgeRegression, so set this to false. + setIntercept(false) + + var yMean = 0.0 + var xColMean: DoubleMatrix = _ + var xColSd: DoubleMatrix = _ + + /** + * Construct a RidgeRegression object with default parameters + */ + 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):_*) + val weightsScaled = weightsMat.div(xColSd) + val interceptScaled = yMean - (weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0)) + + new RidgeRegressionModel(weightsScaled.data, interceptScaled) + } + + override def run( + input: RDD[LabeledPoint], + initialWeights: Array[Double]) + : RidgeRegressionModel = + { + val nfeatures: Int = input.first.features.length + 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 + + 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) + } + + super.run(normalizedData, initialWeights) + } +} + +/** + * Top-level methods for calling RidgeRegression. + */ +object RidgeRegressionWithSGD { + + /** + * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in + * gradient descent are initialized using the initial weights provided. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + * @param initialWeights Initial set of weights to be used. Array should be equal in size to + * the number of features in the data. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, + initialWeights: Array[Double]) + : RidgeRegressionModel = + { + new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run( + input, initialWeights) + } + + /** + * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double) + : RidgeRegressionModel = + { + new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input) + } + + /** + * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. We use the entire data set to + * update the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param stepSize Step size to be used for each iteration of Gradient Descent. + * @param regParam Regularization parameter. + * @param numIterations Number of iterations of gradient descent to run. + * @return a RidgeRegressionModel which has the weights and offset from training. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + regParam: Double) + : RidgeRegressionModel = + { + train(input, numIterations, stepSize, regParam, 1.0) + } + + /** + * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using a step size of 1.0. We use the entire data set to + * update the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @return a RidgeRegressionModel which has the weights and offset from training. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int) + : RidgeRegressionModel = + { + train(input, numIterations, 1.0, 1.0, 1.0) + } + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: RidgeRegression " + + " ") + System.exit(1) + } + val sc = new SparkContext(args(0), "RidgeRegression") + val data = MLUtils.loadLabeledData(sc, args(1)) + val model = RidgeRegressionWithSGD.train(data, args(4).toInt, args(2).toDouble, + args(3).toDouble) + + sc.stop() + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala new file mode 100644 index 0000000000..7fd4623071 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.util + +import org.apache.spark.{RDD, Logging} +import org.apache.spark.mllib.regression.LabeledPoint + +/** + * A collection of methods used to validate data before applying ML algorithms. + */ +object DataValidators extends Logging { + + /** + * Function to check if labels used for classification are either zero or one. + * + * @param data - input data set that needs to be checked + * + * @return True if labels are all zero or one, false otherwise. + */ + val classificationLabels: RDD[LabeledPoint] => Boolean = { data => + val numInvalid = data.filter(x => x.label != 1.0 && x.label != 0.0).count() + if (numInvalid != 0) { + logError("Classification labels should be 0 or 1. Found " + numInvalid + " invalid labels") + } + numInvalid == 0 + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala new file mode 100644 index 0000000000..6500d47183 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.util + +import scala.util.Random + +import org.apache.spark.{RDD, SparkContext} + +/** + * Generate test data for KMeans. This class first chooses k cluster centers + * from a d-dimensional Gaussian distribution scaled by factor r and then creates a Gaussian + * cluster with scale 1 around each center. + */ + +object KMeansDataGenerator { + + /** + * Generate an RDD containing test data for KMeans. + * + * @param sc SparkContext to use for creating the RDD + * @param numPoints Number of points that will be contained in the RDD + * @param k Number of clusters + * @param d Number of dimensions + * @param r Scaling factor for the distribution of the initial centers + * @param numPartitions Number of partitions of the generated RDD; default 2 + */ + def generateKMeansRDD( + sc: SparkContext, + numPoints: Int, + k: Int, + d: Int, + r: Double, + numPartitions: Int = 2) + : RDD[Array[Double]] = + { + // First, generate some centers + val rand = new Random(42) + val centers = Array.fill(k)(Array.fill(d)(rand.nextGaussian() * r)) + // Then generate points around each center + sc.parallelize(0 until numPoints, numPartitions).map { idx => + val center = centers(idx % k) + val rand2 = new Random(42 + idx) + Array.tabulate(d)(i => center(i) + rand2.nextGaussian()) + } + } + + def main(args: Array[String]) { + if (args.length < 6) { + println("Usage: KMeansGenerator " + + " []") + System.exit(1) + } + + val sparkMaster = args(0) + val outputPath = args(1) + val numPoints = args(2).toInt + val k = args(3).toInt + val d = args(4).toInt + val r = args(5).toDouble + val parts = if (args.length >= 7) args(6).toInt else 2 + + val sc = new SparkContext(sparkMaster, "KMeansDataGenerator") + val data = generateKMeansRDD(sc, numPoints, k, d, r, parts) + data.map(_.mkString(" ")).saveAsTextFile(outputPath) + + System.exit(0) + } +} + 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 new file mode 100644 index 0000000000..4c49d484b4 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.util + +import scala.collection.JavaConversions._ +import scala.util.Random + +import org.jblas.DoubleMatrix + +import org.apache.spark.{RDD, SparkContext} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.LabeledPoint + +/** + * Generate sample data used for Linear Data. This class generates + * uniformly random values for every feature and adds Gaussian noise with mean `eps` to the + * response variable `Y`. + */ +object LinearDataGenerator { + + /** + * Return a Java List of synthetic data randomly generated according to a multi + * collinear model. + * @param intercept Data intercept + * @param weights Weights to be applied. + * @param nPoints Number of points in sample. + * @param seed Random seed + * @return Java List of input. + */ + def generateLinearInputAsList( + intercept: Double, + weights: Array[Double], + nPoints: Int, + seed: Int, + eps: Double): java.util.List[LabeledPoint] = { + seqAsJavaList(generateLinearInput(intercept, weights, nPoints, seed, eps)) + } + + /** + * + * @param intercept Data intercept + * @param weights Weights to be applied. + * @param nPoints Number of points in sample. + * @param seed Random seed + * @param eps Epsilon scaling factor. + * @return + */ + def generateLinearInput( + intercept: Double, + weights: Array[Double], + nPoints: Int, + seed: Int, + eps: Double = 0.1): Seq[LabeledPoint] = { + + val rnd = new Random(seed) + val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) + val x = Array.fill[Array[Double]](nPoints)( + Array.fill[Double](weights.length)(2 * rnd.nextDouble - 1.0)) + 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)) + } + + /** + * Generate an RDD containing sample data for Linear Regression models - including Ridge, Lasso, + * and uregularized variants. + * + * @param sc SparkContext to be used for generating the RDD. + * @param nexamples Number of examples that will be contained in the RDD. + * @param nfeatures Number of features to generate for each example. + * @param eps Epsilon factor by which examples are scaled. + * @param weights Weights associated with the first weights.length features. + * @param nparts Number of partitions in the RDD. Default value is 2. + * + * @return RDD of LabeledPoint containing sample data. + */ + def generateLinearRDD( + sc: SparkContext, + nexamples: Int, + nfeatures: Int, + eps: Double, + nparts: Int = 2, + intercept: Double = 0.0) : RDD[LabeledPoint] = { + org.jblas.util.Random.seed(42) + // Random values distributed uniformly in [-0.5, 0.5] + val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) + + val data: RDD[LabeledPoint] = sc.parallelize(0 until nparts, nparts).flatMap { p => + val seed = 42 + p + val examplesInPartition = nexamples / nparts + generateLinearInput(intercept, w.toArray, examplesInPartition, seed, eps) + } + data + } + + def main(args: Array[String]) { + if (args.length < 2) { + println("Usage: LinearDataGenerator " + + " [num_examples] [num_features] [num_partitions]") + System.exit(1) + } + + val sparkMaster: String = args(0) + val outputPath: String = args(1) + val nexamples: Int = if (args.length > 2) args(2).toInt else 1000 + val nfeatures: Int = if (args.length > 3) args(3).toInt else 100 + val parts: Int = if (args.length > 4) args(4).toInt else 2 + val eps = 10 + + val sc = new SparkContext(sparkMaster, "LinearDataGenerator") + val data = generateLinearRDD(sc, nexamples, nfeatures, eps, nparts = parts) + + MLUtils.saveLabeledData(data, outputPath) + sc.stop() + } +} 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 new file mode 100644 index 0000000000..f553298fc5 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.util + +import scala.util.Random + +import org.apache.spark.{RDD, SparkContext} +import org.apache.spark.mllib.regression.LabeledPoint + +/** + * Generate test data for LogisticRegression. This class chooses positive labels + * with probability `probOne` and scales features for positive examples by `eps`. + */ + +object LogisticRegressionDataGenerator { + + /** + * Generate an RDD containing test data for LogisticRegression. + * + * @param sc SparkContext to use for creating the RDD. + * @param nexamples Number of examples that will be contained in the RDD. + * @param nfeatures Number of features to generate for each example. + * @param eps Epsilon factor by which positive examples are scaled. + * @param nparts Number of partitions of the generated RDD. Default value is 2. + * @param probOne Probability that a label is 1 (and not 0). Default value is 0.5. + */ + def generateLogisticRDD( + sc: SparkContext, + nexamples: Int, + nfeatures: Int, + eps: Double, + nparts: Int = 2, + probOne: Double = 0.5): RDD[LabeledPoint] = { + val data = sc.parallelize(0 until nexamples, nparts).map { idx => + val rnd = new Random(42 + idx) + + val y = if (idx % 2 == 0) 0.0 else 1.0 + val x = Array.fill[Double](nfeatures) { + rnd.nextGaussian() + (y * eps) + } + LabeledPoint(y, x) + } + data + } + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: LogisticRegressionGenerator " + + " ") + System.exit(1) + } + + val sparkMaster: String = args(0) + val outputPath: String = args(1) + val nexamples: Int = if (args.length > 2) args(2).toInt else 1000 + val nfeatures: Int = if (args.length > 3) args(3).toInt else 2 + val parts: Int = if (args.length > 4) args(4).toInt else 2 + val eps = 3 + + val sc = new SparkContext(sparkMaster, "LogisticRegressionDataGenerator") + val data = generateLogisticRDD(sc, nexamples, nfeatures, eps, parts) + + MLUtils.saveLabeledData(data, outputPath) + sc.stop() + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala new file mode 100644 index 0000000000..7eb69ae81c --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -0,0 +1,113 @@ +/* + * 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.recommendation + +import scala.util.Random + +import org.jblas.DoubleMatrix + +import org.apache.spark.{RDD, SparkContext} +import org.apache.spark.mllib.util.MLUtils + +/** +* Generate RDD(s) containing data for Matrix Factorization. +* +* This method samples training entries according to the oversampling factor +* 'trainSampFact', which is a multiplicative factor of the number of +* degrees of freedom of the matrix: rank*(m+n-rank). +* +* It optionally samples entries for a testing matrix using +* 'testSampFact', the percentage of the number of training entries +* to use for testing. +* +* This method takes the following inputs: +* sparkMaster (String) The master URL. +* outputPath (String) Directory to save output. +* m (Int) Number of rows in data matrix. +* n (Int) Number of columns in data matrix. +* rank (Int) Underlying rank of data matrix. +* trainSampFact (Double) Oversampling factor. +* noise (Boolean) Whether to add gaussian noise to training data. +* sigma (Double) Standard deviation of added gaussian noise. +* test (Boolean) Whether to create testing RDD. +* testSampFact (Double) Percentage of training data to use as test data. +*/ + +object MFDataGenerator{ + + def main(args: Array[String]) { + if (args.length < 2) { + println("Usage: MFDataGenerator " + + " [m] [n] [rank] [trainSampFact] [noise] [sigma] [test] [testSampFact]") + System.exit(1) + } + + val sparkMaster: String = args(0) + val outputPath: String = args(1) + val m: Int = if (args.length > 2) args(2).toInt else 100 + val n: Int = if (args.length > 3) args(3).toInt else 100 + val rank: Int = if (args.length > 4) args(4).toInt else 10 + val trainSampFact: Double = if (args.length > 5) args(5).toDouble else 1.0 + val noise: Boolean = if (args.length > 6) args(6).toBoolean else false + val sigma: Double = if (args.length > 7) args(7).toDouble else 0.1 + val test: Boolean = if (args.length > 8) args(8).toBoolean else false + val testSampFact: Double = if (args.length > 9) args(9).toDouble else 0.1 + + val sc = new SparkContext(sparkMaster, "MFDataGenerator") + + val A = DoubleMatrix.randn(m, rank) + val B = DoubleMatrix.randn(rank, n) + val z = 1 / (scala.math.sqrt(scala.math.sqrt(rank))) + A.mmuli(z) + B.mmuli(z) + val fullData = A.mmul(B) + + val df = rank * (m + n - rank) + val sampSize = scala.math.min(scala.math.round(trainSampFact * df), + scala.math.round(.99 * m * n)).toInt + val rand = new Random() + val mn = m * n + val shuffled = rand.shuffle(1 to mn toIterable) + + val omega = shuffled.slice(0, sampSize) + val ordered = omega.sortWith(_ < _).toArray + val trainData: RDD[(Int, Int, Double)] = sc.parallelize(ordered) + .map(x => (fullData.indexRows(x - 1), fullData.indexColumns(x - 1), fullData.get(x - 1))) + + // optionally add gaussian noise + if (noise) { + trainData.map(x => (x._1, x._2, x._3 + rand.nextGaussian * sigma)) + } + + trainData.map(x => x._1 + "," + x._2 + "," + x._3).saveAsTextFile(outputPath) + + // optionally generate testing data + if (test) { + val testSampSize = scala.math + .min(scala.math.round(sampSize * testSampFact),scala.math.round(mn - sampSize)).toInt + val testOmega = shuffled.slice(sampSize, sampSize + testSampSize) + val testOrdered = testOmega.sortWith(_ < _).toArray + val testData: RDD[(Int, Int, Double)] = sc.parallelize(testOrdered) + .map(x => (fullData.indexRows(x - 1), fullData.indexColumns(x - 1), fullData.get(x - 1))) + testData.map(x => x._1 + "," + x._2 + "," + x._3).saveAsTextFile(outputPath) + } + + 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 new file mode 100644 index 0000000000..0aeafbe23c --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.util + +import org.apache.spark.{RDD, SparkContext} +import org.apache.spark.SparkContext._ + +import org.jblas.DoubleMatrix +import org.apache.spark.mllib.regression.LabeledPoint + +/** + * Helper methods to load, save and pre-process data used in ML Lib. + */ +object MLUtils { + + /** + * Load labeled data from a file. The data format used here is + * , ... + * where , are feature values in Double and is the corresponding label as Double. + * + * @param sc SparkContext + * @param dir Directory to the input data files. + * @return An RDD of LabeledPoint. Each labeled point has two elements: the first element is + * the label, and the second element represents the feature values (an array of Double). + */ + def loadLabeledData(sc: SparkContext, dir: String): RDD[LabeledPoint] = { + sc.textFile(dir).map { line => + val parts = line.split(',') + val label = parts(0).toDouble + val features = parts(1).trim().split(' ').map(_.toDouble) + LabeledPoint(label, features) + } + } + + /** + * Save labeled data to a file. The data format used here is + * , ... + * where , are feature values in Double and is the corresponding label as Double. + * + * @param data An RDD of LabeledPoints containing data to be saved. + * @param dir Directory to save the data. + */ + def saveLabeledData(data: RDD[LabeledPoint], dir: String) { + val dataStr = data.map(x => x.label + "," + x.features.mkString(" ")) + dataStr.saveAsTextFile(dir) + } + + /** + * 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 + * + * @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 + + // 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 xColSumsMap = xColSumSq.collectAsMap() + + val xColMean = DoubleMatrix.zeros(nfeatures, 1) + val xColSd = DoubleMatrix.zeros(nfeatures, 1) + + // 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 + } + + (yMean, xColMean, xColSd) + } + + /** + * Return the squared Euclidean distance between two vectors. + */ + def squaredDistance(v1: Array[Double], v2: Array[Double]): Double = { + if (v1.length != v2.length) { + throw new IllegalArgumentException("Vector sizes don't match") + } + var i = 0 + var sum = 0.0 + while (i < v1.length) { + sum += (v1(i) - v2(i)) * (v1(i) - v2(i)) + i += 1 + } + sum + } +} 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 new file mode 100644 index 0000000000..d3f191b05b --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -0,0 +1,50 @@ +package org.apache.spark.mllib.util + +import scala.util.Random + +import org.apache.spark.{RDD, SparkContext} + +import org.jblas.DoubleMatrix +import org.apache.spark.mllib.regression.LabeledPoint + +/** + * Generate sample data used for SVM. This class generates uniform random values + * for the features and adds Gaussian noise with weight 0.1 to generate labels. + */ +object SVMDataGenerator { + + def main(args: Array[String]) { + if (args.length < 2) { + println("Usage: SVMGenerator " + + " [num_examples] [num_features] [num_partitions]") + System.exit(1) + } + + val sparkMaster: String = args(0) + val outputPath: String = args(1) + val nexamples: Int = if (args.length > 2) args(2).toInt else 1000 + val nfeatures: Int = if (args.length > 3) args(3).toInt else 2 + val parts: Int = if (args.length > 4) args(4).toInt else 2 + + val sc = new SparkContext(sparkMaster, "SVMGenerator") + + val globalRnd = new Random(94720) + val trueWeights = new DoubleMatrix(1, nfeatures + 1, + Array.fill[Double](nfeatures + 1)(globalRnd.nextGaussian()):_*) + + val data: RDD[LabeledPoint] = sc.parallelize(0 until nexamples, parts).map { idx => + val rnd = new Random(42 + idx) + + val x = Array.fill[Double](nfeatures) { + rnd.nextDouble() * 2.0 - 1.0 + } + 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) + } + + MLUtils.saveLabeledData(data, outputPath) + + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/mllib/classification/ClassificationModel.scala b/mllib/src/main/scala/spark/mllib/classification/ClassificationModel.scala deleted file mode 100644 index 70fae8c15a..0000000000 --- a/mllib/src/main/scala/spark/mllib/classification/ClassificationModel.scala +++ /dev/null @@ -1,21 +0,0 @@ -package spark.mllib.classification - -import spark.RDD - -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 - */ - def predict(testData: RDD[Array[Double]]): RDD[Double] - - /** - * 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 - */ - def predict(testData: Array[Double]): Double -} diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala deleted file mode 100644 index 482e4a6745..0000000000 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ /dev/null @@ -1,188 +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 spark.mllib.classification - -import scala.math.round - -import spark.{Logging, RDD, SparkContext} -import spark.mllib.optimization._ -import spark.mllib.regression._ -import spark.mllib.util.MLUtils -import spark.mllib.util.DataValidators - -import org.jblas.DoubleMatrix - -/** - * Classification model trained using Logistic Regression. - * - * @param weights Weights computed for every feature. - * @param intercept Intercept computed for this model. - */ -class LogisticRegressionModel( - override val weights: Array[Double], - override val intercept: Double) - extends GeneralizedLinearModel(weights, intercept) - with ClassificationModel with Serializable { - - override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, - intercept: Double) = { - val margin = dataMatrix.mmul(weightMatrix).get(0) + intercept - round(1.0/ (1.0 + math.exp(margin * -1))) - } -} - -/** - * Train a classification model for Logistic Regression using Stochastic Gradient Descent. - * NOTE: Labels used in Logistic Regression should be {0, 1} - */ -class LogisticRegressionWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var regParam: Double, - var miniBatchFraction: Double) - 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) - override val validators = List(DataValidators.classificationLabels) - - /** - * Construct a LogisticRegression object with default parameters - */ - def this() = this(1.0, 100, 0.0, 1.0) - - def createModel(weights: Array[Double], intercept: Double) = { - new LogisticRegressionModel(weights, intercept) - } -} - -/** - * Top-level methods for calling Logistic Regression. - * NOTE: Labels used in Logistic Regression should be {0, 1} - */ -object LogisticRegressionWithSGD { - // NOTE(shivaram): We use multiple train methods instead of default arguments to support - // Java programs. - - /** - * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed - * number of iterations of gradient descent using the specified step size. Each iteration uses - * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in - * gradient descent are initialized using the initial weights provided. - * NOTE: Labels used in Logistic Regression should be {0, 1} - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @param stepSize Step size to be used for each iteration of gradient descent. - * @param miniBatchFraction Fraction of data to be used per iteration. - * @param initialWeights Initial set of weights to be used. Array should be equal in size to - * the number of features in the data. - */ - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - miniBatchFraction: Double, - initialWeights: Array[Double]) - : LogisticRegressionModel = - { - new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction).run( - input, initialWeights) - } - - /** - * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed - * number of iterations of gradient descent using the specified step size. Each iteration uses - * `miniBatchFraction` fraction of the data to calculate the gradient. - * NOTE: Labels used in Logistic Regression should be {0, 1} - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @param stepSize Step size to be used for each iteration of gradient descent. - - * @param miniBatchFraction Fraction of data to be used per iteration. - */ - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - miniBatchFraction: Double) - : LogisticRegressionModel = - { - new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction).run( - input) - } - - /** - * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed - * number of iterations of gradient descent using the specified step size. We use the entire data - * set to update the gradient in each iteration. - * NOTE: Labels used in Logistic Regression should be {0, 1} - * - * @param input RDD of (label, array of features) pairs. - * @param stepSize Step size to be used for each iteration of Gradient Descent. - - * @param numIterations Number of iterations of gradient descent to run. - * @return a LogisticRegressionModel which has the weights and offset from training. - */ - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double) - : LogisticRegressionModel = - { - train(input, numIterations, stepSize, 1.0) - } - - /** - * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed - * number of iterations of gradient descent using a step size of 1.0. We use the entire data set - * to update the gradient in each iteration. - * NOTE: Labels used in Logistic Regression should be {0, 1} - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @return a LogisticRegressionModel which has the weights and offset from training. - */ - def train( - input: RDD[LabeledPoint], - numIterations: Int) - : LogisticRegressionModel = - { - train(input, numIterations, 1.0, 1.0) - } - - def main(args: Array[String]) { - if (args.length != 4) { - println("Usage: LogisticRegression " + - "") - System.exit(1) - } - 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) - - sc.stop() - } -} diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala deleted file mode 100644 index 69393cd7b0..0000000000 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ /dev/null @@ -1,187 +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 spark.mllib.classification - -import scala.math.signum - -import spark.{Logging, RDD, SparkContext} -import spark.mllib.optimization._ -import spark.mllib.regression._ -import spark.mllib.util.MLUtils -import spark.mllib.util.DataValidators - -import org.jblas.DoubleMatrix - -/** - * Model built using SVM. - * - * @param weights Weights computed for every feature. - * @param intercept Intercept computed for this model. - */ -class SVMModel( - override val weights: Array[Double], - override val intercept: Double) - extends GeneralizedLinearModel(weights, intercept) - with ClassificationModel with Serializable { - - override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, - intercept: Double) = { - val margin = dataMatrix.dot(weightMatrix) + intercept - if (margin < 0) 0.0 else 1.0 - } -} - -/** - * Train an SVM using Stochastic Gradient Descent. - * NOTE: Labels used in SVM should be {0, 1} - */ -class SVMWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var regParam: Double, - var miniBatchFraction: Double) - extends GeneralizedLinearAlgorithm[SVMModel] with Serializable { - - val gradient = new HingeGradient() - val updater = new SquaredL2Updater() - override val optimizer = new GradientDescent(gradient, updater) - .setStepSize(stepSize) - .setNumIterations(numIterations) - .setRegParam(regParam) - .setMiniBatchFraction(miniBatchFraction) - - override val validators = List(DataValidators.classificationLabels) - - /** - * Construct a SVM object with default parameters - */ - def this() = this(1.0, 100, 1.0, 1.0) - - def createModel(weights: Array[Double], intercept: Double) = { - new SVMModel(weights, intercept) - } -} - -/** - * Top-level methods for calling SVM. NOTE: Labels used in SVM should be {0, 1} - */ -object SVMWithSGD { - - /** - * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. Each iteration uses - * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in - * gradient descent are initialized using the initial weights provided. - * NOTE: Labels used in SVM should be {0, 1} - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @param stepSize Step size to be used for each iteration of gradient descent. - * @param regParam Regularization parameter. - * @param miniBatchFraction Fraction of data to be used per iteration. - * @param initialWeights Initial set of weights to be used. Array should be equal in size to - * the number of features in the data. - */ - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - regParam: Double, - miniBatchFraction: Double, - initialWeights: Array[Double]) - : SVMModel = - { - new SVMWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input, - initialWeights) - } - - /** - * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. Each iteration uses - * `miniBatchFraction` fraction of the data to calculate the gradient. - * NOTE: Labels used in SVM should be {0, 1} - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @param stepSize Step size to be used for each iteration of gradient descent. - * @param regParam Regularization parameter. - * @param miniBatchFraction Fraction of data to be used per iteration. - */ - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - regParam: Double, - miniBatchFraction: Double) - : SVMModel = - { - new SVMWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input) - } - - /** - * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. We use the entire data set to - * update the gradient in each iteration. - * NOTE: Labels used in SVM should be {0, 1} - * - * @param input RDD of (label, array of features) pairs. - * @param stepSize Step size to be used for each iteration of Gradient Descent. - * @param regParam Regularization parameter. - * @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, - stepSize: Double, - regParam: Double) - : SVMModel = - { - train(input, numIterations, stepSize, regParam, 1.0) - } - - /** - * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using a step size of 1.0. We use the entire data set to - * update the gradient in each iteration. - * NOTE: Labels used in SVM should be {0, 1} - * - * @param input RDD of (label, array of features) pairs. - * @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 = - { - train(input, numIterations, 1.0, 1.0, 1.0) - } - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: SVM ") - System.exit(1) - } - 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) - - sc.stop() - } -} diff --git a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala deleted file mode 100644 index 97e3d110ae..0000000000 --- a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala +++ /dev/null @@ -1,335 +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 spark.mllib.clustering - -import scala.collection.mutable.ArrayBuffer -import scala.util.Random - -import spark.{SparkContext, RDD} -import spark.SparkContext._ -import spark.Logging -import spark.mllib.util.MLUtils - -import org.jblas.DoubleMatrix - - -/** - * K-means clustering with support for multiple parallel runs and a k-means++ like initialization - * mode (the k-means|| algorithm by Bahmani et al). When multiple concurrent runs are requested, - * they are executed together with joint passes over the data for efficiency. - * - * This is an iterative algorithm that will make multiple passes over the data, so any RDDs given - * to it should be cached by the user. - */ -class KMeans private ( - var k: Int, - var maxIterations: Int, - var runs: Int, - var initializationMode: String, - var initializationSteps: Int, - var epsilon: Double) - extends Serializable with Logging -{ - private type ClusterCenters = Array[Array[Double]] - - def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 5, 1e-4) - - /** Set the number of clusters to create (k). Default: 2. */ - def setK(k: Int): KMeans = { - this.k = k - this - } - - /** Set maximum number of iterations to run. Default: 20. */ - def setMaxIterations(maxIterations: Int): KMeans = { - this.maxIterations = maxIterations - this - } - - /** - * Set the initialization algorithm. This can be either "random" to choose random points as - * initial cluster centers, or "k-means||" to use a parallel variant of k-means++ - * (Bahmani et al., Scalable K-Means++, VLDB 2012). Default: k-means||. - */ - def setInitializationMode(initializationMode: String): KMeans = { - if (initializationMode != KMeans.RANDOM && initializationMode != KMeans.K_MEANS_PARALLEL) { - throw new IllegalArgumentException("Invalid initialization mode: " + initializationMode) - } - this.initializationMode = initializationMode - this - } - - /** - * Set the number of runs of the algorithm to execute in parallel. We initialize the algorithm - * this many times with random starting conditions (configured by the initialization mode), then - * return the best clustering found over any run. Default: 1. - */ - def setRuns(runs: Int): KMeans = { - if (runs <= 0) { - throw new IllegalArgumentException("Number of runs must be positive") - } - this.runs = runs - this - } - - /** - * Set the number of steps for the k-means|| initialization mode. This is an advanced - * setting -- the default of 5 is almost always enough. Default: 5. - */ - def setInitializationSteps(initializationSteps: Int): KMeans = { - if (initializationSteps <= 0) { - throw new IllegalArgumentException("Number of initialization steps must be positive") - } - this.initializationSteps = initializationSteps - this - } - - /** - * Set the distance threshold within which we've consider centers to have converged. - * If all centers move less than this Euclidean distance, we stop iterating one run. - */ - def setEpsilon(epsilon: Double): KMeans = { - this.epsilon = epsilon - this - } - - /** - * Train a K-means model on the given set of points; `data` should be cached for high - * performance, because this is an iterative algorithm. - */ - def run(data: RDD[Array[Double]]): KMeansModel = { - // TODO: check whether data is persistent; this needs RDD.storageLevel to be publicly readable - - val sc = data.sparkContext - - val centers = if (initializationMode == KMeans.RANDOM) { - initRandom(data) - } else { - initKMeansParallel(data) - } - - val active = Array.fill(runs)(true) - val costs = Array.fill(runs)(0.0) - - var activeRuns = new ArrayBuffer[Int] ++ (0 until runs) - var iteration = 0 - - // Execute iterations of Lloyd's algorithm until all runs have converged - while (iteration < maxIterations && !activeRuns.isEmpty) { - type WeightedPoint = (DoubleMatrix, Long) - def mergeContribs(p1: WeightedPoint, p2: WeightedPoint): WeightedPoint = { - (p1._1.addi(p2._1), p1._2 + p2._2) - } - - val activeCenters = activeRuns.map(r => centers(r)).toArray - val costAccums = activeRuns.map(_ => sc.accumulator(0.0)) - - // Find the sum and count of points mapping to each center - val totalContribs = data.mapPartitions { points => - val runs = activeCenters.length - val k = activeCenters(0).length - val dims = activeCenters(0)(0).length - - val sums = Array.fill(runs, k)(new DoubleMatrix(dims)) - val counts = Array.fill(runs, k)(0L) - - for (point <- points; (centers, runIndex) <- activeCenters.zipWithIndex) { - val (bestCenter, cost) = KMeans.findClosest(centers, point) - costAccums(runIndex) += cost - sums(runIndex)(bestCenter).addi(new DoubleMatrix(point)) - counts(runIndex)(bestCenter) += 1 - } - - val contribs = for (i <- 0 until runs; j <- 0 until k) yield { - ((i, j), (sums(i)(j), counts(i)(j))) - } - contribs.iterator - }.reduceByKey(mergeContribs).collectAsMap() - - // Update the cluster centers and costs for each active run - for ((run, i) <- activeRuns.zipWithIndex) { - var changed = false - for (j <- 0 until k) { - val (sum, count) = totalContribs((i, j)) - if (count != 0) { - val newCenter = sum.divi(count).data - if (MLUtils.squaredDistance(newCenter, centers(run)(j)) > epsilon * epsilon) { - changed = true - } - centers(run)(j) = newCenter - } - } - if (!changed) { - active(run) = false - logInfo("Run " + run + " finished in " + (iteration + 1) + " iterations") - } - costs(run) = costAccums(i).value - } - - activeRuns = activeRuns.filter(active(_)) - iteration += 1 - } - - val bestRun = costs.zipWithIndex.min._2 - new KMeansModel(centers(bestRun)) - } - - /** - * Initialize `runs` sets of cluster centers at random. - */ - private def initRandom(data: RDD[Array[Double]]): Array[ClusterCenters] = { - // Sample all the cluster centers in one pass to avoid repeated scans - val sample = data.takeSample(true, runs * k, new Random().nextInt()).toSeq - Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k).toArray) - } - - /** - * Initialize `runs` sets of cluster centers using the k-means|| algorithm by Bahmani et al. - * (Bahmani et al., Scalable K-Means++, VLDB 2012). This is a variant of k-means++ that tries - * to find with dissimilar cluster centers by starting with a random center and then doing - * passes where more centers are chosen with probability proportional to their squared distance - * to the current cluster set. It results in a provable approximation to an optimal clustering. - * - * The original paper can be found at http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf. - */ - private def initKMeansParallel(data: RDD[Array[Double]]): Array[ClusterCenters] = { - // Initialize each run's center to a random point - val seed = new Random().nextInt() - val sample = data.takeSample(true, runs, seed).toSeq - val centers = Array.tabulate(runs)(r => ArrayBuffer(sample(r))) - - // On each step, sample 2 * k points on average for each run with probability proportional - // to their squared distance from that run's current centers - for (step <- 0 until initializationSteps) { - val centerArrays = centers.map(_.toArray) - val sumCosts = data.flatMap { point => - for (r <- 0 until runs) yield (r, KMeans.pointCost(centerArrays(r), point)) - }.reduceByKey(_ + _).collectAsMap() - val chosen = data.mapPartitionsWithIndex { (index, points) => - val rand = new Random(seed ^ (step << 16) ^ index) - for { - p <- points - r <- 0 until runs - if rand.nextDouble() < KMeans.pointCost(centerArrays(r), p) * 2 * k / sumCosts(r) - } yield (r, p) - }.collect() - for ((r, p) <- chosen) { - centers(r) += p - } - } - - // Finally, we might have a set of more than k candidate centers for each run; weigh each - // candidate by the number of points in the dataset mapping to it and run a local k-means++ - // on the weighted centers to pick just k of them - val centerArrays = centers.map(_.toArray) - val weightMap = data.flatMap { p => - for (r <- 0 until runs) yield ((r, KMeans.findClosest(centerArrays(r), p)._1), 1.0) - }.reduceByKey(_ + _).collectAsMap() - val finalCenters = (0 until runs).map { r => - val myCenters = centers(r).toArray - val myWeights = (0 until myCenters.length).map(i => weightMap.getOrElse((r, i), 0.0)).toArray - LocalKMeans.kMeansPlusPlus(r, myCenters, myWeights, k, 30) - } - - finalCenters.toArray - } -} - - -/** - * Top-level methods for calling K-means clustering. - */ -object KMeans { - // Initialization mode names - val RANDOM = "random" - val K_MEANS_PARALLEL = "k-means||" - - def train( - data: RDD[Array[Double]], - k: Int, - maxIterations: Int, - runs: Int, - initializationMode: String) - : KMeansModel = - { - new KMeans().setK(k) - .setMaxIterations(maxIterations) - .setRuns(runs) - .setInitializationMode(initializationMode) - .run(data) - } - - def train(data: RDD[Array[Double]], k: Int, maxIterations: Int, runs: Int): KMeansModel = { - train(data, k, maxIterations, runs, K_MEANS_PARALLEL) - } - - def train(data: RDD[Array[Double]], k: Int, maxIterations: Int): KMeansModel = { - train(data, k, maxIterations, 1, K_MEANS_PARALLEL) - } - - /** - * Return the index of the closest point in `centers` to `point`, as well as its distance. - */ - private[mllib] def findClosest(centers: Array[Array[Double]], point: Array[Double]) - : (Int, Double) = - { - var bestDistance = Double.PositiveInfinity - var bestIndex = 0 - for (i <- 0 until centers.length) { - val distance = MLUtils.squaredDistance(point, centers(i)) - if (distance < bestDistance) { - bestDistance = distance - bestIndex = i - } - } - (bestIndex, bestDistance) - } - - /** - * Return the K-means cost of a given point against the given cluster centers. - */ - private[mllib] def pointCost(centers: Array[Array[Double]], point: Array[Double]): Double = { - var bestDistance = Double.PositiveInfinity - for (i <- 0 until centers.length) { - val distance = MLUtils.squaredDistance(point, centers(i)) - if (distance < bestDistance) { - bestDistance = distance - } - } - bestDistance - } - - def main(args: Array[String]) { - if (args.length < 4) { - println("Usage: KMeans []") - System.exit(1) - } - val (master, inputFile, k, iters) = (args(0), args(1), args(2).toInt, args(3).toInt) - val runs = if (args.length >= 5) args(4).toInt else 1 - val sc = new SparkContext(master, "KMeans") - val data = sc.textFile(inputFile).map(line => line.split(' ').map(_.toDouble)).cache() - val model = KMeans.train(data, k, iters, runs) - val cost = model.computeCost(data) - println("Cluster centers:") - for (c <- model.clusterCenters) { - println(" " + c.mkString(" ")) - } - println("Cost: " + cost) - System.exit(0) - } -} diff --git a/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala deleted file mode 100644 index b8f80e80cd..0000000000 --- a/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala +++ /dev/null @@ -1,44 +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 spark.mllib.clustering - -import spark.RDD -import spark.SparkContext._ -import spark.mllib.util.MLUtils - - -/** - * A clustering model for K-means. Each point belongs to the cluster with the closest center. - */ -class KMeansModel(val clusterCenters: Array[Array[Double]]) extends Serializable { - /** Total number of clusters. */ - def k: Int = clusterCenters.length - - /** Return the cluster index that a given point belongs to. */ - def predict(point: Array[Double]): Int = { - KMeans.findClosest(clusterCenters, point)._1 - } - - /** - * Return the K-means cost (sum of squared distances of points to their nearest center) for this - * model on the given data. - */ - def computeCost(data: RDD[Array[Double]]): Double = { - data.map(p => KMeans.pointCost(clusterCenters, p)).sum - } -} diff --git a/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala deleted file mode 100644 index 89fe7d7e85..0000000000 --- a/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala +++ /dev/null @@ -1,105 +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 spark.mllib.clustering - -import scala.util.Random - -import org.jblas.{DoubleMatrix, SimpleBlas} - -/** - * An utility object to run K-means locally. This is private to the ML package because it's used - * in the initialization of KMeans but not meant to be publicly exposed. - */ -private[mllib] object LocalKMeans { - /** - * Run K-means++ on the weighted point set `points`. This first does the K-means++ - * initialization procedure and then roudns of Lloyd's algorithm. - */ - def kMeansPlusPlus( - seed: Int, - points: Array[Array[Double]], - weights: Array[Double], - k: Int, - maxIterations: Int) - : Array[Array[Double]] = - { - val rand = new Random(seed) - val dimensions = points(0).length - val centers = new Array[Array[Double]](k) - - // Initialize centers by sampling using the k-means++ procedure - centers(0) = pickWeighted(rand, points, weights) - for (i <- 1 until k) { - // Pick the next center with a probability proportional to cost under current centers - val curCenters = centers.slice(0, i) - val sum = points.zip(weights).map { case (p, w) => - w * KMeans.pointCost(curCenters, p) - }.sum - val r = rand.nextDouble() * sum - var cumulativeScore = 0.0 - var j = 0 - while (j < points.length && cumulativeScore < r) { - cumulativeScore += weights(j) * KMeans.pointCost(curCenters, points(j)) - j += 1 - } - centers(i) = points(j-1) - } - - // Run up to maxIterations iterations of Lloyd's algorithm - val oldClosest = Array.fill(points.length)(-1) - var iteration = 0 - var moved = true - while (moved && iteration < maxIterations) { - moved = false - val sums = Array.fill(k)(new DoubleMatrix(dimensions)) - val counts = Array.fill(k)(0.0) - for ((p, i) <- points.zipWithIndex) { - val index = KMeans.findClosest(centers, p)._1 - SimpleBlas.axpy(weights(i), new DoubleMatrix(p), sums(index)) - counts(index) += weights(i) - if (index != oldClosest(i)) { - moved = true - oldClosest(i) = index - } - } - // Update centers - for (i <- 0 until k) { - if (counts(i) == 0.0) { - // Assign center to a random point - centers(i) = points(rand.nextInt(points.length)) - } else { - centers(i) = sums(i).divi(counts(i)).data - } - } - iteration += 1 - } - - centers - } - - private def pickWeighted[T](rand: Random, data: Array[T], weights: Array[Double]): T = { - val r = rand.nextDouble() * weights.sum - var i = 0 - var curWeight = 0.0 - while (i < data.length && curWeight < r) { - curWeight += weights(i) - i += 1 - } - data(i - 1) - } -} diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala deleted file mode 100644 index 05568f55af..0000000000 --- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala +++ /dev/null @@ -1,98 +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 spark.mllib.optimization - -import org.jblas.DoubleMatrix - -/** - * Class used to compute the gradient for a loss function, given a single data point. - */ -abstract class Gradient extends Serializable { - /** - * Compute the gradient and loss given features of a single data point. - * - * @param data - Feature values for one data point. Column matrix of size nx1 - * where n 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. - * - */ - def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): - (DoubleMatrix, Double) -} - -/** - * Compute gradient and loss for a logistic loss function. - */ -class LogisticGradient extends Gradient { - override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): - (DoubleMatrix, Double) = { - val margin: Double = -1.0 * data.dot(weights) - val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label - - val gradient = data.mul(gradientMultiplier) - val loss = - if (margin > 0) { - math.log(1 + math.exp(0 - margin)) - } else { - math.log(1 + math.exp(margin)) - margin - } - - (gradient, loss) - } -} - -/** - * Compute gradient and loss for a Least-squared loss function. - */ -class SquaredGradient extends Gradient { - override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): - (DoubleMatrix, Double) = { - val diff: Double = data.dot(weights) - label - - val loss = 0.5 * diff * diff - val gradient = data.mul(diff) - - (gradient, loss) - } -} - -/** - * Compute gradient and loss for a Hinge loss function. - * 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) - - // 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) - } else { - (DoubleMatrix.zeros(1, weights.length), 0.0) - } - } -} diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala deleted file mode 100644 index 31917df7e8..0000000000 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ /dev/null @@ -1,166 +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 spark.mllib.optimization - -import spark.{Logging, RDD, SparkContext} -import spark.SparkContext._ - -import org.jblas.DoubleMatrix - -import scala.collection.mutable.ArrayBuffer - -/** - * Class used to solve an optimization problem using Gradient Descent. - * @param gradient Gradient function to be used. - * @param updater Updater to be used to update weights after every iteration. - */ -class GradientDescent(var gradient: Gradient, var updater: Updater) extends Optimizer { - - private var stepSize: Double = 1.0 - private var numIterations: Int = 100 - private var regParam: Double = 0.0 - private var miniBatchFraction: Double = 1.0 - - /** - * Set the step size per-iteration of SGD. Default 1.0. - */ - def setStepSize(step: Double): this.type = { - this.stepSize = step - this - } - - /** - * Set fraction of data to be used for each SGD iteration. Default 1.0. - */ - def setMiniBatchFraction(fraction: Double): this.type = { - this.miniBatchFraction = fraction - this - } - - /** - * Set the number of iterations for SGD. Default 100. - */ - def setNumIterations(iters: Int): this.type = { - this.numIterations = iters - this - } - - /** - * Set the regularization parameter used for SGD. Default 0.0. - */ - def setRegParam(regParam: Double): this.type = { - this.regParam = regParam - this - } - - /** - * Set the gradient function to be used for SGD. - */ - def setGradient(gradient: Gradient): this.type = { - this.gradient = gradient - this - } - - - /** - * Set the updater function to be used for SGD. - */ - def setUpdater(updater: Updater): this.type = { - this.updater = updater - this - } - - def optimize(data: RDD[(Double, Array[Double])], initialWeights: Array[Double]) - : Array[Double] = { - - val (weights, stochasticLossHistory) = GradientDescent.runMiniBatchSGD( - data, - gradient, - updater, - stepSize, - numIterations, - regParam, - miniBatchFraction, - initialWeights) - weights - } - -} - -// Top-level method to run gradient descent. -object GradientDescent extends Logging { - /** - * Run gradient descent in parallel using mini batches. - * - * @param data - Input data for SGD. RDD of form (label, [feature values]). - * @param gradient - Gradient object that will be used to compute the gradient. - * @param updater - Updater object that will be used to update the model. - * @param stepSize - stepSize to be used during update. - * @param numIterations - number of iterations that SGD should be run. - * @param regParam - regularization parameter - * @param miniBatchFraction - fraction of the input data set that should be used for - * one iteration of SGD. Default value 1.0. - * - * @return A tuple containing two elements. The first element is a column matrix containing - * weights for every feature, and the second element is an array containing the stochastic - * loss computed for every iteration. - */ - def runMiniBatchSGD( - data: RDD[(Double, Array[Double])], - gradient: Gradient, - updater: Updater, - stepSize: Double, - numIterations: Int, - regParam: Double, - miniBatchFraction: Double, - initialWeights: Array[Double]) : (Array[Double], Array[Double]) = { - - val stochasticLossHistory = new ArrayBuffer[Double](numIterations) - - val nexamples: Long = data.count() - val miniBatchSize = nexamples * miniBatchFraction - - // Initialize weights as a column vector - var weights = new DoubleMatrix(initialWeights.length, 1, initialWeights:_*) - var regVal = 0.0 - - for (i <- 1 to numIterations) { - 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)) - - /** - * NOTE(Xinghao): lossSum is computed using the weights from the previous iteration - * and regVal is the regularization value computed in the previous iteration as well. - */ - stochasticLossHistory.append(lossSum / miniBatchSize + regVal) - val update = updater.compute( - weights, gradientSum.div(miniBatchSize), stepSize, i, regParam) - weights = update._1 - regVal = update._2 - } - - logInfo("GradientDescent finished. Last 10 stochastic losses %s".format( - stochasticLossHistory.takeRight(10).mkString(", "))) - - (weights.toArray, stochasticLossHistory.toArray) - } -} diff --git a/mllib/src/main/scala/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/spark/mllib/optimization/Optimizer.scala deleted file mode 100644 index 76a519c338..0000000000 --- a/mllib/src/main/scala/spark/mllib/optimization/Optimizer.scala +++ /dev/null @@ -1,29 +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 spark.mllib.optimization - -import spark.RDD - -trait Optimizer { - - /** - * Solve the provided convex optimization problem. - */ - def optimize(data: RDD[(Double, Array[Double])], initialWeights: Array[Double]): Array[Double] - -} diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala deleted file mode 100644 index db67d6b0bc..0000000000 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ /dev/null @@ -1,99 +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 spark.mllib.optimization - -import scala.math._ -import org.jblas.DoubleMatrix - -/** - * Class used to update weights used in Gradient Descent. - */ -abstract class Updater extends Serializable { - /** - * Compute an updated value for weights given the gradient, stepSize, iteration number and - * regularization parameter. Also returns the regularization value computed using the - * *updated* weights. - * - * @param weightsOld - Column matrix of size nx1 where n is the number of features. - * @param gradient - Column matrix of size nx1 where n is the number of features. - * @param stepSize - step size across iterations - * @param iter - Iteration number - * @param regParam - Regularization parameter - * - * @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) -} - -/** - * A simple updater that adaptively adjusts the learning rate the - * square root of the number of iterations. Does not perform any regularization. - */ -class SimpleUpdater extends Updater { - override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, - stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { - val thisIterStepSize = stepSize / math.sqrt(iter) - val normGradient = gradient.mul(thisIterStepSize) - (weightsOld.sub(normGradient), 0) - } -} - -/** - * Updater that adjusts learning rate and performs L1 regularization. - * - * The corresponding proximal operator used is the soft-thresholding function. - * That is, each weight component is shrunk towards 0 by shrinkageVal. - * - * If w > shrinkageVal, set weight component to w-shrinkageVal. - * If w < -shrinkageVal, set weight component to w+shrinkageVal. - * If -shrinkageVal < w < shrinkageVal, set weight component to 0. - * - * 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) = { - val thisIterStepSize = stepSize / math.sqrt(iter) - val normGradient = gradient.mul(thisIterStepSize) - // Take gradient step - val newWeights = weightsOld.sub(normGradient) - // 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)) - } - (newWeights, newWeights.norm1 * regParam) - } -} - -/** - * Updater that adjusts the learning rate and performs L2 regularization - */ -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 normGradient = gradient.mul(thisIterStepSize) - val newWeights = weightsOld.sub(normGradient).div(2.0 * thisIterStepSize * regParam + 1.0) - (newWeights, pow(newWeights.norm2, 2.0) * regParam) - } -} - diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala deleted file mode 100644 index dbfbf59975..0000000000 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ /dev/null @@ -1,453 +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 spark.mllib.recommendation - -import scala.collection.mutable.{ArrayBuffer, BitSet} -import scala.util.Random -import scala.util.Sorting - -import spark.{HashPartitioner, Partitioner, SparkContext, RDD} -import spark.storage.StorageLevel -import spark.KryoRegistrator -import spark.SparkContext._ - -import com.esotericsoftware.kryo.Kryo -import org.jblas.{DoubleMatrix, SimpleBlas, Solve} - - -/** - * Out-link information for a user or product block. This includes the original user/product IDs - * of the elements within this block, and the list of destination blocks that each user or - * product will need to send its feature vector to. - */ -private[recommendation] case class OutLinkBlock(elementIds: Array[Int], shouldSend: Array[BitSet]) - - -/** - * In-link information for a user (or product) block. This includes the original user/product IDs - * of the elements within this block, as well as an array of indices and ratings that specify - * which user in the block will be rated by which products from each product block (or vice-versa). - * Specifically, if this InLinkBlock is for users, ratingsForBlock(b)(i) will contain two arrays, - * indices and ratings, for the i'th product that will be sent to us by product block b (call this - * P). These arrays represent the users that product P had ratings for (by their index in this - * block), as well as the corresponding rating for each one. We can thus use this information when - * we get product block b's message to update the corresponding users. - */ -private[recommendation] case class InLinkBlock( - elementIds: Array[Int], ratingsForBlock: Array[Array[(Array[Int], Array[Double])]]) - - -/** - * A more compact class to represent a rating than Tuple3[Int, Int, Double]. - */ -case class Rating(val user: Int, val product: Int, val rating: Double) - -/** - * Alternating Least Squares matrix factorization. - * - * This is a blocked implementation of the ALS factorization algorithm that groups the two sets - * of factors (referred to as "users" and "products") into blocks and reduces communication by only - * sending one copy of each user vector to each product block on each iteration, and only for the - * product blocks that need that user's feature vector. This is achieved by precomputing some - * information about the ratings matrix to determine the "out-links" of each user (which blocks of - * products it will contribute to) and "in-link" information for each product (which of the feature - * vectors it receives from each user block it will depend on). This allows us to send only an - * array of feature vectors between each user block and product block, and have the product block - * find the users' ratings and update the products based on these messages. - */ -class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var lambda: Double) - extends Serializable -{ - def this() = this(-1, 10, 10, 0.01) - - /** - * Set the number of blocks to parallelize the computation into; pass -1 for an auto-configured - * number of blocks. Default: -1. - */ - def setBlocks(numBlocks: Int): ALS = { - this.numBlocks = numBlocks - this - } - - /** Set the rank of the feature matrices computed (number of features). Default: 10. */ - def setRank(rank: Int): ALS = { - this.rank = rank - this - } - - /** Set the number of iterations to run. Default: 10. */ - def setIterations(iterations: Int): ALS = { - this.iterations = iterations - this - } - - /** Set the regularization parameter, lambda. Default: 0.01. */ - def setLambda(lambda: Double): ALS = { - this.lambda = lambda - this - } - - /** - * Run ALS with the configured parameters on an input RDD of (user, product, rating) triples. - * Returns a MatrixFactorizationModel with feature vectors for each user and product. - */ - def run(ratings: RDD[Rating]): MatrixFactorizationModel = { - val numBlocks = if (this.numBlocks == -1) { - math.max(ratings.context.defaultParallelism, ratings.partitions.size / 2) - } else { - this.numBlocks - } - - val partitioner = new HashPartitioner(numBlocks) - - val ratingsByUserBlock = ratings.map{ rating => (rating.user % numBlocks, rating) } - val ratingsByProductBlock = ratings.map{ rating => - (rating.product % numBlocks, Rating(rating.product, rating.user, rating.rating)) - } - - val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock) - val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) - - // Initialize user and product factors randomly, but use a deterministic seed for each partition - // so that fault recovery works - val seedGen = new Random() - val seed1 = seedGen.nextInt() - val seed2 = seedGen.nextInt() - // Hash an integer to propagate random bits at all positions, similar to java.util.HashTable - def hash(x: Int): Int = { - val r = x ^ (x >>> 20) ^ (x >>> 12) - r ^ (r >>> 7) ^ (r >>> 4) - } - var users = userOutLinks.mapPartitionsWithIndex { (index, itr) => - val rand = new Random(hash(seed1 ^ index)) - itr.map { case (x, y) => - (x, y.elementIds.map(_ => randomFactor(rank, rand))) - } - } - var products = productOutLinks.mapPartitionsWithIndex { (index, itr) => - val rand = new Random(hash(seed2 ^ index)) - itr.map { case (x, y) => - (x, y.elementIds.map(_ => randomFactor(rank, rand))) - } - } - - for (iter <- 0 until iterations) { - // perform ALS update - products = updateFeatures(users, userOutLinks, productInLinks, partitioner, rank, lambda) - users = updateFeatures(products, productOutLinks, userInLinks, partitioner, rank, lambda) - } - - // Flatten and cache the two final RDDs to un-block them - val usersOut = users.join(userOutLinks).flatMap { case (b, (factors, outLinkBlock)) => - for (i <- 0 until factors.length) yield (outLinkBlock.elementIds(i), factors(i)) - } - val productsOut = products.join(productOutLinks).flatMap { case (b, (factors, outLinkBlock)) => - for (i <- 0 until factors.length) yield (outLinkBlock.elementIds(i), factors(i)) - } - - usersOut.persist() - productsOut.persist() - - new MatrixFactorizationModel(rank, usersOut, productsOut) - } - - /** - * Make the out-links table for a block of the users (or products) dataset given the list of - * (user, product, rating) values for the users in that block (or the opposite for products). - */ - private def makeOutLinkBlock(numBlocks: Int, ratings: Array[Rating]): OutLinkBlock = { - val userIds = ratings.map(_.user).distinct.sorted - val numUsers = userIds.length - val userIdToPos = userIds.zipWithIndex.toMap - val shouldSend = Array.fill(numUsers)(new BitSet(numBlocks)) - for (r <- ratings) { - shouldSend(userIdToPos(r.user))(r.product % numBlocks) = true - } - OutLinkBlock(userIds, shouldSend) - } - - /** - * Make the in-links table for a block of the users (or products) dataset given a list of - * (user, product, rating) values for the users in that block (or the opposite for products). - */ - private def makeInLinkBlock(numBlocks: Int, ratings: Array[Rating]): InLinkBlock = { - val userIds = ratings.map(_.user).distinct.sorted - val numUsers = userIds.length - val userIdToPos = userIds.zipWithIndex.toMap - // Split out our ratings by product block - val blockRatings = Array.fill(numBlocks)(new ArrayBuffer[Rating]) - for (r <- ratings) { - blockRatings(r.product % numBlocks) += r - } - val ratingsForBlock = new Array[Array[(Array[Int], Array[Double])]](numBlocks) - for (productBlock <- 0 until numBlocks) { - // Create an array of (product, Seq(Rating)) ratings - val groupedRatings = blockRatings(productBlock).groupBy(_.product).toArray - // Sort them by product ID - val ordering = new Ordering[(Int, ArrayBuffer[Rating])] { - def compare(a: (Int, ArrayBuffer[Rating]), b: (Int, ArrayBuffer[Rating])): Int = a._1 - b._1 - } - Sorting.quickSort(groupedRatings)(ordering) - // Translate the user IDs to indices based on userIdToPos - ratingsForBlock(productBlock) = groupedRatings.map { case (p, rs) => - (rs.view.map(r => userIdToPos(r.user)).toArray, rs.view.map(_.rating).toArray) - } - } - InLinkBlock(userIds, ratingsForBlock) - } - - /** - * Make RDDs of InLinkBlocks and OutLinkBlocks given an RDD of (blockId, (u, p, r)) values for - * the users (or (blockId, (p, u, r)) for the products). We create these simultaneously to avoid - * having to shuffle the (blockId, (u, p, r)) RDD twice, or to cache it. - */ - private def makeLinkRDDs(numBlocks: Int, ratings: RDD[(Int, Rating)]) - : (RDD[(Int, InLinkBlock)], RDD[(Int, OutLinkBlock)]) = - { - val grouped = ratings.partitionBy(new HashPartitioner(numBlocks)) - val links = grouped.mapPartitionsWithIndex((blockId, elements) => { - val ratings = elements.map{_._2}.toArray - val inLinkBlock = makeInLinkBlock(numBlocks, ratings) - val outLinkBlock = makeOutLinkBlock(numBlocks, ratings) - Iterator.single((blockId, (inLinkBlock, outLinkBlock))) - }, true) - links.persist(StorageLevel.MEMORY_AND_DISK) - (links.mapValues(_._1), links.mapValues(_._2)) - } - - /** - * Make a random factor vector with the given random. - */ - private def randomFactor(rank: Int, rand: Random): Array[Double] = { - Array.fill(rank)(rand.nextDouble) - } - - /** - * Compute the user feature vectors given the current products (or vice-versa). This first joins - * the products with their out-links to generate a set of messages to each destination block - * (specifically, the features for the products that user block cares about), then groups these - * by destination and joins them with the in-link info to figure out how to update each user. - * It returns an RDD of new feature vectors for each user block. - */ - private def updateFeatures( - products: RDD[(Int, Array[Array[Double]])], - productOutLinks: RDD[(Int, OutLinkBlock)], - userInLinks: RDD[(Int, InLinkBlock)], - partitioner: Partitioner, - rank: Int, - lambda: Double) - : RDD[(Int, Array[Array[Double]])] = - { - val numBlocks = products.partitions.size - productOutLinks.join(products).flatMap { case (bid, (outLinkBlock, factors)) => - val toSend = Array.fill(numBlocks)(new ArrayBuffer[Array[Double]]) - for (p <- 0 until outLinkBlock.elementIds.length; userBlock <- 0 until numBlocks) { - if (outLinkBlock.shouldSend(p)(userBlock)) { - toSend(userBlock) += factors(p) - } - } - toSend.zipWithIndex.map{ case (buf, idx) => (idx, (bid, buf.toArray)) } - }.groupByKey(partitioner) - .join(userInLinks) - .mapValues{ case (messages, inLinkBlock) => updateBlock(messages, inLinkBlock, rank, lambda) } - } - - /** - * Compute the new feature vectors for a block of the users matrix given the list of factors - * it received from each product and its InLinkBlock. - */ - def updateBlock(messages: Seq[(Int, Array[Array[Double]])], inLinkBlock: InLinkBlock, - rank: Int, lambda: Double) - : Array[Array[Double]] = - { - // Sort the incoming block factor messages by block ID and make them an array - val blockFactors = messages.sortBy(_._1).map(_._2).toArray // Array[Array[Double]] - val numBlocks = blockFactors.length - val numUsers = inLinkBlock.elementIds.length - - // We'll sum up the XtXes using vectors that represent only the lower-triangular part, since - // the matrices are symmetric - val triangleSize = rank * (rank + 1) / 2 - val userXtX = Array.fill(numUsers)(DoubleMatrix.zeros(triangleSize)) - val userXy = Array.fill(numUsers)(DoubleMatrix.zeros(rank)) - - // Some temp variables to avoid memory allocation - val tempXtX = DoubleMatrix.zeros(triangleSize) - val fullXtX = DoubleMatrix.zeros(rank, rank) - - // Compute the XtX and Xy values for each user by adding products it rated in each product block - for (productBlock <- 0 until numBlocks) { - for (p <- 0 until blockFactors(productBlock).length) { - val x = new DoubleMatrix(blockFactors(productBlock)(p)) - fillXtX(x, tempXtX) - val (us, rs) = inLinkBlock.ratingsForBlock(productBlock)(p) - for (i <- 0 until us.length) { - userXtX(us(i)).addi(tempXtX) - SimpleBlas.axpy(rs(i), x, userXy(us(i))) - } - } - } - - // Solve the least-squares problem for each user and return the new feature vectors - userXtX.zipWithIndex.map{ case (triangularXtX, index) => - // Compute the full XtX matrix from the lower-triangular part we got above - fillFullMatrix(triangularXtX, fullXtX) - // Add regularization - (0 until rank).foreach(i => fullXtX.data(i*rank + i) += lambda) - // Solve the resulting matrix, which is symmetric and positive-definite - Solve.solvePositive(fullXtX, userXy(index)).data - } - } - - /** - * Set xtxDest to the lower-triangular part of x transpose * x. For efficiency in summing - * these matrices, we store xtxDest as only rank * (rank+1) / 2 values, namely the values - * at (0,0), (1,0), (1,1), (2,0), (2,1), (2,2), etc in that order. - */ - private def fillXtX(x: DoubleMatrix, xtxDest: DoubleMatrix) { - var i = 0 - var pos = 0 - while (i < x.length) { - var j = 0 - while (j <= i) { - xtxDest.data(pos) = x.data(i) * x.data(j) - pos += 1 - j += 1 - } - i += 1 - } - } - - /** - * Given a triangular matrix in the order of fillXtX above, compute the full symmetric square - * matrix that it represents, storing it into destMatrix. - */ - private def fillFullMatrix(triangularMatrix: DoubleMatrix, destMatrix: DoubleMatrix) { - val rank = destMatrix.rows - var i = 0 - var pos = 0 - while (i < rank) { - var j = 0 - while (j <= i) { - destMatrix.data(i*rank + j) = triangularMatrix.data(pos) - destMatrix.data(j*rank + i) = triangularMatrix.data(pos) - pos += 1 - j += 1 - } - i += 1 - } - } -} - - -/** - * Top-level methods for calling Alternating Least Squares (ALS) matrix factorizaton. - */ -object ALS { - /** - * Train a matrix factorization model given an RDD of ratings given by users to some products, - * in the form of (userID, productID, rating) pairs. We approximate the ratings matrix as the - * product of two lower-rank matrices of a given rank (number of features). To solve for these - * features, we run a given number of iterations of ALS. This is done using a level of - * parallelism given by `blocks`. - * - * @param ratings RDD of (userID, productID, rating) pairs - * @param rank number of features to use - * @param iterations number of iterations of ALS (recommended: 10-20) - * @param lambda regularization factor (recommended: 0.01) - * @param blocks level of parallelism to split computation into - */ - def train( - ratings: RDD[Rating], - rank: Int, - iterations: Int, - lambda: Double, - blocks: Int) - : MatrixFactorizationModel = - { - new ALS(blocks, rank, iterations, lambda).run(ratings) - } - - /** - * Train a matrix factorization model given an RDD of ratings given by users to some products, - * in the form of (userID, productID, rating) pairs. We approximate the ratings matrix as the - * product of two lower-rank matrices of a given rank (number of features). To solve for these - * features, we run a given number of iterations of ALS. The level of parallelism is determined - * automatically based on the number of partitions in `ratings`. - * - * @param ratings RDD of (userID, productID, rating) pairs - * @param rank number of features to use - * @param iterations number of iterations of ALS (recommended: 10-20) - * @param lambda regularization factor (recommended: 0.01) - */ - def train(ratings: RDD[Rating], rank: Int, iterations: Int, lambda: Double) - : MatrixFactorizationModel = - { - train(ratings, rank, iterations, lambda, -1) - } - - /** - * Train a matrix factorization model given an RDD of ratings given by users to some products, - * in the form of (userID, productID, rating) pairs. We approximate the ratings matrix as the - * product of two lower-rank matrices of a given rank (number of features). To solve for these - * features, we run a given number of iterations of ALS. The level of parallelism is determined - * automatically based on the number of partitions in `ratings`. - * - * @param ratings RDD of (userID, productID, rating) pairs - * @param rank number of features to use - * @param iterations number of iterations of ALS (recommended: 10-20) - */ - def train(ratings: RDD[Rating], rank: Int, iterations: Int) - : MatrixFactorizationModel = - { - train(ratings, rank, iterations, 0.01, -1) - } - - private class ALSRegistrator extends KryoRegistrator { - override def registerClasses(kryo: Kryo) { - kryo.register(classOf[Rating]) - } - } - - def main(args: Array[String]) { - if (args.length != 5 && args.length != 6) { - println("Usage: ALS []") - System.exit(1) - } - val (master, ratingsFile, rank, iters, outputDir) = - (args(0), args(1), args(2).toInt, args(3).toInt, args(4)) - val blocks = if (args.length == 6) args(5).toInt else -1 - System.setProperty("spark.serializer", "spark.KryoSerializer") - System.setProperty("spark.kryo.registrator", classOf[ALSRegistrator].getName) - System.setProperty("spark.kryo.referenceTracking", "false") - System.setProperty("spark.kryoserializer.buffer.mb", "8") - System.setProperty("spark.locality.wait", "10000") - val sc = new SparkContext(master, "ALS") - val ratings = sc.textFile(ratingsFile).map { line => - val fields = line.split(',') - Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble) - } - val model = ALS.train(ratings, rank, iters, 0.01, blocks) - model.userFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") } - .saveAsTextFile(outputDir + "/userFeatures") - model.productFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") } - .saveAsTextFile(outputDir + "/productFeatures") - println("Final user/product features written to " + outputDir) - System.exit(0) - } -} diff --git a/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala deleted file mode 100644 index 5e21717da5..0000000000 --- a/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ /dev/null @@ -1,49 +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 spark.mllib.recommendation - -import spark.RDD -import spark.SparkContext._ - -import org.jblas._ - -/** - * Model representing the result of matrix factorization. - * - * @param rank Rank for the features in this model. - * @param userFeatures RDD of tuples where each tuple represents the userId and - * the features computed for this user. - * @param productFeatures RDD of tuples where each tuple represents the productId - * and the features computed for this product. - */ -class MatrixFactorizationModel( - val rank: Int, - val userFeatures: RDD[(Int, Array[Double])], - val productFeatures: RDD[(Int, Array[Double])]) - extends Serializable -{ - /** Predict the rating of one user for one product. */ - def predict(user: Int, product: Int): Double = { - val userVector = new DoubleMatrix(userFeatures.lookup(user).head) - val productVector = new DoubleMatrix(productFeatures.lookup(product).head) - userVector.dot(productVector) - } - - // TODO: Figure out what good bulk prediction methods would look like. - // Probably want a way to get the top users for a product or vice-versa. -} diff --git a/mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala deleted file mode 100644 index d164d415d6..0000000000 --- a/mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ /dev/null @@ -1,159 +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 spark.mllib.regression - -import spark.{Logging, RDD, SparkException} -import spark.mllib.optimization._ - -import org.jblas.DoubleMatrix - -/** - * GeneralizedLinearModel (GLM) represents a model trained using - * GeneralizedLinearAlgorithm. GLMs consist of a weight vector and - * an intercept. - * - * @param weights Weights computed for every feature. - * @param intercept Intercept computed for this model. - */ -abstract class GeneralizedLinearModel(val weights: Array[Double], 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. - * - * @param dataMatrix Row vector containing the features for this data point - * @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 - - /** - * Predict values for the given data set using the model trained. - * - * @param testData RDD representing data points to be predicted - * @return RDD[Double] where each entry contains the corresponding prediction - */ - def predict(testData: spark.RDD[Array[Double]]): RDD[Double] = { - // A small optimization to avoid serializing the entire model. Only the weightsMatrix - // and intercept is needed. - val localWeights = weightsMatrix - val localIntercept = intercept - - testData.map { x => - val dataMatrix = new DoubleMatrix(1, x.length, x:_*) - predictPoint(dataMatrix, localWeights, localIntercept) - } - } - - /** - * Predict values for a single data point using the model trained. - * - * @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) - } -} - -/** - * GeneralizedLinearAlgorithm implements methods to train a Genearalized Linear Model (GLM). - * This class should be extended with an Optimizer to create a new GLM. - */ -abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] - extends Logging with Serializable { - - protected val validators: Seq[RDD[LabeledPoint] => Boolean] = List() - - val optimizer: Optimizer - - protected var addIntercept: Boolean = true - - protected var validateData: Boolean = true - - /** - * Create a model given the weights and intercept - */ - protected def createModel(weights: Array[Double], intercept: Double): M - - /** - * Set if the algorithm should add an intercept. Default true. - */ - def setIntercept(addIntercept: Boolean): this.type = { - this.addIntercept = addIntercept - this - } - - /** - * Set if the algorithm should validate data before training. Default true. - */ - def setValidateData(validateData: Boolean): this.type = { - this.validateData = validateData - this - } - - /** - * 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 = Array.fill(nfeatures)(1.0) - run(input, initialWeights) - } - - /** - * 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 = { - - // Check the data properties before running the optimizer - if (validateData && !validators.forall(func => func(input))) { - throw new SparkException("Input validation failed.") - } - - // Add a extra variable consisting of all 1.0's for the intercept. - val data = if (addIntercept) { - input.map(labeledPoint => (labeledPoint.label, Array(1.0, labeledPoint.features:_*))) - } else { - input.map(labeledPoint => (labeledPoint.label, labeledPoint.features)) - } - - val initialWeightsWithIntercept = if (addIntercept) { - Array(1.0, initialWeights:_*) - } else { - initialWeights - } - - val weights = optimizer.optimize(data, initialWeightsWithIntercept) - val intercept = weights(0) - val weightsScaled = weights.tail - - val model = createModel(weightsScaled, intercept) - - logInfo("Final model weights " + model.weights.mkString(",")) - logInfo("Final model intercept " + model.intercept) - model - } -} diff --git a/mllib/src/main/scala/spark/mllib/regression/LabeledPoint.scala b/mllib/src/main/scala/spark/mllib/regression/LabeledPoint.scala deleted file mode 100644 index 3de60482c5..0000000000 --- a/mllib/src/main/scala/spark/mllib/regression/LabeledPoint.scala +++ /dev/null @@ -1,26 +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 spark.mllib.regression - -/** - * 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(val label: Double, val features: Array[Double]) diff --git a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala deleted file mode 100644 index 0f33456ef4..0000000000 --- a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala +++ /dev/null @@ -1,210 +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 spark.mllib.regression - -import spark.{Logging, RDD, SparkContext} -import spark.mllib.optimization._ -import spark.mllib.util.MLUtils - -import org.jblas.DoubleMatrix - -/** - * Regression model trained using Lasso. - * - * @param weights Weights computed for every feature. - * @param intercept Intercept computed for this model. - */ -class LassoModel( - 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) = { - dataMatrix.dot(weightMatrix) + intercept - } -} - -/** - * Train a regression model with L1-regularization using Stochastic Gradient Descent. - */ -class LassoWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var regParam: Double, - var miniBatchFraction: Double) - extends GeneralizedLinearAlgorithm[LassoModel] - with Serializable { - - val gradient = new SquaredGradient() - val updater = new L1Updater() - @transient val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) - .setNumIterations(numIterations) - .setRegParam(regParam) - .setMiniBatchFraction(miniBatchFraction) - - // We don't want to penalize the intercept, so set this to false. - setIntercept(false) - - var yMean = 0.0 - var xColMean: DoubleMatrix = _ - var xColSd: DoubleMatrix = _ - - /** - * Construct a Lasso object with default parameters - */ - 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):_*) - val weightsScaled = weightsMat.div(xColSd) - val interceptScaled = yMean - (weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0)) - - new LassoModel(weightsScaled.data, interceptScaled) - } - - override def run( - input: RDD[LabeledPoint], - initialWeights: Array[Double]) - : LassoModel = - { - val nfeatures: Int = input.first.features.length - 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 - - 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) - } - - super.run(normalizedData, initialWeights) - } -} - -/** - * Top-level methods for calling Lasso. - */ -object LassoWithSGD { - - /** - * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. Each iteration uses - * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in - * gradient descent are initialized using the initial weights provided. - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @param stepSize Step size to be used for each iteration of gradient descent. - * @param regParam Regularization parameter. - * @param miniBatchFraction Fraction of data to be used per iteration. - * @param initialWeights Initial set of weights to be used. Array should be equal in size to - * the number of features in the data. - */ - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - regParam: Double, - miniBatchFraction: Double, - initialWeights: Array[Double]) - : LassoModel = - { - new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input, - initialWeights) - } - - /** - * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. Each iteration uses - * `miniBatchFraction` fraction of the data to calculate the gradient. - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @param stepSize Step size to be used for each iteration of gradient descent. - * @param regParam Regularization parameter. - * @param miniBatchFraction Fraction of data to be used per iteration. - */ - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - regParam: Double, - miniBatchFraction: Double) - : LassoModel = - { - new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input) - } - - /** - * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. We use the entire data set to - * update the gradient in each iteration. - * - * @param input RDD of (label, array of features) pairs. - * @param stepSize Step size to be used for each iteration of Gradient Descent. - * @param regParam Regularization parameter. - * @param numIterations Number of iterations of gradient descent to run. - * @return a LassoModel which has the weights and offset from training. - */ - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - regParam: Double) - : LassoModel = - { - train(input, numIterations, stepSize, regParam, 1.0) - } - - /** - * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using a step size of 1.0. We use the entire data set to - * update the gradient in each iteration. - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @return a LassoModel which has the weights and offset from training. - */ - def train( - input: RDD[LabeledPoint], - numIterations: Int) - : LassoModel = - { - train(input, numIterations, 1.0, 1.0, 1.0) - } - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: Lasso ") - System.exit(1) - } - 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) - - sc.stop() - } -} diff --git a/mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala deleted file mode 100644 index 885ff5a30d..0000000000 --- a/mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala +++ /dev/null @@ -1,167 +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 spark.mllib.regression - -import spark.{Logging, RDD, SparkContext} -import spark.mllib.optimization._ -import spark.mllib.util.MLUtils - -import org.jblas.DoubleMatrix - -/** - * Regression model trained using LinearRegression. - * - * @param weights Weights computed for every feature. - * @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) = { - dataMatrix.dot(weightMatrix) + intercept - } -} - -/** - * Train a regression model with no regularization using Stochastic Gradient Descent. - */ -class LinearRegressionWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var miniBatchFraction: Double) - extends GeneralizedLinearAlgorithm[LinearRegressionModel] - with Serializable { - - val gradient = new SquaredGradient() - val updater = new SimpleUpdater() - val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) - .setNumIterations(numIterations) - .setMiniBatchFraction(miniBatchFraction) - - /** - * Construct a LinearRegression object with default parameters - */ - def this() = this(1.0, 100, 1.0) - - def createModel(weights: Array[Double], intercept: Double) = { - new LinearRegressionModel(weights, intercept) - } -} - -/** - * Top-level methods for calling LinearRegression. - */ -object LinearRegressionWithSGD { - - /** - * Train a Linear Regression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. Each iteration uses - * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in - * gradient descent are initialized using the initial weights provided. - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @param stepSize Step size to be used for each iteration of gradient descent. - * @param miniBatchFraction Fraction of data to be used per iteration. - * @param initialWeights Initial set of weights to be used. Array should be equal in size to - * the number of features in the data. - */ - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - miniBatchFraction: Double, - initialWeights: Array[Double]) - : LinearRegressionModel = - { - new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction).run(input, - initialWeights) - } - - /** - * Train a LinearRegression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. Each iteration uses - * `miniBatchFraction` fraction of the data to calculate the gradient. - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @param stepSize Step size to be used for each iteration of gradient descent. - * @param miniBatchFraction Fraction of data to be used per iteration. - */ - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - miniBatchFraction: Double) - : LinearRegressionModel = - { - new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction).run(input) - } - - /** - * Train a LinearRegression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. We use the entire data set to - * update the gradient in each iteration. - * - * @param input RDD of (label, array of features) pairs. - * @param stepSize Step size to be used for each iteration of Gradient Descent. - * @param numIterations Number of iterations of gradient descent to run. - * @return a LinearRegressionModel which has the weights and offset from training. - */ - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double) - : LinearRegressionModel = - { - train(input, numIterations, stepSize, 1.0) - } - - /** - * Train a LinearRegression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using a step size of 1.0. We use the entire data set to - * update the gradient in each iteration. - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @return a LinearRegressionModel which has the weights and offset from training. - */ - def train( - input: RDD[LabeledPoint], - numIterations: Int) - : LinearRegressionModel = - { - train(input, numIterations, 1.0, 1.0) - } - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: LinearRegression ") - System.exit(1) - } - 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) - - sc.stop() - } -} diff --git a/mllib/src/main/scala/spark/mllib/regression/RegressionModel.scala b/mllib/src/main/scala/spark/mllib/regression/RegressionModel.scala deleted file mode 100644 index b845ba1a89..0000000000 --- a/mllib/src/main/scala/spark/mllib/regression/RegressionModel.scala +++ /dev/null @@ -1,38 +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 spark.mllib.regression - -import spark.RDD - -trait RegressionModel extends Serializable { - /** - * Predict values for the given data set using the model trained. - * - * @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] - - /** - * Predict values for a single data point using the model trained. - * - * @param testData array representing a single data point - * @return Double prediction from the trained model - */ - def predict(testData: Array[Double]): Double -} diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala deleted file mode 100644 index cb1303dd99..0000000000 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ /dev/null @@ -1,213 +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 spark.mllib.regression - -import spark.{Logging, RDD, SparkContext} -import spark.mllib.optimization._ -import spark.mllib.util.MLUtils - -import org.jblas.DoubleMatrix - -/** - * Regression model trained using RidgeRegression. - * - * @param weights Weights computed for every feature. - * @param intercept Intercept computed for this model. - */ -class RidgeRegressionModel( - 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) = { - dataMatrix.dot(weightMatrix) + intercept - } -} - -/** - * Train a regression model with L2-regularization using Stochastic Gradient Descent. - */ -class RidgeRegressionWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var regParam: Double, - var miniBatchFraction: Double) - extends GeneralizedLinearAlgorithm[RidgeRegressionModel] - with Serializable { - - val gradient = new SquaredGradient() - val updater = new SquaredL2Updater() - - @transient val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) - .setNumIterations(numIterations) - .setRegParam(regParam) - .setMiniBatchFraction(miniBatchFraction) - - // We don't want to penalize the intercept in RidgeRegression, so set this to false. - setIntercept(false) - - var yMean = 0.0 - var xColMean: DoubleMatrix = _ - var xColSd: DoubleMatrix = _ - - /** - * Construct a RidgeRegression object with default parameters - */ - 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):_*) - val weightsScaled = weightsMat.div(xColSd) - val interceptScaled = yMean - (weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0)) - - new RidgeRegressionModel(weightsScaled.data, interceptScaled) - } - - override def run( - input: RDD[LabeledPoint], - initialWeights: Array[Double]) - : RidgeRegressionModel = - { - val nfeatures: Int = input.first.features.length - 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 - - 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) - } - - super.run(normalizedData, initialWeights) - } -} - -/** - * Top-level methods for calling RidgeRegression. - */ -object RidgeRegressionWithSGD { - - /** - * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. Each iteration uses - * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in - * gradient descent are initialized using the initial weights provided. - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @param stepSize Step size to be used for each iteration of gradient descent. - * @param regParam Regularization parameter. - * @param miniBatchFraction Fraction of data to be used per iteration. - * @param initialWeights Initial set of weights to be used. Array should be equal in size to - * the number of features in the data. - */ - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - regParam: Double, - miniBatchFraction: Double, - initialWeights: Array[Double]) - : RidgeRegressionModel = - { - new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run( - input, initialWeights) - } - - /** - * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. Each iteration uses - * `miniBatchFraction` fraction of the data to calculate the gradient. - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @param stepSize Step size to be used for each iteration of gradient descent. - * @param regParam Regularization parameter. - * @param miniBatchFraction Fraction of data to be used per iteration. - */ - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - regParam: Double, - miniBatchFraction: Double) - : RidgeRegressionModel = - { - new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input) - } - - /** - * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using the specified step size. We use the entire data set to - * update the gradient in each iteration. - * - * @param input RDD of (label, array of features) pairs. - * @param stepSize Step size to be used for each iteration of Gradient Descent. - * @param regParam Regularization parameter. - * @param numIterations Number of iterations of gradient descent to run. - * @return a RidgeRegressionModel which has the weights and offset from training. - */ - def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - regParam: Double) - : RidgeRegressionModel = - { - train(input, numIterations, stepSize, regParam, 1.0) - } - - /** - * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number - * of iterations of gradient descent using a step size of 1.0. We use the entire data set to - * update the gradient in each iteration. - * - * @param input RDD of (label, array of features) pairs. - * @param numIterations Number of iterations of gradient descent to run. - * @return a RidgeRegressionModel which has the weights and offset from training. - */ - def train( - input: RDD[LabeledPoint], - numIterations: Int) - : RidgeRegressionModel = - { - train(input, numIterations, 1.0, 1.0, 1.0) - } - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: RidgeRegression " + - " ") - System.exit(1) - } - val sc = new SparkContext(args(0), "RidgeRegression") - val data = MLUtils.loadLabeledData(sc, args(1)) - val model = RidgeRegressionWithSGD.train(data, args(4).toInt, args(2).toDouble, - args(3).toDouble) - - sc.stop() - } -} diff --git a/mllib/src/main/scala/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/spark/mllib/util/DataValidators.scala deleted file mode 100644 index 57553accf1..0000000000 --- a/mllib/src/main/scala/spark/mllib/util/DataValidators.scala +++ /dev/null @@ -1,42 +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 spark.mllib.util - -import spark.{RDD, Logging} -import spark.mllib.regression.LabeledPoint - -/** - * A collection of methods used to validate data before applying ML algorithms. - */ -object DataValidators extends Logging { - - /** - * Function to check if labels used for classification are either zero or one. - * - * @param data - input data set that needs to be checked - * - * @return True if labels are all zero or one, false otherwise. - */ - val classificationLabels: RDD[LabeledPoint] => Boolean = { data => - val numInvalid = data.filter(x => x.label != 1.0 && x.label != 0.0).count() - if (numInvalid != 0) { - logError("Classification labels should be 0 or 1. Found " + numInvalid + " invalid labels") - } - numInvalid == 0 - } -} diff --git a/mllib/src/main/scala/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/KMeansDataGenerator.scala deleted file mode 100644 index 672b63f65a..0000000000 --- a/mllib/src/main/scala/spark/mllib/util/KMeansDataGenerator.scala +++ /dev/null @@ -1,84 +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 spark.mllib.util - -import scala.util.Random - -import spark.{RDD, SparkContext} - -/** - * Generate test data for KMeans. This class first chooses k cluster centers - * from a d-dimensional Gaussian distribution scaled by factor r and then creates a Gaussian - * cluster with scale 1 around each center. - */ - -object KMeansDataGenerator { - - /** - * Generate an RDD containing test data for KMeans. - * - * @param sc SparkContext to use for creating the RDD - * @param numPoints Number of points that will be contained in the RDD - * @param k Number of clusters - * @param d Number of dimensions - * @param r Scaling factor for the distribution of the initial centers - * @param numPartitions Number of partitions of the generated RDD; default 2 - */ - def generateKMeansRDD( - sc: SparkContext, - numPoints: Int, - k: Int, - d: Int, - r: Double, - numPartitions: Int = 2) - : RDD[Array[Double]] = - { - // First, generate some centers - val rand = new Random(42) - val centers = Array.fill(k)(Array.fill(d)(rand.nextGaussian() * r)) - // Then generate points around each center - sc.parallelize(0 until numPoints, numPartitions).map { idx => - val center = centers(idx % k) - val rand2 = new Random(42 + idx) - Array.tabulate(d)(i => center(i) + rand2.nextGaussian()) - } - } - - def main(args: Array[String]) { - if (args.length < 6) { - println("Usage: KMeansGenerator " + - " []") - System.exit(1) - } - - val sparkMaster = args(0) - val outputPath = args(1) - val numPoints = args(2).toInt - val k = args(3).toInt - val d = args(4).toInt - val r = args(5).toDouble - val parts = if (args.length >= 7) args(6).toInt else 2 - - val sc = new SparkContext(sparkMaster, "KMeansDataGenerator") - val data = generateKMeansRDD(sc, numPoints, k, d, r, parts) - data.map(_.mkString(" ")).saveAsTextFile(outputPath) - - System.exit(0) - } -} - diff --git a/mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala deleted file mode 100644 index 9f48477f84..0000000000 --- a/mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala +++ /dev/null @@ -1,132 +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 spark.mllib.util - -import scala.collection.JavaConversions._ -import scala.util.Random - -import org.jblas.DoubleMatrix - -import spark.{RDD, SparkContext} -import spark.mllib.regression.LabeledPoint -import spark.mllib.regression.LabeledPoint - -/** - * Generate sample data used for Linear Data. This class generates - * uniformly random values for every feature and adds Gaussian noise with mean `eps` to the - * response variable `Y`. - */ -object LinearDataGenerator { - - /** - * Return a Java List of synthetic data randomly generated according to a multi - * collinear model. - * @param intercept Data intercept - * @param weights Weights to be applied. - * @param nPoints Number of points in sample. - * @param seed Random seed - * @return Java List of input. - */ - def generateLinearInputAsList( - intercept: Double, - weights: Array[Double], - nPoints: Int, - seed: Int, - eps: Double): java.util.List[LabeledPoint] = { - seqAsJavaList(generateLinearInput(intercept, weights, nPoints, seed, eps)) - } - - /** - * - * @param intercept Data intercept - * @param weights Weights to be applied. - * @param nPoints Number of points in sample. - * @param seed Random seed - * @param eps Epsilon scaling factor. - * @return - */ - def generateLinearInput( - intercept: Double, - weights: Array[Double], - nPoints: Int, - seed: Int, - eps: Double = 0.1): Seq[LabeledPoint] = { - - val rnd = new Random(seed) - val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) - val x = Array.fill[Array[Double]](nPoints)( - Array.fill[Double](weights.length)(2 * rnd.nextDouble - 1.0)) - 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)) - } - - /** - * Generate an RDD containing sample data for Linear Regression models - including Ridge, Lasso, - * and uregularized variants. - * - * @param sc SparkContext to be used for generating the RDD. - * @param nexamples Number of examples that will be contained in the RDD. - * @param nfeatures Number of features to generate for each example. - * @param eps Epsilon factor by which examples are scaled. - * @param weights Weights associated with the first weights.length features. - * @param nparts Number of partitions in the RDD. Default value is 2. - * - * @return RDD of LabeledPoint containing sample data. - */ - def generateLinearRDD( - sc: SparkContext, - nexamples: Int, - nfeatures: Int, - eps: Double, - nparts: Int = 2, - intercept: Double = 0.0) : RDD[LabeledPoint] = { - org.jblas.util.Random.seed(42) - // Random values distributed uniformly in [-0.5, 0.5] - val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) - - val data: RDD[LabeledPoint] = sc.parallelize(0 until nparts, nparts).flatMap { p => - val seed = 42 + p - val examplesInPartition = nexamples / nparts - generateLinearInput(intercept, w.toArray, examplesInPartition, seed, eps) - } - data - } - - def main(args: Array[String]) { - if (args.length < 2) { - println("Usage: LinearDataGenerator " + - " [num_examples] [num_features] [num_partitions]") - System.exit(1) - } - - val sparkMaster: String = args(0) - val outputPath: String = args(1) - val nexamples: Int = if (args.length > 2) args(2).toInt else 1000 - val nfeatures: Int = if (args.length > 3) args(3).toInt else 100 - val parts: Int = if (args.length > 4) args(4).toInt else 2 - val eps = 10 - - val sc = new SparkContext(sparkMaster, "LinearDataGenerator") - val data = generateLinearRDD(sc, nexamples, nfeatures, eps, nparts = parts) - - MLUtils.saveLabeledData(data, outputPath) - sc.stop() - } -} diff --git a/mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala deleted file mode 100644 index d6402f23e2..0000000000 --- a/mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala +++ /dev/null @@ -1,81 +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 spark.mllib.util - -import scala.util.Random - -import spark.{RDD, SparkContext} -import spark.mllib.regression.LabeledPoint - -/** - * Generate test data for LogisticRegression. This class chooses positive labels - * with probability `probOne` and scales features for positive examples by `eps`. - */ - -object LogisticRegressionDataGenerator { - - /** - * Generate an RDD containing test data for LogisticRegression. - * - * @param sc SparkContext to use for creating the RDD. - * @param nexamples Number of examples that will be contained in the RDD. - * @param nfeatures Number of features to generate for each example. - * @param eps Epsilon factor by which positive examples are scaled. - * @param nparts Number of partitions of the generated RDD. Default value is 2. - * @param probOne Probability that a label is 1 (and not 0). Default value is 0.5. - */ - def generateLogisticRDD( - sc: SparkContext, - nexamples: Int, - nfeatures: Int, - eps: Double, - nparts: Int = 2, - probOne: Double = 0.5): RDD[LabeledPoint] = { - val data = sc.parallelize(0 until nexamples, nparts).map { idx => - val rnd = new Random(42 + idx) - - val y = if (idx % 2 == 0) 0.0 else 1.0 - val x = Array.fill[Double](nfeatures) { - rnd.nextGaussian() + (y * eps) - } - LabeledPoint(y, x) - } - data - } - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: LogisticRegressionGenerator " + - " ") - System.exit(1) - } - - val sparkMaster: String = args(0) - val outputPath: String = args(1) - val nexamples: Int = if (args.length > 2) args(2).toInt else 1000 - val nfeatures: Int = if (args.length > 3) args(3).toInt else 2 - val parts: Int = if (args.length > 4) args(4).toInt else 2 - val eps = 3 - - val sc = new SparkContext(sparkMaster, "LogisticRegressionDataGenerator") - val data = generateLogisticRDD(sc, nexamples, nfeatures, eps, parts) - - MLUtils.saveLabeledData(data, outputPath) - sc.stop() - } -} diff --git a/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala deleted file mode 100644 index 88992cde0c..0000000000 --- a/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala +++ /dev/null @@ -1,113 +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 spark.mllib.recommendation - -import scala.util.Random - -import org.jblas.DoubleMatrix - -import spark.{RDD, SparkContext} -import spark.mllib.util.MLUtils - -/** -* Generate RDD(s) containing data for Matrix Factorization. -* -* This method samples training entries according to the oversampling factor -* 'trainSampFact', which is a multiplicative factor of the number of -* degrees of freedom of the matrix: rank*(m+n-rank). -* -* It optionally samples entries for a testing matrix using -* 'testSampFact', the percentage of the number of training entries -* to use for testing. -* -* This method takes the following inputs: -* sparkMaster (String) The master URL. -* outputPath (String) Directory to save output. -* m (Int) Number of rows in data matrix. -* n (Int) Number of columns in data matrix. -* rank (Int) Underlying rank of data matrix. -* trainSampFact (Double) Oversampling factor. -* noise (Boolean) Whether to add gaussian noise to training data. -* sigma (Double) Standard deviation of added gaussian noise. -* test (Boolean) Whether to create testing RDD. -* testSampFact (Double) Percentage of training data to use as test data. -*/ - -object MFDataGenerator{ - - def main(args: Array[String]) { - if (args.length < 2) { - println("Usage: MFDataGenerator " + - " [m] [n] [rank] [trainSampFact] [noise] [sigma] [test] [testSampFact]") - System.exit(1) - } - - val sparkMaster: String = args(0) - val outputPath: String = args(1) - val m: Int = if (args.length > 2) args(2).toInt else 100 - val n: Int = if (args.length > 3) args(3).toInt else 100 - val rank: Int = if (args.length > 4) args(4).toInt else 10 - val trainSampFact: Double = if (args.length > 5) args(5).toDouble else 1.0 - val noise: Boolean = if (args.length > 6) args(6).toBoolean else false - val sigma: Double = if (args.length > 7) args(7).toDouble else 0.1 - val test: Boolean = if (args.length > 8) args(8).toBoolean else false - val testSampFact: Double = if (args.length > 9) args(9).toDouble else 0.1 - - val sc = new SparkContext(sparkMaster, "MFDataGenerator") - - val A = DoubleMatrix.randn(m, rank) - val B = DoubleMatrix.randn(rank, n) - val z = 1 / (scala.math.sqrt(scala.math.sqrt(rank))) - A.mmuli(z) - B.mmuli(z) - val fullData = A.mmul(B) - - val df = rank * (m + n - rank) - val sampSize = scala.math.min(scala.math.round(trainSampFact * df), - scala.math.round(.99 * m * n)).toInt - val rand = new Random() - val mn = m * n - val shuffled = rand.shuffle(1 to mn toIterable) - - val omega = shuffled.slice(0, sampSize) - val ordered = omega.sortWith(_ < _).toArray - val trainData: RDD[(Int, Int, Double)] = sc.parallelize(ordered) - .map(x => (fullData.indexRows(x - 1), fullData.indexColumns(x - 1), fullData.get(x - 1))) - - // optionally add gaussian noise - if (noise) { - trainData.map(x => (x._1, x._2, x._3 + rand.nextGaussian * sigma)) - } - - trainData.map(x => x._1 + "," + x._2 + "," + x._3).saveAsTextFile(outputPath) - - // optionally generate testing data - if (test) { - val testSampSize = scala.math - .min(scala.math.round(sampSize * testSampFact),scala.math.round(mn - sampSize)).toInt - val testOmega = shuffled.slice(sampSize, sampSize + testSampSize) - val testOrdered = testOmega.sortWith(_ < _).toArray - val testData: RDD[(Int, Int, Double)] = sc.parallelize(testOrdered) - .map(x => (fullData.indexRows(x - 1), fullData.indexColumns(x - 1), fullData.get(x - 1))) - testData.map(x => x._1 + "," + x._2 + "," + x._3).saveAsTextFile(outputPath) - } - - sc.stop() - - } -} \ No newline at end of file diff --git a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala deleted file mode 100644 index a8e6ae9953..0000000000 --- a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala +++ /dev/null @@ -1,122 +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 spark.mllib.util - -import spark.{RDD, SparkContext} -import spark.SparkContext._ - -import org.jblas.DoubleMatrix -import spark.mllib.regression.LabeledPoint - -/** - * Helper methods to load, save and pre-process data used in ML Lib. - */ -object MLUtils { - - /** - * Load labeled data from a file. The data format used here is - * , ... - * where , are feature values in Double and is the corresponding label as Double. - * - * @param sc SparkContext - * @param dir Directory to the input data files. - * @return An RDD of LabeledPoint. Each labeled point has two elements: the first element is - * the label, and the second element represents the feature values (an array of Double). - */ - def loadLabeledData(sc: SparkContext, dir: String): RDD[LabeledPoint] = { - sc.textFile(dir).map { line => - val parts = line.split(',') - val label = parts(0).toDouble - val features = parts(1).trim().split(' ').map(_.toDouble) - LabeledPoint(label, features) - } - } - - /** - * Save labeled data to a file. The data format used here is - * , ... - * where , are feature values in Double and is the corresponding label as Double. - * - * @param data An RDD of LabeledPoints containing data to be saved. - * @param dir Directory to save the data. - */ - def saveLabeledData(data: RDD[LabeledPoint], dir: String) { - val dataStr = data.map(x => x.label + "," + x.features.mkString(" ")) - dataStr.saveAsTextFile(dir) - } - - /** - * 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 - * - * @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 - - // 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 xColSumsMap = xColSumSq.collectAsMap() - - val xColMean = DoubleMatrix.zeros(nfeatures, 1) - val xColSd = DoubleMatrix.zeros(nfeatures, 1) - - // 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 - } - - (yMean, xColMean, xColSd) - } - - /** - * Return the squared Euclidean distance between two vectors. - */ - def squaredDistance(v1: Array[Double], v2: Array[Double]): Double = { - if (v1.length != v2.length) { - throw new IllegalArgumentException("Vector sizes don't match") - } - var i = 0 - var sum = 0.0 - while (i < v1.length) { - sum += (v1(i) - v2(i)) * (v1(i) - v2(i)) - i += 1 - } - sum - } -} diff --git a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala deleted file mode 100644 index eff456cad6..0000000000 --- a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala +++ /dev/null @@ -1,50 +0,0 @@ -package spark.mllib.util - -import scala.util.Random - -import spark.{RDD, SparkContext} - -import org.jblas.DoubleMatrix -import spark.mllib.regression.LabeledPoint - -/** - * Generate sample data used for SVM. This class generates uniform random values - * for the features and adds Gaussian noise with weight 0.1 to generate labels. - */ -object SVMDataGenerator { - - def main(args: Array[String]) { - if (args.length < 2) { - println("Usage: SVMGenerator " + - " [num_examples] [num_features] [num_partitions]") - System.exit(1) - } - - val sparkMaster: String = args(0) - val outputPath: String = args(1) - val nexamples: Int = if (args.length > 2) args(2).toInt else 1000 - val nfeatures: Int = if (args.length > 3) args(3).toInt else 2 - val parts: Int = if (args.length > 4) args(4).toInt else 2 - - val sc = new SparkContext(sparkMaster, "SVMGenerator") - - val globalRnd = new Random(94720) - val trueWeights = new DoubleMatrix(1, nfeatures + 1, - Array.fill[Double](nfeatures + 1)(globalRnd.nextGaussian()):_*) - - val data: RDD[LabeledPoint] = sc.parallelize(0 until nexamples, parts).map { idx => - val rnd = new Random(42 + idx) - - val x = Array.fill[Double](nfeatures) { - rnd.nextDouble() * 2.0 - 1.0 - } - 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) - } - - MLUtils.saveLabeledData(data, outputPath) - - sc.stop() - } -} diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java new file mode 100644 index 0000000000..e18e3bc6a8 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java @@ -0,0 +1,98 @@ +/* + * 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.classification; + +import java.io.Serializable; +import java.util.List; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +import org.apache.spark.mllib.regression.LabeledPoint; + +public class JavaLogisticRegressionSuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaLogisticRegressionSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + System.clearProperty("spark.driver.port"); + } + + int validatePrediction(List validationData, LogisticRegressionModel model) { + int numAccurate = 0; + for (LabeledPoint point: validationData) { + Double prediction = model.predict(point.features()); + if (prediction == point.label()) { + numAccurate++; + } + } + return numAccurate; + } + + @Test + public void runLRUsingConstructor() { + int nPoints = 10000; + double A = 2.0; + double B = -1.5; + + JavaRDD testRDD = sc.parallelize( + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); + List validationData = + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 17); + + LogisticRegressionWithSGD lrImpl = new LogisticRegressionWithSGD(); + lrImpl.optimizer().setStepSize(1.0) + .setRegParam(1.0) + .setNumIterations(100); + LogisticRegressionModel model = lrImpl.run(testRDD.rdd()); + + int numAccurate = validatePrediction(validationData, model); + Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); + } + + @Test + public void runLRUsingStaticMethods() { + int nPoints = 10000; + double A = 2.0; + double B = -1.5; + + JavaRDD testRDD = sc.parallelize( + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); + List validationData = + LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 17); + + LogisticRegressionModel model = LogisticRegressionWithSGD.train( + testRDD.rdd(), 100, 1.0, 1.0); + + int numAccurate = validatePrediction(validationData, model); + Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); + } + +} 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 new file mode 100644 index 0000000000..117e5eaa8b --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java @@ -0,0 +1,98 @@ +/* + * 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.classification; + + +import java.io.Serializable; +import java.util.List; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +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 { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaSVMSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + System.clearProperty("spark.driver.port"); + } + + int validatePrediction(List validationData, SVMModel model) { + int numAccurate = 0; + for (LabeledPoint point: validationData) { + Double prediction = model.predict(point.features()); + if (prediction == point.label()) { + numAccurate++; + } + } + return numAccurate; + } + + @Test + public void runSVMUsingConstructor() { + int nPoints = 10000; + double A = 2.0; + double[] weights = {-1.5, 1.0}; + + JavaRDD testRDD = sc.parallelize(SVMSuite.generateSVMInputAsList(A, + weights, nPoints, 42), 2).cache(); + List validationData = + SVMSuite.generateSVMInputAsList(A, weights, nPoints, 17); + + SVMWithSGD svmSGDImpl = new SVMWithSGD(); + svmSGDImpl.optimizer().setStepSize(1.0) + .setRegParam(1.0) + .setNumIterations(100); + SVMModel model = svmSGDImpl.run(testRDD.rdd()); + + int numAccurate = validatePrediction(validationData, model); + Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); + } + + @Test + public void runSVMUsingStaticMethods() { + int nPoints = 10000; + double A = 2.0; + double[] weights = {-1.5, 1.0}; + + JavaRDD testRDD = sc.parallelize(SVMSuite.generateSVMInputAsList(A, + weights, nPoints, 42), 2).cache(); + List validationData = + SVMSuite.generateSVMInputAsList(A, weights, nPoints, 17); + + SVMModel model = SVMWithSGD.train(testRDD.rdd(), 100, 1.0, 1.0, 1.0); + + int numAccurate = validatePrediction(validationData, model); + Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); + } + +} diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java new file mode 100644 index 0000000000..32d3934ac1 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.clustering; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +public class JavaKMeansSuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaKMeans"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + System.clearProperty("spark.driver.port"); + } + + // L1 distance between two points + double distance1(double[] v1, double[] v2) { + double distance = 0.0; + for (int i = 0; i < v1.length; ++i) { + distance = Math.max(distance, Math.abs(v1[i] - v2[i])); + } + return distance; + } + + // Assert that two sets of points are equal, within EPSILON tolerance + void assertSetsEqual(double[][] v1, double[][] v2) { + double EPSILON = 1e-4; + Assert.assertTrue(v1.length == v2.length); + for (int i = 0; i < v1.length; ++i) { + double minDistance = Double.MAX_VALUE; + for (int j = 0; j < v2.length; ++j) { + minDistance = Math.min(minDistance, distance1(v1[i], v2[j])); + } + Assert.assertTrue(minDistance <= EPSILON); + } + + for (int i = 0; i < v2.length; ++i) { + double minDistance = Double.MAX_VALUE; + for (int j = 0; j < v1.length; ++j) { + minDistance = Math.min(minDistance, distance1(v2[i], v1[j])); + } + Assert.assertTrue(minDistance <= EPSILON); + } + } + + + @Test + public void runKMeansUsingStaticMethods() { + List points = new ArrayList(); + points.add(new double[]{1.0, 2.0, 6.0}); + points.add(new double[]{1.0, 3.0, 0.0}); + points.add(new double[]{1.0, 4.0, 6.0}); + + double[][] expectedCenter = { {1.0, 3.0, 4.0} }; + + JavaRDD data = sc.parallelize(points, 2); + KMeansModel model = KMeans.train(data.rdd(), 1, 1); + assertSetsEqual(model.clusterCenters(), expectedCenter); + + model = KMeans.train(data.rdd(), 1, 1, 1, KMeans.RANDOM()); + assertSetsEqual(model.clusterCenters(), expectedCenter); + } + + @Test + public void runKMeansUsingConstructor() { + List points = new ArrayList(); + points.add(new double[]{1.0, 2.0, 6.0}); + points.add(new double[]{1.0, 3.0, 0.0}); + points.add(new double[]{1.0, 4.0, 6.0}); + + double[][] expectedCenter = { {1.0, 3.0, 4.0} }; + + JavaRDD data = sc.parallelize(points, 2); + KMeansModel model = new KMeans().setK(1).setMaxIterations(5).run(data.rdd()); + assertSetsEqual(model.clusterCenters(), expectedCenter); + + model = new KMeans().setK(1) + .setMaxIterations(1) + .setRuns(1) + .setInitializationMode(KMeans.RANDOM()) + .run(data.rdd()); + assertSetsEqual(model.clusterCenters(), expectedCenter); + } +} diff --git a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java new file mode 100644 index 0000000000..3323f6cee2 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java @@ -0,0 +1,110 @@ +/* + * 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.recommendation; + +import java.io.Serializable; +import java.util.List; + +import scala.Tuple2; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +import org.jblas.DoubleMatrix; + +public class JavaALSSuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaALS"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + System.clearProperty("spark.driver.port"); + } + + void validatePrediction(MatrixFactorizationModel model, int users, int products, int features, + DoubleMatrix trueRatings, double matchThreshold) { + DoubleMatrix predictedU = new DoubleMatrix(users, features); + List> userFeatures = model.userFeatures().toJavaRDD().collect(); + for (int i = 0; i < features; ++i) { + for (scala.Tuple2 userFeature : userFeatures) { + predictedU.put((Integer)userFeature._1(), i, userFeature._2()[i]); + } + } + DoubleMatrix predictedP = new DoubleMatrix(products, features); + + List> productFeatures = + model.productFeatures().toJavaRDD().collect(); + for (int i = 0; i < features; ++i) { + for (scala.Tuple2 productFeature : productFeatures) { + predictedP.put((Integer)productFeature._1(), i, productFeature._2()[i]); + } + } + + DoubleMatrix predictedRatings = predictedU.mmul(predictedP.transpose()); + + for (int u = 0; u < users; ++u) { + for (int p = 0; p < products; ++p) { + double prediction = predictedRatings.get(u, p); + double correct = trueRatings.get(u, p); + Assert.assertTrue(Math.abs(prediction - correct) < matchThreshold); + } + } + } + + @Test + public void runALSUsingStaticMethods() { + int features = 1; + int iterations = 15; + int users = 10; + int products = 10; + scala.Tuple2, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( + users, products, features, 0.7); + + JavaRDD data = sc.parallelize(testData._1()); + MatrixFactorizationModel model = ALS.train(data.rdd(), features, iterations); + validatePrediction(model, users, products, features, testData._2(), 0.3); + } + + @Test + public void runALSUsingConstructor() { + int features = 2; + int iterations = 15; + int users = 20; + int products = 30; + scala.Tuple2, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( + users, products, features, 0.7); + + JavaRDD data = sc.parallelize(testData._1()); + + MatrixFactorizationModel model = new ALS().setRank(features) + .setIterations(iterations) + .run(data.rdd()); + validatePrediction(model, users, products, features, testData._2(), 0.3); + } +} 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 new file mode 100644 index 0000000000..f44b25cd44 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java @@ -0,0 +1,97 @@ +/* + * 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.regression; + +import java.io.Serializable; +import java.util.List; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.util.LinearDataGenerator; + +public class JavaLassoSuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaLassoSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + System.clearProperty("spark.driver.port"); + } + + int validatePrediction(List validationData, LassoModel model) { + int numAccurate = 0; + for (LabeledPoint point: validationData) { + Double prediction = model.predict(point.features()); + // A prediction is off if the prediction is more than 0.5 away from expected value. + if (Math.abs(prediction - point.label()) <= 0.5) { + numAccurate++; + } + } + return numAccurate; + } + + @Test + public void runLassoUsingConstructor() { + int nPoints = 10000; + double A = 2.0; + double[] weights = {-1.5, 1.0e-2}; + + JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, + weights, nPoints, 42, 0.1), 2).cache(); + List validationData = + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); + + LassoWithSGD lassoSGDImpl = new LassoWithSGD(); + lassoSGDImpl.optimizer().setStepSize(1.0) + .setRegParam(0.01) + .setNumIterations(20); + LassoModel model = lassoSGDImpl.run(testRDD.rdd()); + + int numAccurate = validatePrediction(validationData, model); + Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); + } + + @Test + public void runLassoUsingStaticMethods() { + int nPoints = 10000; + double A = 2.0; + double[] weights = {-1.5, 1.0e-2}; + + JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, + weights, nPoints, 42, 0.1), 2).cache(); + List validationData = + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); + + LassoModel model = LassoWithSGD.train(testRDD.rdd(), 100, 1.0, 0.01, 1.0); + + int numAccurate = validatePrediction(validationData, model); + Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); + } + +} diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java new file mode 100644 index 0000000000..5a4410a632 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java @@ -0,0 +1,94 @@ +/* + * 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.regression; + +import java.io.Serializable; +import java.util.List; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.util.LinearDataGenerator; + +public class JavaLinearRegressionSuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaLinearRegressionSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + System.clearProperty("spark.driver.port"); + } + + int validatePrediction(List validationData, LinearRegressionModel model) { + int numAccurate = 0; + for (LabeledPoint point: validationData) { + Double prediction = model.predict(point.features()); + // A prediction is off if the prediction is more than 0.5 away from expected value. + if (Math.abs(prediction - point.label()) <= 0.5) { + numAccurate++; + } + } + return numAccurate; + } + + @Test + public void runLinearRegressionUsingConstructor() { + int nPoints = 100; + double A = 3.0; + double[] weights = {10, 10}; + + JavaRDD testRDD = sc.parallelize( + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); + List validationData = + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); + + LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(); + LinearRegressionModel model = linSGDImpl.run(testRDD.rdd()); + + int numAccurate = validatePrediction(validationData, model); + Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); + } + + @Test + public void runLinearRegressionUsingStaticMethods() { + int nPoints = 100; + double A = 3.0; + double[] weights = {10, 10}; + + JavaRDD testRDD = sc.parallelize( + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); + List validationData = + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); + + LinearRegressionModel model = LinearRegressionWithSGD.train(testRDD.rdd(), 100); + + int numAccurate = validatePrediction(validationData, model); + Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.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 new file mode 100644 index 0000000000..2fdd5fc8fd --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java @@ -0,0 +1,110 @@ +/* + * 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.regression; + +import java.io.Serializable; +import java.util.List; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.jblas.DoubleMatrix; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.util.LinearDataGenerator; + +public class JavaRidgeRegressionSuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaRidgeRegressionSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + System.clearProperty("spark.driver.port"); + } + + double predictionError(List validationData, RidgeRegressionModel model) { + double errorSum = 0; + for (LabeledPoint point: validationData) { + Double prediction = model.predict(point.features()); + errorSum += (prediction - point.label()) * (prediction - point.label()); + } + return errorSum / validationData.size(); + } + + List generateRidgeData(int numPoints, int nfeatures, double eps) { + 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); + } + + @Test + public void runRidgeRegressionUsingConstructor() { + int nexamples = 200; + int nfeatures = 20; + double eps = 10.0; + List data = generateRidgeData(2*nexamples, nfeatures, eps); + + JavaRDD testRDD = sc.parallelize(data.subList(0, nexamples)); + List validationData = data.subList(nexamples, 2*nexamples); + + RidgeRegressionWithSGD ridgeSGDImpl = new RidgeRegressionWithSGD(); + ridgeSGDImpl.optimizer().setStepSize(1.0) + .setRegParam(0.0) + .setNumIterations(200); + RidgeRegressionModel model = ridgeSGDImpl.run(testRDD.rdd()); + double unRegularizedErr = predictionError(validationData, model); + + ridgeSGDImpl.optimizer().setRegParam(0.1); + model = ridgeSGDImpl.run(testRDD.rdd()); + double regularizedErr = predictionError(validationData, model); + + Assert.assertTrue(regularizedErr < unRegularizedErr); + } + + @Test + public void runRidgeRegressionUsingStaticMethods() { + int nexamples = 200; + int nfeatures = 20; + double eps = 10.0; + List data = generateRidgeData(2*nexamples, nfeatures, eps); + + JavaRDD testRDD = sc.parallelize(data.subList(0, nexamples)); + List validationData = data.subList(nexamples, 2*nexamples); + + RidgeRegressionModel model = RidgeRegressionWithSGD.train(testRDD.rdd(), 200, 1.0, 0.0); + double unRegularizedErr = predictionError(validationData, model); + + model = RidgeRegressionWithSGD.train(testRDD.rdd(), 200, 1.0, 0.1); + double regularizedErr = predictionError(validationData, model); + + Assert.assertTrue(regularizedErr < unRegularizedErr); + } +} diff --git a/mllib/src/test/java/spark/mllib/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/spark/mllib/classification/JavaLogisticRegressionSuite.java deleted file mode 100644 index e0ebd45cd8..0000000000 --- a/mllib/src/test/java/spark/mllib/classification/JavaLogisticRegressionSuite.java +++ /dev/null @@ -1,98 +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 spark.mllib.classification; - -import java.io.Serializable; -import java.util.List; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; - -import spark.mllib.regression.LabeledPoint; - -public class JavaLogisticRegressionSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaLogisticRegressionSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - System.clearProperty("spark.driver.port"); - } - - int validatePrediction(List validationData, LogisticRegressionModel model) { - int numAccurate = 0; - for (LabeledPoint point: validationData) { - Double prediction = model.predict(point.features()); - if (prediction == point.label()) { - numAccurate++; - } - } - return numAccurate; - } - - @Test - public void runLRUsingConstructor() { - int nPoints = 10000; - double A = 2.0; - double B = -1.5; - - JavaRDD testRDD = sc.parallelize( - LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); - List validationData = - LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 17); - - LogisticRegressionWithSGD lrImpl = new LogisticRegressionWithSGD(); - lrImpl.optimizer().setStepSize(1.0) - .setRegParam(1.0) - .setNumIterations(100); - LogisticRegressionModel model = lrImpl.run(testRDD.rdd()); - - int numAccurate = validatePrediction(validationData, model); - Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); - } - - @Test - public void runLRUsingStaticMethods() { - int nPoints = 10000; - double A = 2.0; - double B = -1.5; - - JavaRDD testRDD = sc.parallelize( - LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); - List validationData = - LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 17); - - LogisticRegressionModel model = LogisticRegressionWithSGD.train( - testRDD.rdd(), 100, 1.0, 1.0); - - int numAccurate = validatePrediction(validationData, model); - Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); - } - -} diff --git a/mllib/src/test/java/spark/mllib/classification/JavaSVMSuite.java b/mllib/src/test/java/spark/mllib/classification/JavaSVMSuite.java deleted file mode 100644 index 7881b3c38f..0000000000 --- a/mllib/src/test/java/spark/mllib/classification/JavaSVMSuite.java +++ /dev/null @@ -1,98 +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 spark.mllib.classification; - - -import java.io.Serializable; -import java.util.List; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; - -import spark.mllib.regression.LabeledPoint; - -public class JavaSVMSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaSVMSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - System.clearProperty("spark.driver.port"); - } - - int validatePrediction(List validationData, SVMModel model) { - int numAccurate = 0; - for (LabeledPoint point: validationData) { - Double prediction = model.predict(point.features()); - if (prediction == point.label()) { - numAccurate++; - } - } - return numAccurate; - } - - @Test - public void runSVMUsingConstructor() { - int nPoints = 10000; - double A = 2.0; - double[] weights = {-1.5, 1.0}; - - JavaRDD testRDD = sc.parallelize(SVMSuite.generateSVMInputAsList(A, - weights, nPoints, 42), 2).cache(); - List validationData = - SVMSuite.generateSVMInputAsList(A, weights, nPoints, 17); - - SVMWithSGD svmSGDImpl = new SVMWithSGD(); - svmSGDImpl.optimizer().setStepSize(1.0) - .setRegParam(1.0) - .setNumIterations(100); - SVMModel model = svmSGDImpl.run(testRDD.rdd()); - - int numAccurate = validatePrediction(validationData, model); - Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); - } - - @Test - public void runSVMUsingStaticMethods() { - int nPoints = 10000; - double A = 2.0; - double[] weights = {-1.5, 1.0}; - - JavaRDD testRDD = sc.parallelize(SVMSuite.generateSVMInputAsList(A, - weights, nPoints, 42), 2).cache(); - List validationData = - SVMSuite.generateSVMInputAsList(A, weights, nPoints, 17); - - SVMModel model = SVMWithSGD.train(testRDD.rdd(), 100, 1.0, 1.0, 1.0); - - int numAccurate = validatePrediction(validationData, model); - Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); - } - -} diff --git a/mllib/src/test/java/spark/mllib/clustering/JavaKMeansSuite.java b/mllib/src/test/java/spark/mllib/clustering/JavaKMeansSuite.java deleted file mode 100644 index 3f2d82bfb4..0000000000 --- a/mllib/src/test/java/spark/mllib/clustering/JavaKMeansSuite.java +++ /dev/null @@ -1,115 +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 spark.mllib.clustering; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; - -public class JavaKMeansSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaKMeans"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - System.clearProperty("spark.driver.port"); - } - - // L1 distance between two points - double distance1(double[] v1, double[] v2) { - double distance = 0.0; - for (int i = 0; i < v1.length; ++i) { - distance = Math.max(distance, Math.abs(v1[i] - v2[i])); - } - return distance; - } - - // Assert that two sets of points are equal, within EPSILON tolerance - void assertSetsEqual(double[][] v1, double[][] v2) { - double EPSILON = 1e-4; - Assert.assertTrue(v1.length == v2.length); - for (int i = 0; i < v1.length; ++i) { - double minDistance = Double.MAX_VALUE; - for (int j = 0; j < v2.length; ++j) { - minDistance = Math.min(minDistance, distance1(v1[i], v2[j])); - } - Assert.assertTrue(minDistance <= EPSILON); - } - - for (int i = 0; i < v2.length; ++i) { - double minDistance = Double.MAX_VALUE; - for (int j = 0; j < v1.length; ++j) { - minDistance = Math.min(minDistance, distance1(v2[i], v1[j])); - } - Assert.assertTrue(minDistance <= EPSILON); - } - } - - - @Test - public void runKMeansUsingStaticMethods() { - List points = new ArrayList(); - points.add(new double[]{1.0, 2.0, 6.0}); - points.add(new double[]{1.0, 3.0, 0.0}); - points.add(new double[]{1.0, 4.0, 6.0}); - - double[][] expectedCenter = { {1.0, 3.0, 4.0} }; - - JavaRDD data = sc.parallelize(points, 2); - KMeansModel model = KMeans.train(data.rdd(), 1, 1); - assertSetsEqual(model.clusterCenters(), expectedCenter); - - model = KMeans.train(data.rdd(), 1, 1, 1, KMeans.RANDOM()); - assertSetsEqual(model.clusterCenters(), expectedCenter); - } - - @Test - public void runKMeansUsingConstructor() { - List points = new ArrayList(); - points.add(new double[]{1.0, 2.0, 6.0}); - points.add(new double[]{1.0, 3.0, 0.0}); - points.add(new double[]{1.0, 4.0, 6.0}); - - double[][] expectedCenter = { {1.0, 3.0, 4.0} }; - - JavaRDD data = sc.parallelize(points, 2); - KMeansModel model = new KMeans().setK(1).setMaxIterations(5).run(data.rdd()); - assertSetsEqual(model.clusterCenters(), expectedCenter); - - model = new KMeans().setK(1) - .setMaxIterations(1) - .setRuns(1) - .setInitializationMode(KMeans.RANDOM()) - .run(data.rdd()); - assertSetsEqual(model.clusterCenters(), expectedCenter); - } -} diff --git a/mllib/src/test/java/spark/mllib/recommendation/JavaALSSuite.java b/mllib/src/test/java/spark/mllib/recommendation/JavaALSSuite.java deleted file mode 100644 index 7993629a6d..0000000000 --- a/mllib/src/test/java/spark/mllib/recommendation/JavaALSSuite.java +++ /dev/null @@ -1,110 +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 spark.mllib.recommendation; - -import java.io.Serializable; -import java.util.List; - -import scala.Tuple2; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; - -import org.jblas.DoubleMatrix; - -public class JavaALSSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaALS"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - System.clearProperty("spark.driver.port"); - } - - void validatePrediction(MatrixFactorizationModel model, int users, int products, int features, - DoubleMatrix trueRatings, double matchThreshold) { - DoubleMatrix predictedU = new DoubleMatrix(users, features); - List> userFeatures = model.userFeatures().toJavaRDD().collect(); - for (int i = 0; i < features; ++i) { - for (scala.Tuple2 userFeature : userFeatures) { - predictedU.put((Integer)userFeature._1(), i, userFeature._2()[i]); - } - } - DoubleMatrix predictedP = new DoubleMatrix(products, features); - - List> productFeatures = - model.productFeatures().toJavaRDD().collect(); - for (int i = 0; i < features; ++i) { - for (scala.Tuple2 productFeature : productFeatures) { - predictedP.put((Integer)productFeature._1(), i, productFeature._2()[i]); - } - } - - DoubleMatrix predictedRatings = predictedU.mmul(predictedP.transpose()); - - for (int u = 0; u < users; ++u) { - for (int p = 0; p < products; ++p) { - double prediction = predictedRatings.get(u, p); - double correct = trueRatings.get(u, p); - Assert.assertTrue(Math.abs(prediction - correct) < matchThreshold); - } - } - } - - @Test - public void runALSUsingStaticMethods() { - int features = 1; - int iterations = 15; - int users = 10; - int products = 10; - scala.Tuple2, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( - users, products, features, 0.7); - - JavaRDD data = sc.parallelize(testData._1()); - MatrixFactorizationModel model = ALS.train(data.rdd(), features, iterations); - validatePrediction(model, users, products, features, testData._2(), 0.3); - } - - @Test - public void runALSUsingConstructor() { - int features = 2; - int iterations = 15; - int users = 20; - int products = 30; - scala.Tuple2, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( - users, products, features, 0.7); - - JavaRDD data = sc.parallelize(testData._1()); - - MatrixFactorizationModel model = new ALS().setRank(features) - .setIterations(iterations) - .run(data.rdd()); - validatePrediction(model, users, products, features, testData._2(), 0.3); - } -} diff --git a/mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java b/mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java deleted file mode 100644 index 5863140baf..0000000000 --- a/mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java +++ /dev/null @@ -1,97 +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 spark.mllib.regression; - -import java.io.Serializable; -import java.util.List; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; -import spark.mllib.util.LinearDataGenerator; - -public class JavaLassoSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaLassoSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - System.clearProperty("spark.driver.port"); - } - - int validatePrediction(List validationData, LassoModel model) { - int numAccurate = 0; - for (LabeledPoint point: validationData) { - Double prediction = model.predict(point.features()); - // A prediction is off if the prediction is more than 0.5 away from expected value. - if (Math.abs(prediction - point.label()) <= 0.5) { - numAccurate++; - } - } - return numAccurate; - } - - @Test - public void runLassoUsingConstructor() { - int nPoints = 10000; - double A = 2.0; - double[] weights = {-1.5, 1.0e-2}; - - JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, - weights, nPoints, 42, 0.1), 2).cache(); - List validationData = - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); - - LassoWithSGD lassoSGDImpl = new LassoWithSGD(); - lassoSGDImpl.optimizer().setStepSize(1.0) - .setRegParam(0.01) - .setNumIterations(20); - LassoModel model = lassoSGDImpl.run(testRDD.rdd()); - - int numAccurate = validatePrediction(validationData, model); - Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); - } - - @Test - public void runLassoUsingStaticMethods() { - int nPoints = 10000; - double A = 2.0; - double[] weights = {-1.5, 1.0e-2}; - - JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, - weights, nPoints, 42, 0.1), 2).cache(); - List validationData = - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); - - LassoModel model = LassoWithSGD.train(testRDD.rdd(), 100, 1.0, 0.01, 1.0); - - int numAccurate = validatePrediction(validationData, model); - Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); - } - -} diff --git a/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java deleted file mode 100644 index 50716c7861..0000000000 --- a/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java +++ /dev/null @@ -1,94 +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 spark.mllib.regression; - -import java.io.Serializable; -import java.util.List; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; -import spark.mllib.util.LinearDataGenerator; - -public class JavaLinearRegressionSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaLinearRegressionSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - System.clearProperty("spark.driver.port"); - } - - int validatePrediction(List validationData, LinearRegressionModel model) { - int numAccurate = 0; - for (LabeledPoint point: validationData) { - Double prediction = model.predict(point.features()); - // A prediction is off if the prediction is more than 0.5 away from expected value. - if (Math.abs(prediction - point.label()) <= 0.5) { - numAccurate++; - } - } - return numAccurate; - } - - @Test - public void runLinearRegressionUsingConstructor() { - int nPoints = 100; - double A = 3.0; - double[] weights = {10, 10}; - - JavaRDD testRDD = sc.parallelize( - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); - List validationData = - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); - - LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(); - LinearRegressionModel model = linSGDImpl.run(testRDD.rdd()); - - int numAccurate = validatePrediction(validationData, model); - Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); - } - - @Test - public void runLinearRegressionUsingStaticMethods() { - int nPoints = 100; - double A = 3.0; - double[] weights = {10, 10}; - - JavaRDD testRDD = sc.parallelize( - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); - List validationData = - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); - - LinearRegressionModel model = LinearRegressionWithSGD.train(testRDD.rdd(), 100); - - int numAccurate = validatePrediction(validationData, model); - Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); - } - -} diff --git a/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java b/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java deleted file mode 100644 index 2c0aabad30..0000000000 --- a/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java +++ /dev/null @@ -1,110 +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 spark.mllib.regression; - -import java.io.Serializable; -import java.util.List; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import org.jblas.DoubleMatrix; - -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; -import spark.mllib.util.LinearDataGenerator; - -public class JavaRidgeRegressionSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaRidgeRegressionSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - System.clearProperty("spark.driver.port"); - } - - double predictionError(List validationData, RidgeRegressionModel model) { - double errorSum = 0; - for (LabeledPoint point: validationData) { - Double prediction = model.predict(point.features()); - errorSum += (prediction - point.label()) * (prediction - point.label()); - } - return errorSum / validationData.size(); - } - - List generateRidgeData(int numPoints, int nfeatures, double eps) { - 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); - } - - @Test - public void runRidgeRegressionUsingConstructor() { - int nexamples = 200; - int nfeatures = 20; - double eps = 10.0; - List data = generateRidgeData(2*nexamples, nfeatures, eps); - - JavaRDD testRDD = sc.parallelize(data.subList(0, nexamples)); - List validationData = data.subList(nexamples, 2*nexamples); - - RidgeRegressionWithSGD ridgeSGDImpl = new RidgeRegressionWithSGD(); - ridgeSGDImpl.optimizer().setStepSize(1.0) - .setRegParam(0.0) - .setNumIterations(200); - RidgeRegressionModel model = ridgeSGDImpl.run(testRDD.rdd()); - double unRegularizedErr = predictionError(validationData, model); - - ridgeSGDImpl.optimizer().setRegParam(0.1); - model = ridgeSGDImpl.run(testRDD.rdd()); - double regularizedErr = predictionError(validationData, model); - - Assert.assertTrue(regularizedErr < unRegularizedErr); - } - - @Test - public void runRidgeRegressionUsingStaticMethods() { - int nexamples = 200; - int nfeatures = 20; - double eps = 10.0; - List data = generateRidgeData(2*nexamples, nfeatures, eps); - - JavaRDD testRDD = sc.parallelize(data.subList(0, nexamples)); - List validationData = data.subList(nexamples, 2*nexamples); - - RidgeRegressionModel model = RidgeRegressionWithSGD.train(testRDD.rdd(), 200, 1.0, 0.0); - double unRegularizedErr = predictionError(validationData, model); - - model = RidgeRegressionWithSGD.train(testRDD.rdd(), 200, 1.0, 0.1); - double regularizedErr = predictionError(validationData, model); - - Assert.assertTrue(regularizedErr < unRegularizedErr); - } -} 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 new file mode 100644 index 0000000000..34c67294e9 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -0,0 +1,150 @@ +/* + * 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.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.regression._ + +object LogisticRegressionSuite { + + def generateLogisticInputAsList( + offset: Double, + scale: Double, + nPoints: Int, + seed: Int): java.util.List[LabeledPoint] = { + seqAsJavaList(generateLogisticInput(offset, scale, nPoints, seed)) + } + + // Generate input of the form Y = logistic(offset + scale*X) + def generateLogisticInput( + offset: Double, + scale: Double, + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val rnd = new Random(seed) + val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + + // NOTE: if U is uniform[0, 1] then ln(u) - ln(1-u) is Logistic(0,1) + val unifRand = new scala.util.Random(45) + val rLogis = (0 until nPoints).map { i => + val u = unifRand.nextDouble() + math.log(u) - math.log(1.0-u) + } + + // y <- A + B*x + rLogis() + // y <- as.numeric(y > 0) + val y: Seq[Int] = (0 until nPoints).map { i => + val yVal = offset + scale * x1(i) + rLogis(i) + if (yVal > 0) 1 else 0 + } + + val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Array(x1(i)))) + testData + } + +} + +class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll with ShouldMatchers { + @transient private var sc: SparkContext = _ + + override def beforeAll() { + sc = new SparkContext("local", "test") + } + + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) { + val numOffPredictions = predictions.zip(input).filter { case (prediction, expected) => + (prediction != expected.label) + }.size + // At least 83% of the predictions should be on. + ((input.length - numOffPredictions).toDouble / input.length) should be > 0.83 + } + + // Test if we can correctly learn A, B where Y = logistic(A + B*X) + test("logistic regression") { + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 42) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + val lr = new LogisticRegressionWithSGD() + lr.optimizer.setStepSize(10.0).setNumIterations(20) + + val model = lr.run(testRDD) + + // Test the weights + val weight0 = model.weights(0) + assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") + assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + + val validationData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + // 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) + } + + test("logistic regression with initial weights") { + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 42) + + val initialB = -1.0 + val initialWeights = Array(initialB) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + // Use half as many iterations as the previous test. + val lr = new LogisticRegressionWithSGD() + lr.optimizer.setStepSize(10.0).setNumIterations(10) + + val model = lr.run(testRDD, initialWeights) + + val weight0 = model.weights(0) + assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") + assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + + val validationData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + // 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) + } +} 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 new file mode 100644 index 0000000000..6a957e3ddc --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala @@ -0,0 +1,169 @@ +/* + * 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.classification + +import scala.util.Random +import scala.math.signum +import scala.collection.JavaConversions._ + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import org.jblas.DoubleMatrix + +import org.apache.spark.{SparkException, SparkContext} +import org.apache.spark.mllib.regression._ + +object SVMSuite { + + def generateSVMInputAsList( + intercept: Double, + weights: Array[Double], + nPoints: Int, + seed: Int): java.util.List[LabeledPoint] = { + seqAsJavaList(generateSVMInput(intercept, weights, nPoints, seed)) + } + + // Generate noisy input of the form Y = signum(x.dot(weights) + intercept + noise) + def generateSVMInput( + intercept: Double, + weights: Array[Double], + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val rnd = new Random(seed) + val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) + val x = Array.fill[Array[Double]](nPoints)( + Array.fill[Double](weights.length)(rnd.nextDouble() * 2.0 - 1.0)) + val y = x.map { xi => + val yD = (new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) + + intercept + 0.01 * rnd.nextGaussian() + if (yD < 0) 0.0 else 1.0 + } + y.zip(x).map(p => LabeledPoint(p._1, p._2)) + } + +} + +class SVMSuite extends FunSuite with BeforeAndAfterAll { + @transient private var sc: SparkContext = _ + + override def beforeAll() { + sc = new SparkContext("local", "test") + } + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) { + val numOffPredictions = predictions.zip(input).filter { case (prediction, expected) => + (prediction != expected.label) + }.size + // At least 80% of the predictions should be on. + assert(numOffPredictions < input.length / 5) + } + + + test("SVM using local random SGD") { + val nPoints = 10000 + + // NOTE: Intercept should be small for generating equal 0s and 1s + val A = 0.01 + val B = -1.5 + val C = 1.0 + + val testData = SVMSuite.generateSVMInput(A, Array[Double](B,C), nPoints, 42) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + val svm = new SVMWithSGD() + svm.optimizer.setStepSize(1.0).setRegParam(1.0).setNumIterations(100) + + val model = svm.run(testRDD) + + val validationData = SVMSuite.generateSVMInput(A, Array[Double](B,C), nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + + // 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) + } + + test("SVM local random SGD with initial weights") { + val nPoints = 10000 + + // NOTE: Intercept should be small for generating equal 0s and 1s + val A = 0.01 + val B = -1.5 + val C = 1.0 + + val testData = SVMSuite.generateSVMInput(A, Array[Double](B,C), nPoints, 42) + + val initialB = -1.0 + val initialC = -1.0 + val initialWeights = Array(initialB,initialC) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + val svm = new SVMWithSGD() + svm.optimizer.setStepSize(1.0).setRegParam(1.0).setNumIterations(100) + + val model = svm.run(testRDD, initialWeights) + + val validationData = SVMSuite.generateSVMInput(A, Array[Double](B,C), nPoints, 17) + val validationRDD = sc.parallelize(validationData,2) + + // 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) + } + + test("SVM with invalid labels") { + val nPoints = 10000 + + // NOTE: Intercept should be small for generating equal 0s and 1s + val A = 0.01 + val B = -1.5 + val C = 1.0 + + val testData = SVMSuite.generateSVMInput(A, Array[Double](B,C), nPoints, 42) + val testRDD = sc.parallelize(testData, 2) + + val testRDDInvalid = testRDD.map { lp => + if (lp.label == 0.0) { + LabeledPoint(-1.0, lp.features) + } else { + lp + } + } + + intercept[SparkException] { + val model = SVMWithSGD.train(testRDDInvalid, 100) + } + + // Turning off data validation should not throw an exception + val noValidationModel = new SVMWithSGD().setValidateData(false).run(testRDDInvalid) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala new file mode 100644 index 0000000000..94245f6027 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.clustering + +import scala.util.Random + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ + +import org.jblas._ + +class KMeansSuite extends FunSuite with BeforeAndAfterAll { + @transient private var sc: SparkContext = _ + + override def beforeAll() { + sc = new SparkContext("local", "test") + } + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + val EPSILON = 1e-4 + + import KMeans.{RANDOM, K_MEANS_PARALLEL} + + def prettyPrint(point: Array[Double]): String = point.mkString("(", ", ", ")") + + def prettyPrint(points: Array[Array[Double]]): String = { + points.map(prettyPrint).mkString("(", "; ", ")") + } + + // L1 distance between two points + def distance1(v1: Array[Double], v2: Array[Double]): Double = { + v1.zip(v2).map{ case (a, b) => math.abs(a-b) }.max + } + + // Assert that two vectors are equal within tolerance EPSILON + def assertEqual(v1: Array[Double], v2: Array[Double]) { + def errorMessage = prettyPrint(v1) + " did not equal " + prettyPrint(v2) + assert(v1.length == v2.length, errorMessage) + assert(distance1(v1, v2) <= EPSILON, errorMessage) + } + + // Assert that two sets of points are equal, within EPSILON tolerance + def assertSetsEqual(set1: Array[Array[Double]], set2: Array[Array[Double]]) { + def errorMessage = prettyPrint(set1) + " did not equal " + prettyPrint(set2) + assert(set1.length == set2.length, errorMessage) + for (v <- set1) { + val closestDistance = set2.map(w => distance1(v, w)).min + if (closestDistance > EPSILON) { + fail(errorMessage) + } + } + for (v <- set2) { + val closestDistance = set1.map(w => distance1(v, w)).min + if (closestDistance > EPSILON) { + fail(errorMessage) + } + } + } + + test("single cluster") { + val data = sc.parallelize(Array( + Array(1.0, 2.0, 6.0), + Array(1.0, 3.0, 0.0), + Array(1.0, 4.0, 6.0) + )) + + // No matter how many runs or iterations we use, we should get one cluster, + // centered at the mean of the points + + var model = KMeans.train(data, k=1, maxIterations=1) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=2) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=5) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=1, runs=5) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=1, runs=5) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train( + data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + } + + test("single cluster with big dataset") { + val smallData = Array( + Array(1.0, 2.0, 6.0), + Array(1.0, 3.0, 0.0), + Array(1.0, 4.0, 6.0) + ) + val data = sc.parallelize((1 to 100).flatMap(_ => smallData), 4) + + // No matter how many runs or iterations we use, we should get one cluster, + // centered at the mean of the points + + var model = KMeans.train(data, k=1, maxIterations=1) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=2) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=5) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=1, runs=5) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=1, runs=5) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + + model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) + assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + } + + test("k-means|| initialization") { + val points = Array( + Array(1.0, 2.0, 6.0), + Array(1.0, 3.0, 0.0), + Array(1.0, 4.0, 6.0), + Array(1.0, 0.0, 1.0), + Array(1.0, 1.0, 1.0) + ) + val rdd = sc.parallelize(points) + + // K-means|| initialization should place all clusters into distinct centers because + // it will make at least five passes, and it will give non-zero probability to each + // unselected point as long as it hasn't yet selected all of them + + var model = KMeans.train(rdd, k=5, maxIterations=1) + assertSetsEqual(model.clusterCenters, points) + + // Iterations of Lloyd's should not change the answer either + model = KMeans.train(rdd, k=5, maxIterations=10) + assertSetsEqual(model.clusterCenters, points) + + // Neither should more runs + model = KMeans.train(rdd, k=5, maxIterations=10, runs=5) + assertSetsEqual(model.clusterCenters, points) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala new file mode 100644 index 0000000000..347ef238f4 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala @@ -0,0 +1,125 @@ +/* + * 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.recommendation + +import scala.collection.JavaConversions._ +import scala.util.Random + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ + +import org.jblas._ + +object ALSSuite { + + def generateRatingsAsJavaList( + users: Int, + products: Int, + features: Int, + samplingRate: Double): (java.util.List[Rating], DoubleMatrix) = { + val (sampledRatings, trueRatings) = generateRatings(users, products, features, samplingRate) + (seqAsJavaList(sampledRatings), trueRatings) + } + + def generateRatings( + users: Int, + products: Int, + features: Int, + samplingRate: Double): (Seq[Rating], DoubleMatrix) = { + val rand = new Random(42) + + // Create a random matrix with uniform values from -1 to 1 + def randomMatrix(m: Int, n: Int) = + new DoubleMatrix(m, n, Array.fill(m * n)(rand.nextDouble() * 2 - 1): _*) + + val userMatrix = randomMatrix(users, features) + val productMatrix = randomMatrix(features, products) + val trueRatings = userMatrix.mmul(productMatrix) + + val sampledRatings = { + for (u <- 0 until users; p <- 0 until products if rand.nextDouble() < samplingRate) + yield Rating(u, p, trueRatings.get(u, p)) + } + + (sampledRatings, trueRatings) + } + +} + + +class ALSSuite extends FunSuite with BeforeAndAfterAll { + @transient private var sc: SparkContext = _ + + override def beforeAll() { + sc = new SparkContext("local", "test") + } + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + test("rank-1 matrices") { + testALS(10, 20, 1, 15, 0.7, 0.3) + } + + test("rank-2 matrices") { + testALS(20, 30, 2, 15, 0.7, 0.3) + } + + /** + * Test if we can correctly factorize R = U * P where U and P are of known rank. + * + * @param users number of users + * @param products number of products + * @param features number of features (rank of problem) + * @param iterations number of iterations to run + * @param samplingRate what fraction of the user-product pairs are known + * @param matchThreshold max difference allowed to consider a predicted rating correct + */ + def testALS(users: Int, products: Int, features: Int, iterations: Int, + samplingRate: Double, matchThreshold: Double) + { + val (sampledRatings, trueRatings) = ALSSuite.generateRatings(users, products, + features, samplingRate) + val model = ALS.train(sc.parallelize(sampledRatings), features, iterations) + + val predictedU = new DoubleMatrix(users, features) + for ((u, vec) <- model.userFeatures.collect(); i <- 0 until features) { + predictedU.put(u, i, vec(i)) + } + val predictedP = new DoubleMatrix(products, features) + for ((p, vec) <- model.productFeatures.collect(); i <- 0 until features) { + predictedP.put(p, i, vec(i)) + } + val predictedRatings = predictedU.mmul(predictedP.transpose) + + for (u <- 0 until users; p <- 0 until products) { + val prediction = predictedRatings.get(u, p) + val correct = trueRatings.get(u, p) + if (math.abs(prediction - correct) > matchThreshold) { + fail("Model failed to predict (%d, %d): %f vs %f\ncorr: %s\npred: %s\nU: %s\n P: %s".format( + u, p, correct, prediction, trueRatings, predictedRatings, predictedU, predictedP)) + } + } + } +} + 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 new file mode 100644 index 0000000000..db980c7bae --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala @@ -0,0 +1,121 @@ +/* + * 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.regression + +import scala.collection.JavaConversions._ +import scala.util.Random + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.mllib.util.LinearDataGenerator + + +class LassoSuite extends FunSuite with BeforeAndAfterAll { + @transient private var sc: SparkContext = _ + + override def beforeAll() { + sc = new SparkContext("local", "test") + } + + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) { + val numOffPredictions = predictions.zip(input).filter { case (prediction, expected) => + // A prediction is off if the prediction is more than 0.5 away from expected value. + math.abs(prediction - expected.label) > 0.5 + }.size + // At least 80% of the predictions should be on. + assert(numOffPredictions < input.length / 5) + } + + test("Lasso local random SGD") { + val nPoints = 10000 + + 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 testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + val ls = new LassoWithSGD() + 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]") + 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 validationData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + + // 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) + } + + test("Lasso local random SGD with initial weights") { + val nPoints = 10000 + + 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 initialB = -1.0 + val initialC = -1.0 + val initialWeights = Array(initialB,initialC) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + val ls = new LassoWithSGD() + 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]") + 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 validationData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 17) + val validationRDD = sc.parallelize(validationData,2) + + // 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) + } +} 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 new file mode 100644 index 0000000000..ef500c704c --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala @@ -0,0 +1,72 @@ +/* + * 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.regression + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.util.LinearDataGenerator + +class LinearRegressionSuite extends FunSuite with BeforeAndAfterAll { + @transient private var sc: SparkContext = _ + + override def beforeAll() { + sc = new SparkContext("local", "test") + } + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) { + val numOffPredictions = predictions.zip(input).filter { case (prediction, expected) => + // A prediction is off if the prediction is more than 0.5 away from expected value. + math.abs(prediction - expected.label) > 0.5 + }.size + // At least 80% of the predictions should be on. + assert(numOffPredictions < input.length / 5) + } + + // Test if we can correctly learn Y = 3 + 10*X1 + 10*X2 + test("linear regression") { + val testRDD = sc.parallelize(LinearDataGenerator.generateLinearInput( + 3.0, Array(10.0, 10.0), 100, 42), 2).cache() + val linReg = new LinearRegressionWithSGD() + 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 validationData = LinearDataGenerator.generateLinearInput( + 3.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) + } +} 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 new file mode 100644 index 0000000000..c18092d804 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala @@ -0,0 +1,90 @@ +/* + * 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.regression + +import scala.collection.JavaConversions._ +import scala.util.Random + +import org.jblas.DoubleMatrix +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.util.LinearDataGenerator + +class RidgeRegressionSuite extends FunSuite with BeforeAndAfterAll { + @transient private var sc: SparkContext = _ + + override def beforeAll() { + sc = new SparkContext("local", "test") + } + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + def predictionError(predictions: Seq[Double], input: Seq[LabeledPoint]) = { + predictions.zip(input).map { case (prediction, expected) => + (prediction - expected.label) * (prediction - expected.label) + }.reduceLeft(_ + _) / predictions.size + } + + test("regularization with skewed weights") { + val nexamples = 200 + val nfeatures = 20 + val eps = 10 + + 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) + + // 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 testRDD = sc.parallelize(testData, 2).cache() + val validationRDD = sc.parallelize(validationData, 2).cache() + + // First run without regularization. + val linearReg = new LinearRegressionWithSGD() + linearReg.optimizer.setNumIterations(200) + .setStepSize(1.0) + + val linearModel = linearReg.run(testRDD) + val linearErr = predictionError( + linearModel.predict(validationRDD.map(_.features)).collect(), validationData) + + val ridgeReg = new RidgeRegressionWithSGD() + ridgeReg.optimizer.setNumIterations(200) + .setRegParam(0.1) + .setStepSize(1.0) + val ridgeModel = ridgeReg.run(testRDD) + val ridgeErr = predictionError( + ridgeModel.predict(validationRDD.map(_.features)).collect(), validationData) + + // Ridge CV-error should be lower than linear regression + assert(ridgeErr < linearErr, + "ridgeError (" + ridgeErr + ") was not less than linearError(" + linearErr + ")") + } +} diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala deleted file mode 100644 index bd87c528c3..0000000000 --- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala +++ /dev/null @@ -1,150 +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 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 spark.SparkContext -import spark.mllib.regression._ - -object LogisticRegressionSuite { - - def generateLogisticInputAsList( - offset: Double, - scale: Double, - nPoints: Int, - seed: Int): java.util.List[LabeledPoint] = { - seqAsJavaList(generateLogisticInput(offset, scale, nPoints, seed)) - } - - // Generate input of the form Y = logistic(offset + scale*X) - def generateLogisticInput( - offset: Double, - scale: Double, - nPoints: Int, - seed: Int): Seq[LabeledPoint] = { - val rnd = new Random(seed) - val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - - // NOTE: if U is uniform[0, 1] then ln(u) - ln(1-u) is Logistic(0,1) - val unifRand = new scala.util.Random(45) - val rLogis = (0 until nPoints).map { i => - val u = unifRand.nextDouble() - math.log(u) - math.log(1.0-u) - } - - // y <- A + B*x + rLogis() - // y <- as.numeric(y > 0) - val y: Seq[Int] = (0 until nPoints).map { i => - val yVal = offset + scale * x1(i) + rLogis(i) - if (yVal > 0) 1 else 0 - } - - val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Array(x1(i)))) - testData - } - -} - -class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll with ShouldMatchers { - @transient private var sc: SparkContext = _ - - override def beforeAll() { - sc = new SparkContext("local", "test") - } - - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) { - val numOffPredictions = predictions.zip(input).filter { case (prediction, expected) => - (prediction != expected.label) - }.size - // At least 83% of the predictions should be on. - ((input.length - numOffPredictions).toDouble / input.length) should be > 0.83 - } - - // Test if we can correctly learn A, B where Y = logistic(A + B*X) - test("logistic regression") { - val nPoints = 10000 - val A = 2.0 - val B = -1.5 - - val testData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 42) - - val testRDD = sc.parallelize(testData, 2) - testRDD.cache() - val lr = new LogisticRegressionWithSGD() - lr.optimizer.setStepSize(10.0).setNumIterations(20) - - val model = lr.run(testRDD) - - // Test the weights - val weight0 = model.weights(0) - assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") - assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") - - val validationData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 17) - val validationRDD = sc.parallelize(validationData, 2) - // 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) - } - - test("logistic regression with initial weights") { - val nPoints = 10000 - val A = 2.0 - val B = -1.5 - - val testData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 42) - - val initialB = -1.0 - val initialWeights = Array(initialB) - - val testRDD = sc.parallelize(testData, 2) - testRDD.cache() - - // Use half as many iterations as the previous test. - val lr = new LogisticRegressionWithSGD() - lr.optimizer.setStepSize(10.0).setNumIterations(10) - - val model = lr.run(testRDD, initialWeights) - - val weight0 = model.weights(0) - assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") - assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") - - val validationData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 17) - val validationRDD = sc.parallelize(validationData, 2) - // 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) - } -} diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala deleted file mode 100644 index 894ae458ad..0000000000 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ /dev/null @@ -1,169 +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 spark.mllib.classification - -import scala.util.Random -import scala.math.signum -import scala.collection.JavaConversions._ - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import spark.SparkContext -import spark.mllib.regression._ - -import org.jblas.DoubleMatrix - -object SVMSuite { - - def generateSVMInputAsList( - intercept: Double, - weights: Array[Double], - nPoints: Int, - seed: Int): java.util.List[LabeledPoint] = { - seqAsJavaList(generateSVMInput(intercept, weights, nPoints, seed)) - } - - // Generate noisy input of the form Y = signum(x.dot(weights) + intercept + noise) - def generateSVMInput( - intercept: Double, - weights: Array[Double], - nPoints: Int, - seed: Int): Seq[LabeledPoint] = { - val rnd = new Random(seed) - val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) - val x = Array.fill[Array[Double]](nPoints)( - Array.fill[Double](weights.length)(rnd.nextDouble() * 2.0 - 1.0)) - val y = x.map { xi => - val yD = (new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) + - intercept + 0.01 * rnd.nextGaussian() - if (yD < 0) 0.0 else 1.0 - } - y.zip(x).map(p => LabeledPoint(p._1, p._2)) - } - -} - -class SVMSuite extends FunSuite with BeforeAndAfterAll { - @transient private var sc: SparkContext = _ - - override def beforeAll() { - sc = new SparkContext("local", "test") - } - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) { - val numOffPredictions = predictions.zip(input).filter { case (prediction, expected) => - (prediction != expected.label) - }.size - // At least 80% of the predictions should be on. - assert(numOffPredictions < input.length / 5) - } - - - test("SVM using local random SGD") { - val nPoints = 10000 - - // NOTE: Intercept should be small for generating equal 0s and 1s - val A = 0.01 - val B = -1.5 - val C = 1.0 - - val testData = SVMSuite.generateSVMInput(A, Array[Double](B,C), nPoints, 42) - - val testRDD = sc.parallelize(testData, 2) - testRDD.cache() - - val svm = new SVMWithSGD() - svm.optimizer.setStepSize(1.0).setRegParam(1.0).setNumIterations(100) - - val model = svm.run(testRDD) - - val validationData = SVMSuite.generateSVMInput(A, Array[Double](B,C), nPoints, 17) - val validationRDD = sc.parallelize(validationData, 2) - - // 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) - } - - test("SVM local random SGD with initial weights") { - val nPoints = 10000 - - // NOTE: Intercept should be small for generating equal 0s and 1s - val A = 0.01 - val B = -1.5 - val C = 1.0 - - val testData = SVMSuite.generateSVMInput(A, Array[Double](B,C), nPoints, 42) - - val initialB = -1.0 - val initialC = -1.0 - val initialWeights = Array(initialB,initialC) - - val testRDD = sc.parallelize(testData, 2) - testRDD.cache() - - val svm = new SVMWithSGD() - svm.optimizer.setStepSize(1.0).setRegParam(1.0).setNumIterations(100) - - val model = svm.run(testRDD, initialWeights) - - val validationData = SVMSuite.generateSVMInput(A, Array[Double](B,C), nPoints, 17) - val validationRDD = sc.parallelize(validationData,2) - - // 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) - } - - test("SVM with invalid labels") { - val nPoints = 10000 - - // NOTE: Intercept should be small for generating equal 0s and 1s - val A = 0.01 - val B = -1.5 - val C = 1.0 - - val testData = SVMSuite.generateSVMInput(A, Array[Double](B,C), nPoints, 42) - val testRDD = sc.parallelize(testData, 2) - - val testRDDInvalid = testRDD.map { lp => - if (lp.label == 0.0) { - LabeledPoint(-1.0, lp.features) - } else { - lp - } - } - - intercept[spark.SparkException] { - val model = SVMWithSGD.train(testRDDInvalid, 100) - } - - // Turning off data validation should not throw an exception - val noValidationModel = new SVMWithSGD().setValidateData(false).run(testRDDInvalid) - } -} diff --git a/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala deleted file mode 100644 index d5d95c8639..0000000000 --- a/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala +++ /dev/null @@ -1,173 +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 spark.mllib.clustering - -import scala.util.Random - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import spark.SparkContext -import spark.SparkContext._ - -import org.jblas._ - -class KMeansSuite extends FunSuite with BeforeAndAfterAll { - @transient private var sc: SparkContext = _ - - override def beforeAll() { - sc = new SparkContext("local", "test") - } - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - val EPSILON = 1e-4 - - import KMeans.{RANDOM, K_MEANS_PARALLEL} - - def prettyPrint(point: Array[Double]): String = point.mkString("(", ", ", ")") - - def prettyPrint(points: Array[Array[Double]]): String = { - points.map(prettyPrint).mkString("(", "; ", ")") - } - - // L1 distance between two points - def distance1(v1: Array[Double], v2: Array[Double]): Double = { - v1.zip(v2).map{ case (a, b) => math.abs(a-b) }.max - } - - // Assert that two vectors are equal within tolerance EPSILON - def assertEqual(v1: Array[Double], v2: Array[Double]) { - def errorMessage = prettyPrint(v1) + " did not equal " + prettyPrint(v2) - assert(v1.length == v2.length, errorMessage) - assert(distance1(v1, v2) <= EPSILON, errorMessage) - } - - // Assert that two sets of points are equal, within EPSILON tolerance - def assertSetsEqual(set1: Array[Array[Double]], set2: Array[Array[Double]]) { - def errorMessage = prettyPrint(set1) + " did not equal " + prettyPrint(set2) - assert(set1.length == set2.length, errorMessage) - for (v <- set1) { - val closestDistance = set2.map(w => distance1(v, w)).min - if (closestDistance > EPSILON) { - fail(errorMessage) - } - } - for (v <- set2) { - val closestDistance = set1.map(w => distance1(v, w)).min - if (closestDistance > EPSILON) { - fail(errorMessage) - } - } - } - - test("single cluster") { - val data = sc.parallelize(Array( - Array(1.0, 2.0, 6.0), - Array(1.0, 3.0, 0.0), - Array(1.0, 4.0, 6.0) - )) - - // No matter how many runs or iterations we use, we should get one cluster, - // centered at the mean of the points - - var model = KMeans.train(data, k=1, maxIterations=1) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) - - model = KMeans.train(data, k=1, maxIterations=2) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) - - model = KMeans.train(data, k=1, maxIterations=5) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) - - model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) - - model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) - - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) - - model = KMeans.train( - data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) - } - - test("single cluster with big dataset") { - val smallData = Array( - Array(1.0, 2.0, 6.0), - Array(1.0, 3.0, 0.0), - Array(1.0, 4.0, 6.0) - ) - val data = sc.parallelize((1 to 100).flatMap(_ => smallData), 4) - - // No matter how many runs or iterations we use, we should get one cluster, - // centered at the mean of the points - - var model = KMeans.train(data, k=1, maxIterations=1) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) - - model = KMeans.train(data, k=1, maxIterations=2) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) - - model = KMeans.train(data, k=1, maxIterations=5) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) - - model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) - - model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) - - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) - - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) - } - - test("k-means|| initialization") { - val points = Array( - Array(1.0, 2.0, 6.0), - Array(1.0, 3.0, 0.0), - Array(1.0, 4.0, 6.0), - Array(1.0, 0.0, 1.0), - Array(1.0, 1.0, 1.0) - ) - val rdd = sc.parallelize(points) - - // K-means|| initialization should place all clusters into distinct centers because - // it will make at least five passes, and it will give non-zero probability to each - // unselected point as long as it hasn't yet selected all of them - - var model = KMeans.train(rdd, k=5, maxIterations=1) - assertSetsEqual(model.clusterCenters, points) - - // Iterations of Lloyd's should not change the answer either - model = KMeans.train(rdd, k=5, maxIterations=10) - assertSetsEqual(model.clusterCenters, points) - - // Neither should more runs - model = KMeans.train(rdd, k=5, maxIterations=10, runs=5) - assertSetsEqual(model.clusterCenters, points) - } -} diff --git a/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala deleted file mode 100644 index 15a60efda6..0000000000 --- a/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala +++ /dev/null @@ -1,125 +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 spark.mllib.recommendation - -import scala.collection.JavaConversions._ -import scala.util.Random - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import spark.SparkContext -import spark.SparkContext._ - -import org.jblas._ - -object ALSSuite { - - def generateRatingsAsJavaList( - users: Int, - products: Int, - features: Int, - samplingRate: Double): (java.util.List[Rating], DoubleMatrix) = { - val (sampledRatings, trueRatings) = generateRatings(users, products, features, samplingRate) - (seqAsJavaList(sampledRatings), trueRatings) - } - - def generateRatings( - users: Int, - products: Int, - features: Int, - samplingRate: Double): (Seq[Rating], DoubleMatrix) = { - val rand = new Random(42) - - // Create a random matrix with uniform values from -1 to 1 - def randomMatrix(m: Int, n: Int) = - new DoubleMatrix(m, n, Array.fill(m * n)(rand.nextDouble() * 2 - 1): _*) - - val userMatrix = randomMatrix(users, features) - val productMatrix = randomMatrix(features, products) - val trueRatings = userMatrix.mmul(productMatrix) - - val sampledRatings = { - for (u <- 0 until users; p <- 0 until products if rand.nextDouble() < samplingRate) - yield Rating(u, p, trueRatings.get(u, p)) - } - - (sampledRatings, trueRatings) - } - -} - - -class ALSSuite extends FunSuite with BeforeAndAfterAll { - @transient private var sc: SparkContext = _ - - override def beforeAll() { - sc = new SparkContext("local", "test") - } - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - test("rank-1 matrices") { - testALS(10, 20, 1, 15, 0.7, 0.3) - } - - test("rank-2 matrices") { - testALS(20, 30, 2, 15, 0.7, 0.3) - } - - /** - * Test if we can correctly factorize R = U * P where U and P are of known rank. - * - * @param users number of users - * @param products number of products - * @param features number of features (rank of problem) - * @param iterations number of iterations to run - * @param samplingRate what fraction of the user-product pairs are known - * @param matchThreshold max difference allowed to consider a predicted rating correct - */ - def testALS(users: Int, products: Int, features: Int, iterations: Int, - samplingRate: Double, matchThreshold: Double) - { - val (sampledRatings, trueRatings) = ALSSuite.generateRatings(users, products, - features, samplingRate) - val model = ALS.train(sc.parallelize(sampledRatings), features, iterations) - - val predictedU = new DoubleMatrix(users, features) - for ((u, vec) <- model.userFeatures.collect(); i <- 0 until features) { - predictedU.put(u, i, vec(i)) - } - val predictedP = new DoubleMatrix(products, features) - for ((p, vec) <- model.productFeatures.collect(); i <- 0 until features) { - predictedP.put(p, i, vec(i)) - } - val predictedRatings = predictedU.mmul(predictedP.transpose) - - for (u <- 0 until users; p <- 0 until products) { - val prediction = predictedRatings.get(u, p) - val correct = trueRatings.get(u, p) - if (math.abs(prediction - correct) > matchThreshold) { - fail("Model failed to predict (%d, %d): %f vs %f\ncorr: %s\npred: %s\nU: %s\n P: %s".format( - u, p, correct, prediction, trueRatings, predictedRatings, predictedU, predictedP)) - } - } - } -} - diff --git a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala deleted file mode 100644 index 622dbbab7f..0000000000 --- a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala +++ /dev/null @@ -1,121 +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 spark.mllib.regression - -import scala.collection.JavaConversions._ -import scala.util.Random - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import spark.SparkContext -import spark.mllib.util.LinearDataGenerator - - -class LassoSuite extends FunSuite with BeforeAndAfterAll { - @transient private var sc: SparkContext = _ - - override def beforeAll() { - sc = new SparkContext("local", "test") - } - - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) { - val numOffPredictions = predictions.zip(input).filter { case (prediction, expected) => - // A prediction is off if the prediction is more than 0.5 away from expected value. - math.abs(prediction - expected.label) > 0.5 - }.size - // At least 80% of the predictions should be on. - assert(numOffPredictions < input.length / 5) - } - - test("Lasso local random SGD") { - val nPoints = 10000 - - 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 testRDD = sc.parallelize(testData, 2) - testRDD.cache() - - val ls = new LassoWithSGD() - 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]") - 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 validationData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 17) - val validationRDD = sc.parallelize(validationData, 2) - - // 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) - } - - test("Lasso local random SGD with initial weights") { - val nPoints = 10000 - - 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 initialB = -1.0 - val initialC = -1.0 - val initialWeights = Array(initialB,initialC) - - val testRDD = sc.parallelize(testData, 2) - testRDD.cache() - - val ls = new LassoWithSGD() - 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]") - 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 validationData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 17) - val validationRDD = sc.parallelize(validationData,2) - - // 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) - } -} diff --git a/mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala deleted file mode 100644 index acc48a3283..0000000000 --- a/mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala +++ /dev/null @@ -1,72 +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 spark.mllib.regression - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import spark.SparkContext -import spark.SparkContext._ -import spark.mllib.util.LinearDataGenerator - -class LinearRegressionSuite extends FunSuite with BeforeAndAfterAll { - @transient private var sc: SparkContext = _ - - override def beforeAll() { - sc = new SparkContext("local", "test") - } - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) { - val numOffPredictions = predictions.zip(input).filter { case (prediction, expected) => - // A prediction is off if the prediction is more than 0.5 away from expected value. - math.abs(prediction - expected.label) > 0.5 - }.size - // At least 80% of the predictions should be on. - assert(numOffPredictions < input.length / 5) - } - - // Test if we can correctly learn Y = 3 + 10*X1 + 10*X2 - test("linear regression") { - val testRDD = sc.parallelize(LinearDataGenerator.generateLinearInput( - 3.0, Array(10.0, 10.0), 100, 42), 2).cache() - val linReg = new LinearRegressionWithSGD() - 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 validationData = LinearDataGenerator.generateLinearInput( - 3.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) - } -} diff --git a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala deleted file mode 100644 index c482035706..0000000000 --- a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala +++ /dev/null @@ -1,90 +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 spark.mllib.regression - -import scala.collection.JavaConversions._ -import scala.util.Random - -import org.jblas.DoubleMatrix -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import spark.SparkContext -import spark.SparkContext._ -import spark.mllib.util.LinearDataGenerator - -class RidgeRegressionSuite extends FunSuite with BeforeAndAfterAll { - @transient private var sc: SparkContext = _ - - override def beforeAll() { - sc = new SparkContext("local", "test") - } - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - def predictionError(predictions: Seq[Double], input: Seq[LabeledPoint]) = { - predictions.zip(input).map { case (prediction, expected) => - (prediction - expected.label) * (prediction - expected.label) - }.reduceLeft(_ + _) / predictions.size - } - - test("regularization with skewed weights") { - val nexamples = 200 - val nfeatures = 20 - val eps = 10 - - 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) - - // 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 testRDD = sc.parallelize(testData, 2).cache() - val validationRDD = sc.parallelize(validationData, 2).cache() - - // First run without regularization. - val linearReg = new LinearRegressionWithSGD() - linearReg.optimizer.setNumIterations(200) - .setStepSize(1.0) - - val linearModel = linearReg.run(testRDD) - val linearErr = predictionError( - linearModel.predict(validationRDD.map(_.features)).collect(), validationData) - - val ridgeReg = new RidgeRegressionWithSGD() - ridgeReg.optimizer.setNumIterations(200) - .setRegParam(0.1) - .setStepSize(1.0) - val ridgeModel = ridgeReg.run(testRDD) - val ridgeErr = predictionError( - ridgeModel.predict(validationRDD.map(_.features)).collect(), validationData) - - // Ridge CV-error should be lower than linear regression - assert(ridgeErr < linearErr, - "ridgeError (" + ridgeErr + ") was not less than linearError(" + linearErr + ")") - } -} -- cgit v1.2.3