aboutsummaryrefslogtreecommitdiff
path: root/mllib
diff options
context:
space:
mode:
authorMatei Zaharia <matei.zaharia@gmail.com>2013-07-19 13:33:04 -0700
committerMatei Zaharia <matei.zaharia@gmail.com>2013-07-19 13:33:04 -0700
commitc40f0f21f10a644e39c8c4b53cda6b5a8eed1741 (patch)
treeb4647deee96130e6344f69bf0800d022aba40afe /mllib
parent413b84172ee06164fdffd45805aa8a15e9f27fe6 (diff)
parent2c9ea56db4a1f4324f1dc8fadaf2491d83eda8ca (diff)
downloadspark-c40f0f21f10a644e39c8c4b53cda6b5a8eed1741.tar.gz
spark-c40f0f21f10a644e39c8c4b53cda6b5a8eed1741.tar.bz2
spark-c40f0f21f10a644e39c8c4b53cda6b5a8eed1741.zip
Merge pull request #711 from shivaram/ml-generators
Move ML lib data generator files to util/
Diffstat (limited to 'mllib')
-rw-r--r--mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala (renamed from mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala)50
-rw-r--r--mllib/src/main/scala/spark/mllib/util/RidgeRegressionDataGenerator.scala (renamed from mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala)64
2 files changed, 75 insertions, 39 deletions
diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala
index 8094d22405..8d659cd97c 100644
--- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala
+++ b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala
@@ -15,16 +15,43 @@
* limitations under the License.
*/
-package spark.mllib.regression
+package spark.mllib.util
import scala.util.Random
-import org.jblas.DoubleMatrix
-
import spark.{RDD, SparkContext}
-import spark.mllib.util.MLUtils
-object LogisticRegressionGenerator {
+object LogisticRegressionDataGenerator {
+
+ /**
+ * Generate an RDD containing test data for LogisticRegression. This function chooses
+ * positive labels with probability `probOne` and scales positive examples by `eps`.
+ *
+ * @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[(Double, Array[Double])] = {
+ 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)
+ }
+ (y, x)
+ }
+ data
+ }
def main(args: Array[String]) {
if (args.length != 5) {
@@ -40,17 +67,8 @@ object LogisticRegressionGenerator {
val parts: Int = if (args.length > 4) args(4).toInt else 2
val eps = 3
- val sc = new SparkContext(sparkMaster, "LogisticRegressionGenerator")
-
- val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx =>
- val rnd = new Random(42 + idx)
-
- val y = if (idx % 2 == 0) 0 else 1
- val x = Array.fill[Double](nfeatures) {
- rnd.nextGaussian() + (y * eps)
- }
- (y, x)
- }
+ 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/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionDataGenerator.scala
index c2260ae286..c5b8a29942 100644
--- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala
+++ b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionDataGenerator.scala
@@ -15,43 +15,42 @@
* limitations under the License.
*/
-package spark.mllib.regression
+package spark.mllib.util
import scala.util.Random
import org.jblas.DoubleMatrix
import spark.{RDD, SparkContext}
-import spark.mllib.util.MLUtils
-
-
-object RidgeRegressionGenerator {
-
- def main(args: Array[String]) {
- if (args.length != 5) {
- println("Usage: RidgeRegressionGenerator " +
- "<master> <output_dir> <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
+object RidgeRegressionDataGenerator {
+
+ /**
+ * Generate an RDD containing test data used for RidgeRegression. This function generates
+ * uniformly random values for every feature and adds Gaussian noise with mean `eps` to the
+ * response variable `Y`.
+ *
+ * @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 nparts Number of partitions in the RDD. Default value is 2.
+ */
+ def generateRidgeRDD(
+ sc: SparkContext,
+ nexamples: Int,
+ nfeatures: Int,
+ eps: Double,
+ nparts: Int = 2) : RDD[(Double, Array[Double])] = {
org.jblas.util.Random.seed(42)
- val sc = new SparkContext(sparkMaster, "RidgeRegressionGenerator")
-
// Random values distributed uniformly in [-0.5, 0.5]
val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5)
w.put(0, 0, 10)
w.put(1, 0, 10)
- val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until parts, parts).flatMap { p =>
+ val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nparts, nparts).flatMap { p =>
org.jblas.util.Random.seed(42 + p)
- val examplesInPartition = nexamples / parts
+ val examplesInPartition = nexamples / nparts
val X = DoubleMatrix.rand(examplesInPartition, nfeatures)
val y = X.mmul(w)
@@ -65,6 +64,25 @@ object RidgeRegressionGenerator {
(yObs.get(i, 0), X.getRow(i).toArray)
}
}
+ data
+ }
+
+ def main(args: Array[String]) {
+ if (args.length != 5) {
+ println("Usage: RidgeRegressionGenerator " +
+ "<master> <output_dir> <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, "RidgeRegressionDataGenerator")
+ val data = generateRidgeRDD(sc, nexamples, nfeatures, eps, parts)
MLUtils.saveLabeledData(data, outputPath)
sc.stop()