From 64b88e039a66fffdc1d0f0eb1caa9328d76bbb06 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 14:11:44 -0700 Subject: Move ML lib data generator files to util/ --- .../regression/LogisticRegressionGenerator.scala | 58 ----------------- .../regression/RidgeRegressionGenerator.scala | 72 ---------------------- .../mllib/util/LogisticRegressionGenerator.scala | 58 +++++++++++++++++ .../mllib/util/RidgeRegressionGenerator.scala | 72 ++++++++++++++++++++++ 4 files changed, 130 insertions(+), 130 deletions(-) delete mode 100644 mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala delete mode 100644 mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala create mode 100644 mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala create mode 100644 mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala deleted file mode 100644 index 8094d22405..0000000000 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala +++ /dev/null @@ -1,58 +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.util.Random - -import org.jblas.DoubleMatrix - -import spark.{RDD, SparkContext} -import spark.mllib.util.MLUtils - -object LogisticRegressionGenerator { - - 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, "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) - } - - 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/regression/RidgeRegressionGenerator.scala deleted file mode 100644 index c2260ae286..0000000000 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.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 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 " + - " ") - 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 - - 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 => - org.jblas.util.Random.seed(42 + p) - val examplesInPartition = nexamples / parts - - val X = DoubleMatrix.rand(examplesInPartition, nfeatures) - val y = X.mmul(w) - - val rnd = new Random(42 + p) - - val normalValues = Array.fill[Double](examplesInPartition)(rnd.nextGaussian() * eps) - val yObs = new DoubleMatrix(normalValues).addi(y) - - Iterator.tabulate(examplesInPartition) { i => - (yObs.get(i, 0), X.getRow(i).toArray) - } - } - - MLUtils.saveLabeledData(data, outputPath) - sc.stop() - } -} diff --git a/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala new file mode 100644 index 0000000000..4c580b44da --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.mllib.util + +import scala.util.Random + +import org.jblas.DoubleMatrix + +import spark.{RDD, SparkContext} +import spark.mllib.util.MLUtils + +object LogisticRegressionGenerator { + + 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, "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) + } + + MLUtils.saveLabeledData(data, outputPath) + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala new file mode 100644 index 0000000000..a5bb2e762f --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.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 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 " + + " ") + 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 + + 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 => + org.jblas.util.Random.seed(42 + p) + val examplesInPartition = nexamples / parts + + val X = DoubleMatrix.rand(examplesInPartition, nfeatures) + val y = X.mmul(w) + + val rnd = new Random(42 + p) + + val normalValues = Array.fill[Double](examplesInPartition)(rnd.nextGaussian() * eps) + val yObs = new DoubleMatrix(normalValues).addi(y) + + Iterator.tabulate(examplesInPartition) { i => + (yObs.get(i, 0), X.getRow(i).toArray) + } + } + + MLUtils.saveLabeledData(data, outputPath) + sc.stop() + } +} -- cgit v1.2.3 From 7ab1170503ab46ffd838fb98590dd3d66748dd38 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 18 Jul 2013 11:55:19 -0700 Subject: Refactor data generators to have a function that can be used in tests. --- .../mllib/util/LogisticRegressionGenerator.scala | 48 ++++++++++++------ .../mllib/util/RidgeRegressionGenerator.scala | 57 ++++++++++++++-------- 2 files changed, 71 insertions(+), 34 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala index 4c580b44da..8d659cd97c 100644 --- a/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala @@ -19,12 +19,39 @@ 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/util/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala index a5bb2e762f..6861913dc7 100644 --- a/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala @@ -22,36 +22,36 @@ 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 " + - " ") - 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 - + /** + * 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 +65,25 @@ object RidgeRegressionGenerator { (yObs.get(i, 0), X.getRow(i).toArray) } } + data + } + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: RidgeRegressionGenerator " + + " ") + 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, "RidgeRegressionGenerator") + val data = generateRidgeRDD(sc, nexamples, nfeatures, eps, parts) MLUtils.saveLabeledData(data, outputPath) sc.stop() -- cgit v1.2.3 From 2c9ea56db4a1f4324f1dc8fadaf2491d83eda8ca Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 18 Jul 2013 11:57:14 -0700 Subject: Rename classes to be called DataGenerator --- .../util/LogisticRegressionDataGenerator.scala | 76 ++++++++++++++++++ .../mllib/util/LogisticRegressionGenerator.scala | 76 ------------------ .../mllib/util/RidgeRegressionDataGenerator.scala | 90 +++++++++++++++++++++ .../mllib/util/RidgeRegressionGenerator.scala | 91 ---------------------- 4 files changed, 166 insertions(+), 167 deletions(-) create mode 100644 mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala delete mode 100644 mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala create mode 100644 mllib/src/main/scala/spark/mllib/util/RidgeRegressionDataGenerator.scala delete mode 100644 mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala diff --git a/mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala new file mode 100644 index 0000000000..8d659cd97c --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala @@ -0,0 +1,76 @@ +/* + * 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} + +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) { + 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/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala deleted file mode 100644 index 8d659cd97c..0000000000 --- a/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala +++ /dev/null @@ -1,76 +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} - -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) { - 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/RidgeRegressionDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionDataGenerator.scala new file mode 100644 index 0000000000..c5b8a29942 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionDataGenerator.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 spark.mllib.util + +import scala.util.Random + +import org.jblas.DoubleMatrix + +import spark.{RDD, SparkContext} + +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) + // 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 nparts, nparts).flatMap { p => + org.jblas.util.Random.seed(42 + p) + val examplesInPartition = nexamples / nparts + + val X = DoubleMatrix.rand(examplesInPartition, nfeatures) + val y = X.mmul(w) + + val rnd = new Random(42 + p) + + val normalValues = Array.fill[Double](examplesInPartition)(rnd.nextGaussian() * eps) + val yObs = new DoubleMatrix(normalValues).addi(y) + + Iterator.tabulate(examplesInPartition) { i => + (yObs.get(i, 0), X.getRow(i).toArray) + } + } + data + } + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: RidgeRegressionGenerator " + + " ") + 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() + } +} diff --git a/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala deleted file mode 100644 index 6861913dc7..0000000000 --- a/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala +++ /dev/null @@ -1,91 +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 org.jblas.DoubleMatrix - -import spark.{RDD, SparkContext} - - -object RidgeRegressionGenerator { - - /** - * 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) - // 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 nparts, nparts).flatMap { p => - org.jblas.util.Random.seed(42 + p) - val examplesInPartition = nexamples / nparts - - val X = DoubleMatrix.rand(examplesInPartition, nfeatures) - val y = X.mmul(w) - - val rnd = new Random(42 + p) - - val normalValues = Array.fill[Double](examplesInPartition)(rnd.nextGaussian() * eps) - val yObs = new DoubleMatrix(normalValues).addi(y) - - Iterator.tabulate(examplesInPartition) { i => - (yObs.get(i, 0), X.getRow(i).toArray) - } - } - data - } - - def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: RidgeRegressionGenerator " + - " ") - 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, "RidgeRegressionGenerator") - val data = generateRidgeRDD(sc, nexamples, nfeatures, eps, parts) - - MLUtils.saveLabeledData(data, outputPath) - sc.stop() - } -} -- cgit v1.2.3 From a613628c5078cf41feb973d0ee8a06eb69615bcf Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 19 Jul 2013 16:59:12 +0800 Subject: Do not copy local jars given to SparkContext in yarn mode since the Context is not running on local. This bug causes failure when jars can not be found. Example codes (such as spark.examples.SparkPi) can not work without this fix under yarn mode. --- core/src/main/scala/spark/SparkContext.scala | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 46b9935cb7..957c541ecf 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -102,6 +102,7 @@ class SparkContext( } private val isLocal = (master == "local" || master.startsWith("local[")) + private val isYarn = (master == "yarn-standalone") // Create the Spark execution environment (cache, map output tracker, etc) private[spark] val env = SparkEnv.createFromSystemProperties( @@ -577,11 +578,18 @@ class SparkContext( } else { val uri = new URI(path) val key = uri.getScheme match { - case null | "file" => env.httpFileServer.addJar(new File(uri.getPath)) + case null | "file" => + if (!isYarn) + env.httpFileServer.addJar(new File(uri.getPath)) + else + null case _ => path } - addedJars(key) = System.currentTimeMillis - logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) + + if (key != null) { + addedJars(key) = System.currentTimeMillis + logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) + } } } -- cgit v1.2.3 From aa6f83289b87f38481dbae60ad91d2ac78ccea46 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 19 Jul 2013 22:25:28 +0800 Subject: A better fix for giving local jars unde Yarn mode. --- .../hadoop1/scala/spark/deploy/SparkHadoopUtil.scala | 3 +++ .../hadoop2/scala/spark/deploy/SparkHadoopUtil.scala | 3 +++ core/src/main/scala/spark/SparkContext.scala | 17 +++++++---------- 3 files changed, 13 insertions(+), 10 deletions(-) diff --git a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala index df55be1254..9f040faac3 100644 --- a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala @@ -41,4 +41,7 @@ object SparkHadoopUtil { // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster def addCredentials(conf: JobConf) {} + + def isYarnMode(): Boolean = { False } + } diff --git a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala index df55be1254..9f040faac3 100644 --- a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala @@ -41,4 +41,7 @@ object SparkHadoopUtil { // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster def addCredentials(conf: JobConf) {} + + def isYarnMode(): Boolean = { False } + } diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 957c541ecf..c01e315e35 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -102,7 +102,6 @@ class SparkContext( } private val isLocal = (master == "local" || master.startsWith("local[")) - private val isYarn = (master == "yarn-standalone") // Create the Spark execution environment (cache, map output tracker, etc) private[spark] val env = SparkEnv.createFromSystemProperties( @@ -579,17 +578,15 @@ class SparkContext( val uri = new URI(path) val key = uri.getScheme match { case null | "file" => - if (!isYarn) - env.httpFileServer.addJar(new File(uri.getPath)) - else - null + if (SparkHadoopUtil.isYarnMode()) { + logWarning("local jar specified as parameter to addJar under Yarn mode") + return + } + env.httpFileServer.addJar(new File(uri.getPath)) case _ => path } - - if (key != null) { - addedJars(key) = System.currentTimeMillis - logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) - } + addedJars(key) = System.currentTimeMillis + logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) } } -- cgit v1.2.3 From 4530e8a9bfe35b6d562876b1fb66e534ff5c286d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 20 Jul 2013 00:04:25 +0800 Subject: fix typo. --- core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala | 2 +- core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala index 9f040faac3..617954cb98 100644 --- a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala @@ -42,6 +42,6 @@ object SparkHadoopUtil { // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster def addCredentials(conf: JobConf) {} - def isYarnMode(): Boolean = { False } + def isYarnMode(): Boolean = { false } } diff --git a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala index 9f040faac3..617954cb98 100644 --- a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala @@ -42,6 +42,6 @@ object SparkHadoopUtil { // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster def addCredentials(conf: JobConf) {} - def isYarnMode(): Boolean = { False } + def isYarnMode(): Boolean = { false } } -- cgit v1.2.3 From d1738d72ba2ff28c38e03beb3e17f03d3dd77e1a Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 20 Jul 2013 00:37:24 +0800 Subject: also exclude asm for hadoop2. hadoop1 looks like no need to do that too. --- project/SparkBuild.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 59edda5097..2f2cbf646a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -190,8 +190,8 @@ object SparkBuild extends Build { ) } else { Seq( - "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty), - "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty) + "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm) ) } } else { -- cgit v1.2.3 From cfce9a6a365a8a4b156e99f412157704cab592b9 Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Thu, 18 Jul 2013 17:43:27 -0700 Subject: Regression: default webui-port can't be set via command line "--webui-port" anymore --- core/src/main/scala/spark/deploy/master/Master.scala | 2 +- core/src/main/scala/spark/deploy/master/MasterArguments.scala | 5 ++++- core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala | 6 ++---- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index e5a7a87e2e..eddcafd84d 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -53,7 +53,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act var firstApp: Option[ApplicationInfo] = None - val webUi = new MasterWebUI(self) + val webUi = new MasterWebUI(self, webUiPort) Utils.checkHost(host, "Expected hostname") diff --git a/core/src/main/scala/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/spark/deploy/master/MasterArguments.scala index d0ec3d5ea0..0ae0160767 100644 --- a/core/src/main/scala/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/spark/deploy/master/MasterArguments.scala @@ -38,7 +38,10 @@ private[spark] class MasterArguments(args: Array[String]) { if (System.getenv("SPARK_MASTER_WEBUI_PORT") != null) { webUiPort = System.getenv("SPARK_MASTER_WEBUI_PORT").toInt } - + if (System.getProperty("master.ui.port") != null) { + webUiPort = System.getProperty("master.ui.port").toInt + } + parse(args.toList) def parse(args: List[String]): Unit = args match { diff --git a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala index 04b32c7968..dabc2d8dc7 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -32,12 +32,11 @@ import spark.ui.JettyUtils._ * Web UI server for the standalone master. */ private[spark] -class MasterWebUI(val master: ActorRef, requestedPort: Option[Int] = None) extends Logging { +class MasterWebUI(val master: ActorRef, requestedPort: Int) extends Logging { implicit val timeout = Duration.create( System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") val host = Utils.localHostName() - val port = requestedPort.getOrElse( - System.getProperty("master.ui.port", MasterWebUI.DEFAULT_PORT).toInt) + val port = requestedPort var server: Option[Server] = None var boundPort: Option[Int] = None @@ -72,5 +71,4 @@ class MasterWebUI(val master: ActorRef, requestedPort: Option[Int] = None) exten private[spark] object MasterWebUI { val STATIC_RESOURCE_DIR = "spark/ui/static" - val DEFAULT_PORT = "8080" } -- cgit v1.2.3