diff options
-rw-r--r-- | core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala | 3 | ||||
-rw-r--r-- | core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala | 3 | ||||
-rw-r--r-- | core/src/main/scala/spark/SparkContext.scala | 7 | ||||
-rw-r--r-- | core/src/main/scala/spark/deploy/master/Master.scala | 2 | ||||
-rw-r--r-- | core/src/main/scala/spark/deploy/master/MasterArguments.scala | 5 | ||||
-rw-r--r-- | core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala | 6 | ||||
-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 | ||||
-rw-r--r-- | project/SparkBuild.scala | 4 |
9 files changed, 96 insertions, 48 deletions
diff --git a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala index df55be1254..617954cb98 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..617954cb98 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 46b9935cb7..c01e315e35 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -577,7 +577,12 @@ 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 (SparkHadoopUtil.isYarnMode()) { + logWarning("local jar specified as parameter to addJar under Yarn mode") + return + } + env.httpFileServer.addJar(new File(uri.getPath)) case _ => path } addedJars(key) = System.currentTimeMillis 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" } 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() 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 { |