From 85a35c68401e171df0b72b172a689d8c4e412199 Mon Sep 17 00:00:00 2001 From: Christoph Grothaus Date: Fri, 15 Feb 2013 14:11:34 +0100 Subject: Fix SPARK-698. From ExecutorRunner, launch java directly instead via the run scripts. --- .../scala/spark/deploy/worker/ExecutorRunner.scala | 43 ++++++++++++++++++++-- run | 3 ++ run2.cmd | 3 ++ 3 files changed, 45 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index de11771c8e..214c44fc88 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -75,9 +75,45 @@ private[spark] class ExecutorRunner( def buildCommandSeq(): Seq[String] = { val command = appDesc.command - val script = if (System.getProperty("os.name").startsWith("Windows")) "run.cmd" else "run" - val runScript = new File(sparkHome, script).getCanonicalPath - Seq(runScript, command.mainClass) ++ (command.arguments ++ Seq(appId)).map(substituteVariables) + val runner = if (getEnvOrEmpty("JAVA_HOME") == "") { + "java" + } else { + getEnvOrEmpty("JAVA_HOME") + "/bin/java" + } + // SPARK-698: do not call the run.cmd script, as process.destroy() + // fails to kill a process tree on Windows + Seq(runner) ++ buildJavaOpts() ++ Seq(command.mainClass) ++ + command.arguments.map(substituteVariables) + } + + /* + * Attention: this must always be aligned with the environment variables in the run scripts and the + * way the JAVA_OPTS are assembled there. + */ + def buildJavaOpts(): Seq[String] = { + val _javaLibPath = if (getEnvOrEmpty("SPARK_LIBRARY_PATH") == "") { + "" + } else { + "-Djava.library.path=" + getEnvOrEmpty("SPARK_LIBRARY_PATH") + } + + Seq("-cp", + getEnvOrEmpty("CLASSPATH"), + // SPARK_JAVA_OPTS is overwritten with SPARK_DAEMON_JAVA_OPTS for running the worker + getEnvOrEmpty("SPARK_NONDAEMON_JAVA_OPTS"), + _javaLibPath, + "-Xms" + memory.toString + "M", + "-Xmx" + memory.toString + "M") + .filter(_ != "") + } + + def getEnvOrEmpty(key: String): String = { + val result = System.getenv(key) + if (result == null) { + "" + } else { + result + } } /** Spawn a thread that will redirect a given stream to a file */ @@ -113,7 +149,6 @@ private[spark] class ExecutorRunner( for ((key, value) <- appDesc.command.environment) { env.put(key, value) } - env.put("SPARK_MEM", memory.toString + "m") // In case we are running this from within the Spark Shell, avoid creating a "scala" // parent process for the executor command env.put("SPARK_LAUNCH_WITH_SCALA", "0") diff --git a/run b/run index 82b1da005a..b5f693f1fa 100755 --- a/run +++ b/run @@ -22,6 +22,8 @@ fi # values for that; it doesn't need a lot if [ "$1" = "spark.deploy.master.Master" -o "$1" = "spark.deploy.worker.Worker" ]; then SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m} + # Backup current SPARK_JAVA_OPTS for use in ExecutorRunner.scala + SPARK_NONDAEMON_JAVA_OPTS=$SPARK_JAVA_OPTS SPARK_JAVA_OPTS=$SPARK_DAEMON_JAVA_OPTS # Empty by default fi @@ -70,6 +72,7 @@ if [ -e $FWDIR/conf/java-opts ] ; then JAVA_OPTS+=" `cat $FWDIR/conf/java-opts`" fi export JAVA_OPTS +# Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! CORE_DIR="$FWDIR/core" REPL_DIR="$FWDIR/repl" diff --git a/run2.cmd b/run2.cmd index c913a5195e..a93bbad0b9 100644 --- a/run2.cmd +++ b/run2.cmd @@ -22,6 +22,8 @@ if "%1"=="spark.deploy.master.Master" set RUNNING_DAEMON=1 if "%1"=="spark.deploy.worker.Worker" set RUNNING_DAEMON=1 if "x%SPARK_DAEMON_MEMORY%" == "x" set SPARK_DAEMON_MEMORY=512m if "%RUNNING_DAEMON%"=="1" set SPARK_MEM=%SPARK_DAEMON_MEMORY% +rem Backup current SPARK_JAVA_OPTS for use in ExecutorRunner.scala +if "%RUNNING_DAEMON%"=="1" set SPARK_NONDAEMON_JAVA_OPTS=%SPARK_JAVA_OPTS% if "%RUNNING_DAEMON%"=="1" set SPARK_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% rem Check that SCALA_HOME has been specified @@ -42,6 +44,7 @@ rem Set JAVA_OPTS to be able to load native libraries and to set heap size set JAVA_OPTS=%SPARK_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%SPARK_MEM% -Xmx%SPARK_MEM% rem Load extra JAVA_OPTS from conf/java-opts, if it exists if exist "%FWDIR%conf\java-opts.cmd" call "%FWDIR%conf\java-opts.cmd" +rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! set CORE_DIR=%FWDIR%core set REPL_DIR=%FWDIR%repl -- cgit v1.2.3 From f39f2b7636f52568a556987c8b7f7393299b0351 Mon Sep 17 00:00:00 2001 From: Christoph Grothaus Date: Sun, 24 Feb 2013 21:24:30 +0100 Subject: Incorporate feedback from mateiz: - we do not need getEnvOrEmpty - Instead of saving SPARK_NONDAEMON_JAVA_OPTS, it would be better to modify the scripts to use a different variable name for the JAVA_OPTS they do eventually use --- .../scala/spark/deploy/worker/ExecutorRunner.scala | 24 +++++++--------------- run | 9 ++++---- run2.cmd | 8 ++++---- 3 files changed, 16 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 214c44fc88..38216ce62f 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -75,10 +75,10 @@ private[spark] class ExecutorRunner( def buildCommandSeq(): Seq[String] = { val command = appDesc.command - val runner = if (getEnvOrEmpty("JAVA_HOME") == "") { + val runner = if (System.getenv("JAVA_HOME") == null) { "java" } else { - getEnvOrEmpty("JAVA_HOME") + "/bin/java" + System.getenv("JAVA_HOME") + "/bin/java" } // SPARK-698: do not call the run.cmd script, as process.destroy() // fails to kill a process tree on Windows @@ -91,29 +91,19 @@ private[spark] class ExecutorRunner( * way the JAVA_OPTS are assembled there. */ def buildJavaOpts(): Seq[String] = { - val _javaLibPath = if (getEnvOrEmpty("SPARK_LIBRARY_PATH") == "") { + val _javaLibPath = if (System.getenv("SPARK_LIBRARY_PATH") == null) { "" } else { - "-Djava.library.path=" + getEnvOrEmpty("SPARK_LIBRARY_PATH") + "-Djava.library.path=" + System.getenv("SPARK_LIBRARY_PATH") } Seq("-cp", - getEnvOrEmpty("CLASSPATH"), - // SPARK_JAVA_OPTS is overwritten with SPARK_DAEMON_JAVA_OPTS for running the worker - getEnvOrEmpty("SPARK_NONDAEMON_JAVA_OPTS"), + System.getenv("CLASSPATH"), + System.getenv("SPARK_JAVA_OPTS"), _javaLibPath, "-Xms" + memory.toString + "M", "-Xmx" + memory.toString + "M") - .filter(_ != "") - } - - def getEnvOrEmpty(key: String): String = { - val result = System.getenv(key) - if (result == null) { - "" - } else { - result - } + .filter(_ != null) } /** Spawn a thread that will redirect a given stream to a file */ diff --git a/run b/run index b5f693f1fa..e1482dafbe 100755 --- a/run +++ b/run @@ -22,9 +22,10 @@ fi # values for that; it doesn't need a lot if [ "$1" = "spark.deploy.master.Master" -o "$1" = "spark.deploy.worker.Worker" ]; then SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m} - # Backup current SPARK_JAVA_OPTS for use in ExecutorRunner.scala - SPARK_NONDAEMON_JAVA_OPTS=$SPARK_JAVA_OPTS - SPARK_JAVA_OPTS=$SPARK_DAEMON_JAVA_OPTS # Empty by default + # Do not overwrite SPARK_JAVA_OPTS environment variable in this script + OUR_JAVA_OPTS=$SPARK_DAEMON_JAVA_OPTS # Empty by default +else + OUR_JAVA_OPTS=$SPARK_JAVA_OPTS fi if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then @@ -64,7 +65,7 @@ fi export SPARK_MEM # Set JAVA_OPTS to be able to load native libraries and to set heap size -JAVA_OPTS="$SPARK_JAVA_OPTS" +JAVA_OPTS="$OUR_JAVA_OPTS" JAVA_OPTS+=" -Djava.library.path=$SPARK_LIBRARY_PATH" JAVA_OPTS+=" -Xms$SPARK_MEM -Xmx$SPARK_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists diff --git a/run2.cmd b/run2.cmd index a93bbad0b9..8648c0380a 100644 --- a/run2.cmd +++ b/run2.cmd @@ -22,9 +22,9 @@ if "%1"=="spark.deploy.master.Master" set RUNNING_DAEMON=1 if "%1"=="spark.deploy.worker.Worker" set RUNNING_DAEMON=1 if "x%SPARK_DAEMON_MEMORY%" == "x" set SPARK_DAEMON_MEMORY=512m if "%RUNNING_DAEMON%"=="1" set SPARK_MEM=%SPARK_DAEMON_MEMORY% -rem Backup current SPARK_JAVA_OPTS for use in ExecutorRunner.scala -if "%RUNNING_DAEMON%"=="1" set SPARK_NONDAEMON_JAVA_OPTS=%SPARK_JAVA_OPTS% -if "%RUNNING_DAEMON%"=="1" set SPARK_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% +rem Do not overwrite SPARK_JAVA_OPTS environment variable in this script +if "%RUNNING_DAEMON%"=="0" set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% +if "%RUNNING_DAEMON%"=="1" set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% rem Check that SCALA_HOME has been specified if not "x%SCALA_HOME%"=="x" goto scala_exists @@ -41,7 +41,7 @@ rem variable so that our process sees it and can report it to Mesos if "x%SPARK_MEM%"=="x" set SPARK_MEM=512m rem Set JAVA_OPTS to be able to load native libraries and to set heap size -set JAVA_OPTS=%SPARK_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%SPARK_MEM% -Xmx%SPARK_MEM% +set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%SPARK_MEM% -Xmx%SPARK_MEM% rem Load extra JAVA_OPTS from conf/java-opts, if it exists if exist "%FWDIR%conf\java-opts.cmd" call "%FWDIR%conf\java-opts.cmd" rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! -- cgit v1.2.3 From 4aa1205202f26663f59347f25a7d1f03c755545d Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 11 Mar 2013 12:37:29 -0600 Subject: adding typesafe repo to streaming resolvers so that akka-zeromq is found --- project/SparkBuild.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b0b6e21681..44c8058e9d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -162,6 +162,9 @@ object SparkBuild extends Build { def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", + resolvers ++= Seq( + "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/" + ), libraryDependencies ++= Seq( "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile", "com.github.sgroschupf" % "zkclient" % "0.1", -- cgit v1.2.3 From c1c3682c984c83f75352fc22dcadd3e46058cfaf Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 11 Mar 2013 12:40:44 -0600 Subject: adding checkpoint dir to .gitignore --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index 155e785b01..6c9ffa5426 100644 --- a/.gitignore +++ b/.gitignore @@ -36,3 +36,4 @@ streaming-tests.log dependency-reduced-pom.xml .ensime .ensime_lucene +checkpoint -- cgit v1.2.3 From c07087364bac672ed7ded6dfeef00bab628c2f9b Mon Sep 17 00:00:00 2001 From: Harold Lim Date: Mon, 4 Mar 2013 16:37:27 -0500 Subject: Made changes to the SparkContext to have a DynamicVariable for setting local properties that can be passed down the stack. Added an implementation of the fair scheduler --- core/src/main/scala/spark/SparkContext.scala | 38 ++- .../main/scala/spark/scheduler/DAGScheduler.scala | 36 ++- .../scala/spark/scheduler/DAGSchedulerEvent.scala | 5 +- core/src/main/scala/spark/scheduler/Stage.scala | 5 +- core/src/main/scala/spark/scheduler/TaskSet.scala | 4 +- .../cluster/fair/FairClusterScheduler.scala | 341 +++++++++++++++++++++ .../cluster/fair/FairTaskSetManager.scala | 130 ++++++++ 7 files changed, 530 insertions(+), 29 deletions(-) create mode 100644 core/src/main/scala/spark/scheduler/cluster/fair/FairClusterScheduler.scala create mode 100644 core/src/main/scala/spark/scheduler/cluster/fair/FairTaskSetManager.scala diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 4957a54c1b..bd2261cf0d 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -3,11 +3,13 @@ package spark import java.io._ import java.util.concurrent.atomic.AtomicInteger import java.net.URI +import java.util.Properties import scala.collection.Map import scala.collection.generic.Growable import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ +import scala.util.DynamicVariable import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration @@ -72,6 +74,11 @@ class SparkContext( if (System.getProperty("spark.driver.port") == null) { System.setProperty("spark.driver.port", "0") } + + //Set the default task scheduler + if (System.getProperty("spark.cluster.taskscheduler") == null) { + System.setProperty("spark.cluster.taskscheduler", "spark.scheduler.cluster.ClusterScheduler") + } private val isLocal = (master == "local" || master.startsWith("local[")) @@ -112,7 +119,7 @@ class SparkContext( } } executorEnvs ++= environment - + // Create and start the scheduler private var taskScheduler: TaskScheduler = { // Regular expression used for local[N] master format @@ -137,7 +144,7 @@ class SparkContext( new LocalScheduler(threads.toInt, maxFailures.toInt, this) case SPARK_REGEX(sparkUrl) => - val scheduler = new ClusterScheduler(this) + val scheduler = Class.forName(System.getProperty("spark.cluster.taskscheduler")).getConstructors()(0).newInstance(Array[AnyRef](this):_*).asInstanceOf[ClusterScheduler] val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName) scheduler.initialize(backend) scheduler @@ -153,7 +160,7 @@ class SparkContext( memoryPerSlaveInt, sparkMemEnvInt)) } - val scheduler = new ClusterScheduler(this) + val scheduler = Class.forName(System.getProperty("spark.cluster.taskscheduler")).getConstructors()(0).newInstance(Array[AnyRef](this):_*).asInstanceOf[ClusterScheduler] val localCluster = new LocalSparkCluster( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) val sparkUrl = localCluster.start() @@ -169,7 +176,7 @@ class SparkContext( logWarning("Master %s does not match expected format, parsing as Mesos URL".format(master)) } MesosNativeLibrary.load() - val scheduler = new ClusterScheduler(this) + val scheduler = Class.forName(System.getProperty("spark.cluster.taskscheduler")).getConstructors()(0).newInstance(Array[AnyRef](this):_*).asInstanceOf[ClusterScheduler] val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean val masterWithoutProtocol = master.replaceFirst("^mesos://", "") // Strip initial mesos:// val backend = if (coarseGrained) { @@ -206,6 +213,20 @@ class SparkContext( } private[spark] var checkpointDir: Option[String] = None + + // Thread Local variable that can be used by users to pass information down the stack + private val localProperties = new DynamicVariable[Properties](null) + + def initLocalProperties() { + localProperties.value = new Properties() + } + + def addLocalProperties(key: String, value: String) { + if(localProperties.value == null) { + localProperties.value = new Properties() + } + localProperties.value.setProperty(key,value) + } // Methods for creating RDDs @@ -578,7 +599,7 @@ class SparkContext( val callSite = Utils.getSparkCallSite logInfo("Starting job: " + callSite) val start = System.nanoTime - val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal, resultHandler) + val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal, resultHandler,localProperties.value) logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s") rdd.doCheckpoint() result @@ -649,7 +670,7 @@ class SparkContext( val processFunc = (context: TaskContext, iter: Iterator[T]) => processPartition(iter) runJob[T, U](rdd, processFunc, 0 until rdd.partitions.size, false, resultHandler) } - + /** * Run a job that can return approximate results. */ @@ -657,12 +678,11 @@ class SparkContext( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, evaluator: ApproximateEvaluator[U, R], - timeout: Long - ): PartialResult[R] = { + timeout: Long): PartialResult[R] = { val callSite = Utils.getSparkCallSite logInfo("Starting job: " + callSite) val start = System.nanoTime - val result = dagScheduler.runApproximateJob(rdd, func, evaluator, callSite, timeout) + val result = dagScheduler.runApproximateJob(rdd, func, evaluator, callSite, timeout, localProperties.value) logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s") result } diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index c54dce51d7..2ad73f3232 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -4,6 +4,7 @@ import cluster.TaskInfo import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.LinkedBlockingQueue import java.util.concurrent.TimeUnit +import java.util.Properties import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} @@ -128,11 +129,11 @@ class DAGScheduler( * The priority value passed in will be used if the stage doesn't already exist with * a lower priority (we assume that priorities always increase across jobs for now). */ - private def getShuffleMapStage(shuffleDep: ShuffleDependency[_,_], priority: Int): Stage = { + private def getShuffleMapStage(shuffleDep: ShuffleDependency[_,_], priority: Int, properties: Properties): Stage = { shuffleToMapStage.get(shuffleDep.shuffleId) match { case Some(stage) => stage case None => - val stage = newStage(shuffleDep.rdd, Some(shuffleDep), priority) + val stage = newStage(shuffleDep.rdd, Some(shuffleDep), priority, properties) shuffleToMapStage(shuffleDep.shuffleId) = stage stage } @@ -143,7 +144,7 @@ class DAGScheduler( * as a result stage for the final RDD used directly in an action. The stage will also be given * the provided priority. */ - private def newStage(rdd: RDD[_], shuffleDep: Option[ShuffleDependency[_,_]], priority: Int): Stage = { + private def newStage(rdd: RDD[_], shuffleDep: Option[ShuffleDependency[_,_]], priority: Int, properties: Properties): Stage = { if (shuffleDep != None) { // Kind of ugly: need to register RDDs with the cache and map output tracker here // since we can't do it in the RDD constructor because # of partitions is unknown @@ -151,7 +152,7 @@ class DAGScheduler( mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.partitions.size) } val id = nextStageId.getAndIncrement() - val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority), priority) + val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority, properties), priority, properties) idToStage(id) = stage stageToInfos(stage) = StageInfo(stage) stage @@ -161,7 +162,7 @@ class DAGScheduler( * Get or create the list of parent stages for a given RDD. The stages will be assigned the * provided priority if they haven't already been created with a lower priority. */ - private def getParentStages(rdd: RDD[_], priority: Int): List[Stage] = { + private def getParentStages(rdd: RDD[_], priority: Int, properties: Properties): List[Stage] = { val parents = new HashSet[Stage] val visited = new HashSet[RDD[_]] def visit(r: RDD[_]) { @@ -172,7 +173,7 @@ class DAGScheduler( for (dep <- r.dependencies) { dep match { case shufDep: ShuffleDependency[_,_] => - parents += getShuffleMapStage(shufDep, priority) + parents += getShuffleMapStage(shufDep, priority, properties) case _ => visit(dep.rdd) } @@ -193,7 +194,7 @@ class DAGScheduler( for (dep <- rdd.dependencies) { dep match { case shufDep: ShuffleDependency[_,_] => - val mapStage = getShuffleMapStage(shufDep, stage.priority) + val mapStage = getShuffleMapStage(shufDep, stage.priority, stage.properties) if (!mapStage.isAvailable) { missing += mapStage } @@ -221,13 +222,14 @@ class DAGScheduler( partitions: Seq[Int], callSite: String, allowLocal: Boolean, - resultHandler: (Int, U) => Unit) + resultHandler: (Int, U) => Unit, + properties: Properties = null) : (JobSubmitted, JobWaiter[U]) = { assert(partitions.size > 0) val waiter = new JobWaiter(partitions.size, resultHandler) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] - val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter) + val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties) return (toSubmit, waiter) } @@ -237,13 +239,13 @@ class DAGScheduler( partitions: Seq[Int], callSite: String, allowLocal: Boolean, - resultHandler: (Int, U) => Unit) + resultHandler: (Int, U) => Unit, properties: Properties = null) { if (partitions.size == 0) { return } val (toSubmit, waiter) = prepareJob( - finalRdd, func, partitions, callSite, allowLocal, resultHandler) + finalRdd, func, partitions, callSite, allowLocal, resultHandler, properties) eventQueue.put(toSubmit) waiter.awaitResult() match { case JobSucceeded => {} @@ -258,13 +260,13 @@ class DAGScheduler( func: (TaskContext, Iterator[T]) => U, evaluator: ApproximateEvaluator[U, R], callSite: String, - timeout: Long) + timeout: Long, properties: Properties = null) : PartialResult[R] = { val listener = new ApproximateActionListener(rdd, func, evaluator, timeout) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] val partitions = (0 until rdd.partitions.size).toArray - eventQueue.put(JobSubmitted(rdd, func2, partitions, false, callSite, listener)) + eventQueue.put(JobSubmitted(rdd, func2, partitions, false, callSite, listener, properties)) return listener.awaitResult() // Will throw an exception if the job fails } @@ -274,9 +276,9 @@ class DAGScheduler( */ private[scheduler] def processEvent(event: DAGSchedulerEvent): Boolean = { event match { - case JobSubmitted(finalRDD, func, partitions, allowLocal, callSite, listener) => + case JobSubmitted(finalRDD, func, partitions, allowLocal, callSite, listener, properties) => val runId = nextRunId.getAndIncrement() - val finalStage = newStage(finalRDD, None, runId) + val finalStage = newStage(finalRDD, None, runId, properties) val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener) clearCacheLocs() logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length + @@ -458,7 +460,7 @@ class DAGScheduler( myPending ++= tasks logDebug("New pending tasks: " + myPending) taskSched.submitTasks( - new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.priority)) + new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.priority, stage.properties)) if (!stage.submissionTime.isDefined) { stage.submissionTime = Some(System.currentTimeMillis()) } @@ -663,7 +665,7 @@ class DAGScheduler( for (dep <- rdd.dependencies) { dep match { case shufDep: ShuffleDependency[_,_] => - val mapStage = getShuffleMapStage(shufDep, stage.priority) + val mapStage = getShuffleMapStage(shufDep, stage.priority, stage.properties) if (!mapStage.isAvailable) { visitedStages += mapStage visit(mapStage.rdd) diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index ed0b9bf178..79588891e7 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -1,5 +1,8 @@ package spark.scheduler + +import java.util.Properties + import spark.scheduler.cluster.TaskInfo import scala.collection.mutable.Map @@ -20,7 +23,7 @@ private[spark] case class JobSubmitted( partitions: Array[Int], allowLocal: Boolean, callSite: String, - listener: JobListener) + listener: JobListener, properties: Properties) extends DAGSchedulerEvent private[spark] case class CompletionEvent( diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala index 552061e46b..97afa27a60 100644 --- a/core/src/main/scala/spark/scheduler/Stage.scala +++ b/core/src/main/scala/spark/scheduler/Stage.scala @@ -1,10 +1,12 @@ package spark.scheduler import java.net.URI +import java.util.Properties import spark._ import spark.storage.BlockManagerId + /** * A stage is a set of independent tasks all computing the same function that need to run as part * of a Spark job, where all the tasks have the same shuffle dependencies. Each DAG of tasks run @@ -24,7 +26,8 @@ private[spark] class Stage( val rdd: RDD[_], val shuffleDep: Option[ShuffleDependency[_,_]], // Output shuffle if stage is a map stage val parents: List[Stage], - val priority: Int) + val priority: Int, + val properties: Properties = null) extends Logging { val isShuffleMap = shuffleDep != None diff --git a/core/src/main/scala/spark/scheduler/TaskSet.scala b/core/src/main/scala/spark/scheduler/TaskSet.scala index a3002ca477..2498e8a5aa 100644 --- a/core/src/main/scala/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/spark/scheduler/TaskSet.scala @@ -1,10 +1,12 @@ package spark.scheduler +import java.util.Properties + /** * A set of tasks submitted together to the low-level TaskScheduler, usually representing * missing partitions of a particular stage. */ -private[spark] class TaskSet(val tasks: Array[Task[_]], val stageId: Int, val attempt: Int, val priority: Int) { +private[spark] class TaskSet(val tasks: Array[Task[_]], val stageId: Int, val attempt: Int, val priority: Int, val properties: Properties) { val id: String = stageId + "." + attempt override def toString: String = "TaskSet " + id diff --git a/core/src/main/scala/spark/scheduler/cluster/fair/FairClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/fair/FairClusterScheduler.scala new file mode 100644 index 0000000000..37d98ccb2a --- /dev/null +++ b/core/src/main/scala/spark/scheduler/cluster/fair/FairClusterScheduler.scala @@ -0,0 +1,341 @@ +package spark.scheduler.cluster.fair + +import java.io.{File, FileInputStream, FileOutputStream} +import java.util.{TimerTask, Timer} + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet +import scala.util.control.Breaks._ +import scala.xml._ + +import spark._ +import spark.TaskState.TaskState +import spark.scheduler._ +import spark.scheduler.cluster._ +import java.nio.ByteBuffer +import java.util.concurrent.atomic.AtomicLong +import scala.io.Source + +/** + * An implementation of a fair TaskScheduler, for running tasks on a cluster. Clients should first call + * start(), then submit task sets through the runTasks method. + * + * The current implementation makes the following assumptions: A pool has a fixed configuration of weight. + * Within a pool, it just uses FIFO. + * Also, currently we assume that pools are statically defined + * We currently don't support min shares + */ +private[spark] class FairClusterScheduler(override val sc: SparkContext) + extends ClusterScheduler(sc) + with Logging { + + + val schedulerAllocFile = System.getProperty("mapred.fairscheduler.allocation.file","unspecified") + + val poolNameToPool= new HashMap[String, Pool] + var pools = new ArrayBuffer[Pool] + + loadPoolProperties() + + def loadPoolProperties() { + //first check if the file exists + val file = new File(schedulerAllocFile) + if(!file.exists()) { + //if file does not exist, we just create 1 pool, default + val pool = new Pool("default",100) + pools += pool + poolNameToPool("default") = pool + logInfo("Created a default pool with weight = 100") + } + else { + val xml = XML.loadFile(file) + for (poolNode <- (xml \\ "pool")) { + if((poolNode \ "weight").text != ""){ + val pool = new Pool((poolNode \ "@name").text,(poolNode \ "weight").text.toInt) + pools += pool + poolNameToPool((poolNode \ "@name").text) = pool + logInfo("Created pool "+ pool.name +"with weight = "+pool.weight) + } else { + val pool = new Pool((poolNode \ "@name").text,100) + pools += pool + poolNameToPool((poolNode \ "@name").text) = pool + logInfo("Created pool "+ pool.name +"with weight = 100") + } + } + if(!poolNameToPool.contains("default")) { + val pool = new Pool("default", 100) + pools += pool + poolNameToPool("default") = pool + logInfo("Created a default pool with weight = 100") + } + + } + } + + def taskFinished(manager: TaskSetManager) { + var poolName = "default" + if(manager.taskSet.properties != null) + poolName = manager.taskSet.properties.getProperty("spark.scheduler.cluster.fair.pool","default") + + this.synchronized { + //have to check that poolName exists + if(poolNameToPool.contains(poolName)) + { + poolNameToPool(poolName).numRunningTasks -= 1 + } + else + { + poolNameToPool("default").numRunningTasks -= 1 + } + } + } + + override def submitTasks(taskSet: TaskSet) { + val tasks = taskSet.tasks + + + var poolName = "default" + if(taskSet.properties != null) + poolName = taskSet.properties.getProperty("spark.scheduler.cluster.fair.pool","default") + + this.synchronized { + if(poolNameToPool.contains(poolName)) + { + val manager = new FairTaskSetManager(this, taskSet) + poolNameToPool(poolName).activeTaskSetsQueue += manager + activeTaskSets(taskSet.id) = manager + //activeTaskSetsQueue += manager + taskSetTaskIds(taskSet.id) = new HashSet[Long]() + logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks to pool "+poolName) + } + else //If the pool name does not exists, where do we put them? We put them in default + { + val manager = new FairTaskSetManager(this, taskSet) + poolNameToPool("default").activeTaskSetsQueue += manager + activeTaskSets(taskSet.id) = manager + //activeTaskSetsQueue += manager + taskSetTaskIds(taskSet.id) = new HashSet[Long]() + logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks to pool default") + } + if (hasReceivedTask == false) { + starvationTimer.scheduleAtFixedRate(new TimerTask() { + override def run() { + if (!hasLaunchedTask) { + logWarning("Initial job has not accepted any resources; " + + "check your cluster UI to ensure that workers are registered") + } else { + this.cancel() + } + } + }, STARVATION_TIMEOUT, STARVATION_TIMEOUT) + } + hasReceivedTask = true; + + } + backend.reviveOffers() + } + + override def taskSetFinished(manager: TaskSetManager) { + + var poolName = "default" + if(manager.taskSet.properties != null) + poolName = manager.taskSet.properties.getProperty("spark.scheduler.cluster.fair.pool","default") + + + this.synchronized { + //have to check that poolName exists + if(poolNameToPool.contains(poolName)) + { + poolNameToPool(poolName).activeTaskSetsQueue -= manager + } + else + { + poolNameToPool("default").activeTaskSetsQueue -= manager + } + //activeTaskSetsQueue -= manager + activeTaskSets -= manager.taskSet.id + taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id) + taskIdToExecutorId --= taskSetTaskIds(manager.taskSet.id) + taskSetTaskIds.remove(manager.taskSet.id) + } + //backend.reviveOffers() + } + + /** + * This is the comparison function used for sorting to determine which + * pool to allocate next based on fairness. + * The algorithm is as follows: we sort by the pool's running tasks to weight ratio + * (pools number running tast / pool's weight) + */ + def poolFairCompFn(pool1: Pool, pool2: Pool): Boolean = { + val tasksToWeightRatio1 = pool1.numRunningTasks.toDouble / pool1.weight.toDouble + val tasksToWeightRatio2 = pool2.numRunningTasks.toDouble / pool2.weight.toDouble + var res = Math.signum(tasksToWeightRatio1 - tasksToWeightRatio2) + if (res == 0) { + //Jobs are tied in fairness ratio. We break the tie by name + res = pool1.name.compareTo(pool2.name) + } + if (res < 0) + return true + else + return false + } + + /** + * Called by cluster manager to offer resources on slaves. We respond by asking our active task + * sets for tasks in order of priority. We fill each node with tasks in a fair manner so + * that tasks are balanced across the cluster. + */ + override def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = { + synchronized { + SparkEnv.set(sc.env) + // Mark each slave as alive and remember its hostname + for (o <- offers) { + executorIdToHost(o.executorId) = o.hostname + if (!executorsByHost.contains(o.hostname)) { + executorsByHost(o.hostname) = new HashSet() + } + } + // Build a list of tasks to assign to each slave + val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) + val availableCpus = offers.map(o => o.cores).toArray + var launchedTask = false + + for (i <- 0 until offers.size) { //we loop through the list of offers + val execId = offers(i).executorId + val host = offers(i).hostname + var breakOut = false + while(availableCpus(i) > 0 && !breakOut) { + breakable{ + launchedTask = false + for (pool <- pools.sortWith(poolFairCompFn)) { //we loop through the list of pools + if(!pool.activeTaskSetsQueue.isEmpty) { + //sort the tasksetmanager in the pool + pool.activeTaskSetsQueue.sortBy(m => (m.taskSet.priority, m.taskSet.stageId)) + for(manager <- pool.activeTaskSetsQueue) { //we loop through the activeTaskSets in this pool +// val manager = pool.activeTaskSetsQueue.head + //Make an offer + manager.slaveOffer(execId, host, availableCpus(i)) match { + case Some(task) => + tasks(i) += task + val tid = task.taskId + taskIdToTaskSetId(tid) = manager.taskSet.id + taskSetTaskIds(manager.taskSet.id) += tid + taskIdToExecutorId(tid) = execId + activeExecutorIds += execId + executorsByHost(host) += execId + availableCpus(i) -= 1 + pool.numRunningTasks += 1 + launchedTask = true + logInfo("launched task for pool"+pool.name); + break + case None => {} + } + } + } + } + //If there is not one pool that can assign the task then we have to exit the outer loop and continue to the next offer + if(!launchedTask){ + breakOut = true + } + } + } + } + if (tasks.size > 0) { + hasLaunchedTask = true + } + return tasks + } + } + + override def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { + var taskSetToUpdate: Option[TaskSetManager] = None + var failedExecutor: Option[String] = None + var taskFailed = false + synchronized { + try { + if (state == TaskState.LOST && taskIdToExecutorId.contains(tid)) { + // We lost this entire executor, so remember that it's gone + val execId = taskIdToExecutorId(tid) + if (activeExecutorIds.contains(execId)) { + removeExecutor(execId) + failedExecutor = Some(execId) + } + } + taskIdToTaskSetId.get(tid) match { + case Some(taskSetId) => + if (activeTaskSets.contains(taskSetId)) { + taskSetToUpdate = Some(activeTaskSets(taskSetId)) + } + if (TaskState.isFinished(state)) { + taskIdToTaskSetId.remove(tid) + if (taskSetTaskIds.contains(taskSetId)) { + taskSetTaskIds(taskSetId) -= tid + } + taskIdToExecutorId.remove(tid) + } + if (state == TaskState.FAILED) { + taskFailed = true + } + case None => + logInfo("Ignoring update from TID " + tid + " because its task set is gone") + } + } catch { + case e: Exception => logError("Exception in statusUpdate", e) + } + } + // Update the task set and DAGScheduler without holding a lock on this, since that can deadlock + if (taskSetToUpdate != None) { + taskSetToUpdate.get.statusUpdate(tid, state, serializedData) + } + if (failedExecutor != None) { + listener.executorLost(failedExecutor.get) + backend.reviveOffers() + } + if (taskFailed) { + // Also revive offers if a task had failed for some reason other than host lost + backend.reviveOffers() + } + } + + // Check for speculatable tasks in all our active jobs. + override def checkSpeculatableTasks() { + var shouldRevive = false + synchronized { + for (pool <- pools) { + for (ts <- pool.activeTaskSetsQueue) { + shouldRevive |= ts.checkSpeculatableTasks() + } + } + } + if (shouldRevive) { + backend.reviveOffers() + } + } + + /** Remove an executor from all our data structures and mark it as lost */ + private def removeExecutor(executorId: String) { + activeExecutorIds -= executorId + val host = executorIdToHost(executorId) + val execs = executorsByHost.getOrElse(host, new HashSet) + execs -= executorId + if (execs.isEmpty) { + executorsByHost -= host + } + executorIdToHost -= executorId + for (pool <- pools) { + pool.activeTaskSetsQueue.foreach(_.executorLost(executorId, host)) + } + } + +} + +/** + * An internal representation of a pool. It contains an ArrayBuffer of TaskSets and also weight and minshare + */ +class Pool(val name: String, val weight: Int) +{ + var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] + var numRunningTasks: Int = 0 +} diff --git a/core/src/main/scala/spark/scheduler/cluster/fair/FairTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/fair/FairTaskSetManager.scala new file mode 100644 index 0000000000..4b0277d2d5 --- /dev/null +++ b/core/src/main/scala/spark/scheduler/cluster/fair/FairTaskSetManager.scala @@ -0,0 +1,130 @@ +package spark.scheduler.cluster.fair + +import scala.collection.mutable.ArrayBuffer + +import spark._ +import spark.scheduler._ +import spark.scheduler.cluster._ +import spark.TaskState.TaskState +import java.nio.ByteBuffer + +/** + * Schedules the tasks within a single TaskSet in the FairClusterScheduler. + */ +private[spark] class FairTaskSetManager(sched: FairClusterScheduler, override val taskSet: TaskSet) extends TaskSetManager(sched, taskSet) with Logging { + + // Add a task to all the pending-task lists that it should be on. + private def addPendingTask(index: Int) { + val locations = tasks(index).preferredLocations.toSet & sched.hostsAlive + if (locations.size == 0) { + pendingTasksWithNoPrefs += index + } else { + for (host <- locations) { + val list = pendingTasksForHost.getOrElseUpdate(host, ArrayBuffer()) + list += index + } + } + allPendingTasks += index + } + + override def taskFinished(tid: Long, state: TaskState, serializedData: ByteBuffer) { + val info = taskInfos(tid) + if (info.failed) { + // We might get two task-lost messages for the same task in coarse-grained Mesos mode, + // or even from Mesos itself when acks get delayed. + return + } + val index = info.index + info.markSuccessful() + sched.taskFinished(this) + if (!finished(index)) { + tasksFinished += 1 + logInfo("Finished TID %s in %d ms (progress: %d/%d)".format( + tid, info.duration, tasksFinished, numTasks)) + // Deserialize task result and pass it to the scheduler + val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) + result.metrics.resultSize = serializedData.limit() + sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) + // Mark finished and stop if we've finished all the tasks + finished(index) = true + if (tasksFinished == numTasks) { + sched.taskSetFinished(this) + } + } else { + logInfo("Ignoring task-finished event for TID " + tid + + " because task " + index + " is already finished") + } + } + + override def taskLost(tid: Long, state: TaskState, serializedData: ByteBuffer) { + val info = taskInfos(tid) + if (info.failed) { + // We might get two task-lost messages for the same task in coarse-grained Mesos mode, + // or even from Mesos itself when acks get delayed. + return + } + val index = info.index + info.markFailed() + //Bookkeeping necessary for the pools in the scheduler + sched.taskFinished(this) + if (!finished(index)) { + logInfo("Lost TID %s (task %s:%d)".format(tid, taskSet.id, index)) + copiesRunning(index) -= 1 + // Check if the problem is a map output fetch failure. In that case, this + // task will never succeed on any node, so tell the scheduler about it. + if (serializedData != null && serializedData.limit() > 0) { + val reason = ser.deserialize[TaskEndReason](serializedData, getClass.getClassLoader) + reason match { + case fetchFailed: FetchFailed => + logInfo("Loss was due to fetch failure from " + fetchFailed.bmAddress) + sched.listener.taskEnded(tasks(index), fetchFailed, null, null, info, null) + finished(index) = true + tasksFinished += 1 + sched.taskSetFinished(this) + return + + case ef: ExceptionFailure => + val key = ef.exception.toString + val now = System.currentTimeMillis + val (printFull, dupCount) = { + if (recentExceptions.contains(key)) { + val (dupCount, printTime) = recentExceptions(key) + if (now - printTime > EXCEPTION_PRINT_INTERVAL) { + recentExceptions(key) = (0, now) + (true, 0) + } else { + recentExceptions(key) = (dupCount + 1, printTime) + (false, dupCount + 1) + } + } else { + recentExceptions(key) = (0, now) + (true, 0) + } + } + if (printFull) { + val locs = ef.exception.getStackTrace.map(loc => "\tat %s".format(loc.toString)) + logInfo("Loss was due to %s\n%s".format(ef.exception.toString, locs.mkString("\n"))) + } else { + logInfo("Loss was due to %s [duplicate %d]".format(ef.exception.toString, dupCount)) + } + + case _ => {} + } + } + // On non-fetch failures, re-enqueue the task as pending for a max number of retries + addPendingTask(index) + // Count failed attempts only on FAILED and LOST state (not on KILLED) + if (state == TaskState.FAILED || state == TaskState.LOST) { + numFailures(index) += 1 + if (numFailures(index) > MAX_TASK_FAILURES) { + logError("Task %s:%d failed more than %d times; aborting job".format( + taskSet.id, index, MAX_TASK_FAILURES)) + abort("Task %s:%d failed more than %d times".format(taskSet.id, index, MAX_TASK_FAILURES)) + } + } + } else { + logInfo("Ignoring task-lost event for TID " + tid + + " because task " + index + " is already finished") + } + } +} \ No newline at end of file -- cgit v1.2.3 From 54ed7c4af4591ebfec31bd168f830ef3ac01a41f Mon Sep 17 00:00:00 2001 From: Harold Lim Date: Mon, 4 Mar 2013 16:57:46 -0500 Subject: Changed the name of the system property to set the allocation xml --- .../main/scala/spark/scheduler/cluster/fair/FairClusterScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/fair/FairClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/fair/FairClusterScheduler.scala index 37d98ccb2a..591736faa2 100644 --- a/core/src/main/scala/spark/scheduler/cluster/fair/FairClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/fair/FairClusterScheduler.scala @@ -31,7 +31,7 @@ private[spark] class FairClusterScheduler(override val sc: SparkContext) with Logging { - val schedulerAllocFile = System.getProperty("mapred.fairscheduler.allocation.file","unspecified") + val schedulerAllocFile = System.getProperty("spark.fairscheduler.allocation.file","unspecified") val poolNameToPool= new HashMap[String, Pool] var pools = new ArrayBuffer[Pool] -- cgit v1.2.3 From b5325182a3e92ff80185850c39cf70680fca46b7 Mon Sep 17 00:00:00 2001 From: Harold Lim Date: Fri, 8 Mar 2013 15:15:59 -0500 Subject: Updated/Refactored the Fair Task Scheduler. It does not inherit ClusterScheduler anymore. Rather, ClusterScheduler internally uses TaskSetQueuesManager that handles the scheduling of taskset queues. This is the class that should be extended to support other scheduling policies --- core/src/main/scala/spark/SparkContext.scala | 19 +- .../spark/scheduler/cluster/ClusterScheduler.scala | 43 ++- .../cluster/FIFOTaskSetQueuesManager.scala | 63 ++++ .../cluster/FairTaskSetQueuesManager.scala | 183 +++++++++++ .../spark/scheduler/cluster/TaskSetManager.scala | 2 + .../scheduler/cluster/TaskSetQueuesManager.scala | 19 ++ .../cluster/fair/FairClusterScheduler.scala | 341 --------------------- .../cluster/fair/FairTaskSetManager.scala | 130 -------- 8 files changed, 312 insertions(+), 488 deletions(-) create mode 100644 core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala create mode 100644 core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala create mode 100644 core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala delete mode 100644 core/src/main/scala/spark/scheduler/cluster/fair/FairClusterScheduler.scala delete mode 100644 core/src/main/scala/spark/scheduler/cluster/fair/FairTaskSetManager.scala diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index bd2261cf0d..f6ee399898 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -39,7 +39,7 @@ import spark.partial.PartialResult import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD} import spark.scheduler._ import spark.scheduler.local.LocalScheduler -import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler} +import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler, TaskSetQueuesManager} import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import spark.storage.BlockManagerUI import spark.util.{MetadataCleaner, TimeStampedHashMap} @@ -77,7 +77,7 @@ class SparkContext( //Set the default task scheduler if (System.getProperty("spark.cluster.taskscheduler") == null) { - System.setProperty("spark.cluster.taskscheduler", "spark.scheduler.cluster.ClusterScheduler") + System.setProperty("spark.cluster.taskscheduler", "spark.scheduler.cluster.FIFOTaskSetQueuesManager") } private val isLocal = (master == "local" || master.startsWith("local[")) @@ -144,9 +144,10 @@ class SparkContext( new LocalScheduler(threads.toInt, maxFailures.toInt, this) case SPARK_REGEX(sparkUrl) => - val scheduler = Class.forName(System.getProperty("spark.cluster.taskscheduler")).getConstructors()(0).newInstance(Array[AnyRef](this):_*).asInstanceOf[ClusterScheduler] + val scheduler = new ClusterScheduler(this)//Class.forName(System.getProperty("spark.cluster.taskscheduler")).getConstructors()(0).newInstance(Array[AnyRef](this):_*).asInstanceOf[ClusterScheduler] val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName) - scheduler.initialize(backend) + val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")).newInstance().asInstanceOf[TaskSetQueuesManager] + scheduler.initialize(backend, taskSetQueuesManager) scheduler case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => @@ -160,12 +161,13 @@ class SparkContext( memoryPerSlaveInt, sparkMemEnvInt)) } - val scheduler = Class.forName(System.getProperty("spark.cluster.taskscheduler")).getConstructors()(0).newInstance(Array[AnyRef](this):_*).asInstanceOf[ClusterScheduler] + val scheduler = new ClusterScheduler(this)//Class.forName(System.getProperty("spark.cluster.taskscheduler")).getConstructors()(0).newInstance(Array[AnyRef](this):_*).asInstanceOf[ClusterScheduler] val localCluster = new LocalSparkCluster( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) val sparkUrl = localCluster.start() val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName) - scheduler.initialize(backend) + val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")).newInstance().asInstanceOf[TaskSetQueuesManager] + scheduler.initialize(backend, taskSetQueuesManager) backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => { localCluster.stop() } @@ -176,7 +178,7 @@ class SparkContext( logWarning("Master %s does not match expected format, parsing as Mesos URL".format(master)) } MesosNativeLibrary.load() - val scheduler = Class.forName(System.getProperty("spark.cluster.taskscheduler")).getConstructors()(0).newInstance(Array[AnyRef](this):_*).asInstanceOf[ClusterScheduler] + val scheduler = new ClusterScheduler(this)//Class.forName(System.getProperty("spark.cluster.taskscheduler")).getConstructors()(0).newInstance(Array[AnyRef](this):_*).asInstanceOf[ClusterScheduler] val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean val masterWithoutProtocol = master.replaceFirst("^mesos://", "") // Strip initial mesos:// val backend = if (coarseGrained) { @@ -184,7 +186,8 @@ class SparkContext( } else { new MesosSchedulerBackend(scheduler, this, masterWithoutProtocol, appName) } - scheduler.initialize(backend) + val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")).newInstance().asInstanceOf[TaskSetQueuesManager] + scheduler.initialize(backend, taskSetQueuesManager) scheduler } } diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 26fdef101b..0b5bf7a86c 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -27,7 +27,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val STARVATION_TIMEOUT = System.getProperty("spark.starvation.timeout", "15000").toLong val activeTaskSets = new HashMap[String, TaskSetManager] - var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] + // var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] val taskIdToTaskSetId = new HashMap[Long, String] val taskIdToExecutorId = new HashMap[Long, String] @@ -61,13 +61,16 @@ private[spark] class ClusterScheduler(val sc: SparkContext) var backend: SchedulerBackend = null val mapOutputTracker = SparkEnv.get.mapOutputTracker + + var taskSetQueuesManager: TaskSetQueuesManager = null override def setListener(listener: TaskSchedulerListener) { this.listener = listener } - def initialize(context: SchedulerBackend) { + def initialize(context: SchedulerBackend, taskSetQueuesManager: TaskSetQueuesManager) { backend = context + this.taskSetQueuesManager = taskSetQueuesManager } def newTaskId(): Long = nextTaskId.getAndIncrement() @@ -99,7 +102,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) this.synchronized { val manager = new TaskSetManager(this, taskSet) activeTaskSets(taskSet.id) = manager - activeTaskSetsQueue += manager + taskSetQueuesManager.addTaskSetManager(manager) taskSetTaskIds(taskSet.id) = new HashSet[Long]() if (hasReceivedTask == false) { @@ -122,13 +125,19 @@ private[spark] class ClusterScheduler(val sc: SparkContext) def taskSetFinished(manager: TaskSetManager) { this.synchronized { activeTaskSets -= manager.taskSet.id - activeTaskSetsQueue -= manager + taskSetQueuesManager.removeTaskSetManager(manager) taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id) taskIdToExecutorId --= taskSetTaskIds(manager.taskSet.id) taskSetTaskIds.remove(manager.taskSet.id) } } + def taskFinished(manager: TaskSetManager) { + this.synchronized { + taskSetQueuesManager.taskFinished(manager) + } + } + /** * Called by cluster manager to offer resources on slaves. We respond by asking our active task * sets for tasks in order of priority. We fill each node with tasks in a round-robin manner so @@ -144,8 +153,26 @@ private[spark] class ClusterScheduler(val sc: SparkContext) executorsByHost(o.hostname) = new HashSet() } } + // Build a list of tasks to assign to each slave val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) + val taskSetIds = taskSetQueuesManager.receiveOffer(tasks, offers) + //We populate the necessary bookkeeping structures + for (i <- 0 until offers.size) { + val execId = offers(i).executorId + val host = offers(i).hostname + for(j <- 0 until tasks(i).size) { + val tid = tasks(i)(j).taskId + val taskSetid = taskSetIds(i)(j) + taskIdToTaskSetId(tid) = taskSetid + taskSetTaskIds(taskSetid) += tid + taskIdToExecutorId(tid) = execId + activeExecutorIds += execId + executorsByHost(host) += execId + } + } + + /*val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) val availableCpus = offers.map(o => o.cores).toArray var launchedTask = false for (manager <- activeTaskSetsQueue.sortBy(m => (m.taskSet.priority, m.taskSet.stageId))) { @@ -170,7 +197,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } } } while (launchedTask) - } + }*/ if (tasks.size > 0) { hasLaunchedTask = true } @@ -264,9 +291,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) def checkSpeculatableTasks() { var shouldRevive = false synchronized { - for (ts <- activeTaskSetsQueue) { - shouldRevive |= ts.checkSpeculatableTasks() - } + shouldRevive = taskSetQueuesManager.checkSpeculatableTasks() } if (shouldRevive) { backend.reviveOffers() @@ -309,6 +334,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext) executorsByHost -= host } executorIdToHost -= executorId - activeTaskSetsQueue.foreach(_.executorLost(executorId, host)) + taskSetQueuesManager.removeExecutor(executorId, host) } } diff --git a/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala new file mode 100644 index 0000000000..99a9c94222 --- /dev/null +++ b/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala @@ -0,0 +1,63 @@ +package spark.scheduler.cluster + +import scala.collection.mutable.ArrayBuffer + +import spark.Logging + +/** + * A FIFO Implementation of the TaskSetQueuesManager + */ +private[spark] class FIFOTaskSetQueuesManager extends TaskSetQueuesManager with Logging { + + var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] + + override def addTaskSetManager(manager: TaskSetManager) { + activeTaskSetsQueue += manager + } + + override def removeTaskSetManager(manager: TaskSetManager) { + activeTaskSetsQueue -= manager + } + + override def taskFinished(manager: TaskSetManager) { + //do nothing + } + + override def removeExecutor(executorId: String, host: String) { + activeTaskSetsQueue.foreach(_.executorLost(executorId, host)) + } + + override def receiveOffer(tasks: Seq[ArrayBuffer[TaskDescription]], offers: Seq[WorkerOffer]): Seq[Seq[String]] = { + val taskSetIds = offers.map(o => new ArrayBuffer[String](o.cores)) + val availableCpus = offers.map(o => o.cores).toArray + var launchedTask = false + for (manager <- activeTaskSetsQueue.sortBy(m => (m.taskSet.priority, m.taskSet.stageId))) { + do { + launchedTask = false + for (i <- 0 until offers.size) { + val execId = offers(i).executorId + val host = offers(i).hostname + manager.slaveOffer(execId, host, availableCpus(i)) match { + case Some(task) => + tasks(i) += task + taskSetIds(i) += manager.taskSet.id + availableCpus(i) -= 1 + launchedTask = true + + case None => {} + } + } + } while (launchedTask) + } + return taskSetIds + } + + override def checkSpeculatableTasks(): Boolean = { + var shouldRevive = false + for (ts <- activeTaskSetsQueue) { + shouldRevive |= ts.checkSpeculatableTasks() + } + return shouldRevive + } + +} \ No newline at end of file diff --git a/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala new file mode 100644 index 0000000000..ca308a5229 --- /dev/null +++ b/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala @@ -0,0 +1,183 @@ +package spark.scheduler.cluster + +import java.io.{File, FileInputStream, FileOutputStream} + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet +import scala.util.control.Breaks._ +import scala.xml._ + +import spark.Logging + +/** + * A Fair Implementation of the TaskSetQueuesManager + * + * The current implementation makes the following assumptions: A pool has a fixed configuration of weight. + * Within a pool, it just uses FIFO. + * Also, currently we assume that pools are statically defined + * We currently don't support min shares + */ +private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with Logging { + + val schedulerAllocFile = System.getProperty("spark.fairscheduler.allocation.file","unspecified") + val poolNameToPool= new HashMap[String, Pool] + var pools = new ArrayBuffer[Pool] + + loadPoolProperties() + + def loadPoolProperties() { + //first check if the file exists + val file = new File(schedulerAllocFile) + if(!file.exists()) { + //if file does not exist, we just create 1 pool, default + val pool = new Pool("default",100) + pools += pool + poolNameToPool("default") = pool + logInfo("Created a default pool with weight = 100") + } + else { + val xml = XML.loadFile(file) + for (poolNode <- (xml \\ "pool")) { + if((poolNode \ "weight").text != ""){ + val pool = new Pool((poolNode \ "@name").text,(poolNode \ "weight").text.toInt) + pools += pool + poolNameToPool((poolNode \ "@name").text) = pool + logInfo("Created pool "+ pool.name +"with weight = "+pool.weight) + } else { + val pool = new Pool((poolNode \ "@name").text,100) + pools += pool + poolNameToPool((poolNode \ "@name").text) = pool + logInfo("Created pool "+ pool.name +"with weight = 100") + } + } + if(!poolNameToPool.contains("default")) { + val pool = new Pool("default", 100) + pools += pool + poolNameToPool("default") = pool + logInfo("Created a default pool with weight = 100") + } + + } + } + + override def addTaskSetManager(manager: TaskSetManager) { + var poolName = "default" + if(manager.taskSet.properties != null) + poolName = manager.taskSet.properties.getProperty("spark.scheduler.cluster.fair.pool","default") + if(poolNameToPool.contains(poolName)) + poolNameToPool(poolName).activeTaskSetsQueue += manager + else + poolNameToPool("default").activeTaskSetsQueue += manager + logInfo("Added task set " + manager.taskSet.id + " tasks to pool "+poolName) + + } + + override def removeTaskSetManager(manager: TaskSetManager) { + var poolName = "default" + if(manager.taskSet.properties != null) + poolName = manager.taskSet.properties.getProperty("spark.scheduler.cluster.fair.pool","default") + if(poolNameToPool.contains(poolName)) + poolNameToPool(poolName).activeTaskSetsQueue -= manager + else + poolNameToPool("default").activeTaskSetsQueue -= manager + } + + override def taskFinished(manager: TaskSetManager) { + var poolName = "default" + if(manager.taskSet.properties != null) + poolName = manager.taskSet.properties.getProperty("spark.scheduler.cluster.fair.pool","default") + if(poolNameToPool.contains(poolName)) + poolNameToPool(poolName).numRunningTasks -= 1 + else + poolNameToPool("default").numRunningTasks -= 1 + } + + override def removeExecutor(executorId: String, host: String) { + for (pool <- pools) { + pool.activeTaskSetsQueue.foreach(_.executorLost(executorId, host)) + } + } + + /** + * This is the comparison function used for sorting to determine which + * pool to allocate next based on fairness. + * The algorithm is as follows: we sort by the pool's running tasks to weight ratio + * (pools number running tast / pool's weight) + */ + def poolFairCompFn(pool1: Pool, pool2: Pool): Boolean = { + val tasksToWeightRatio1 = pool1.numRunningTasks.toDouble / pool1.weight.toDouble + val tasksToWeightRatio2 = pool2.numRunningTasks.toDouble / pool2.weight.toDouble + var res = Math.signum(tasksToWeightRatio1 - tasksToWeightRatio2) + if (res == 0) { + //Jobs are tied in fairness ratio. We break the tie by name + res = pool1.name.compareTo(pool2.name) + } + if (res < 0) + return true + else + return false + } + + override def receiveOffer(tasks: Seq[ArrayBuffer[TaskDescription]], offers: Seq[WorkerOffer]): Seq[Seq[String]] = { + val taskSetIds = offers.map(o => new ArrayBuffer[String](o.cores)) + val availableCpus = offers.map(o => o.cores).toArray + var launchedTask = false + + for (i <- 0 until offers.size) { //we loop through the list of offers + val execId = offers(i).executorId + val host = offers(i).hostname + var breakOut = false + while(availableCpus(i) > 0 && !breakOut) { + breakable{ + launchedTask = false + for (pool <- pools.sortWith(poolFairCompFn)) { //we loop through the list of pools + if(!pool.activeTaskSetsQueue.isEmpty) { + //sort the tasksetmanager in the pool + pool.activeTaskSetsQueue.sortBy(m => (m.taskSet.priority, m.taskSet.stageId)) + for(manager <- pool.activeTaskSetsQueue) { //we loop through the activeTaskSets in this pool + //Make an offer + manager.slaveOffer(execId, host, availableCpus(i)) match { + case Some(task) => + tasks(i) += task + taskSetIds(i) += manager.taskSet.id + availableCpus(i) -= 1 + pool.numRunningTasks += 1 + launchedTask = true + logInfo("launched task for pool"+pool.name); + break + case None => {} + } + } + } + } + //If there is not one pool that can assign the task then we have to exit the outer loop and continue to the next offer + if(!launchedTask){ + breakOut = true + } + } + } + } + return taskSetIds + } + + override def checkSpeculatableTasks(): Boolean = { + var shouldRevive = false + for (pool <- pools) { + for (ts <- pool.activeTaskSetsQueue) { + shouldRevive |= ts.checkSpeculatableTasks() + } + } + return shouldRevive + } +} + +/** + * An internal representation of a pool. It contains an ArrayBuffer of TaskSets and also weight + */ +class Pool(val name: String, val weight: Int) +{ + var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] + var numRunningTasks: Int = 0 +} \ No newline at end of file diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index c9f2c48804..015092b60b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -253,6 +253,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe } val index = info.index info.markSuccessful() + sched.taskFinished(this) if (!finished(index)) { tasksFinished += 1 logInfo("Finished TID %s in %d ms (progress: %d/%d)".format( @@ -281,6 +282,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe } val index = info.index info.markFailed() + sched.taskFinished(this) if (!finished(index)) { logInfo("Lost TID %s (task %s:%d)".format(tid, taskSet.id, index)) copiesRunning(index) -= 1 diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala new file mode 100644 index 0000000000..b0c30e9e8b --- /dev/null +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala @@ -0,0 +1,19 @@ +package spark.scheduler.cluster + +import scala.collection.mutable.ArrayBuffer + +/** + * An interface for managing TaskSet queue/s that allows plugging different policy for + * offering tasks to resources + * + */ +private[spark] trait TaskSetQueuesManager { + def addTaskSetManager(manager: TaskSetManager): Unit + def removeTaskSetManager(manager: TaskSetManager): Unit + def taskFinished(manager: TaskSetManager): Unit + def removeExecutor(executorId: String, host: String): Unit + //The receiveOffers function, accepts tasks and offers. It populates the tasks to the actual task from TaskSet + //It returns a list of TaskSet ID that corresponds to each assigned tasks + def receiveOffer(tasks: Seq[ArrayBuffer[TaskDescription]], offers: Seq[WorkerOffer]): Seq[Seq[String]] + def checkSpeculatableTasks(): Boolean +} \ No newline at end of file diff --git a/core/src/main/scala/spark/scheduler/cluster/fair/FairClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/fair/FairClusterScheduler.scala deleted file mode 100644 index 591736faa2..0000000000 --- a/core/src/main/scala/spark/scheduler/cluster/fair/FairClusterScheduler.scala +++ /dev/null @@ -1,341 +0,0 @@ -package spark.scheduler.cluster.fair - -import java.io.{File, FileInputStream, FileOutputStream} -import java.util.{TimerTask, Timer} - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet -import scala.util.control.Breaks._ -import scala.xml._ - -import spark._ -import spark.TaskState.TaskState -import spark.scheduler._ -import spark.scheduler.cluster._ -import java.nio.ByteBuffer -import java.util.concurrent.atomic.AtomicLong -import scala.io.Source - -/** - * An implementation of a fair TaskScheduler, for running tasks on a cluster. Clients should first call - * start(), then submit task sets through the runTasks method. - * - * The current implementation makes the following assumptions: A pool has a fixed configuration of weight. - * Within a pool, it just uses FIFO. - * Also, currently we assume that pools are statically defined - * We currently don't support min shares - */ -private[spark] class FairClusterScheduler(override val sc: SparkContext) - extends ClusterScheduler(sc) - with Logging { - - - val schedulerAllocFile = System.getProperty("spark.fairscheduler.allocation.file","unspecified") - - val poolNameToPool= new HashMap[String, Pool] - var pools = new ArrayBuffer[Pool] - - loadPoolProperties() - - def loadPoolProperties() { - //first check if the file exists - val file = new File(schedulerAllocFile) - if(!file.exists()) { - //if file does not exist, we just create 1 pool, default - val pool = new Pool("default",100) - pools += pool - poolNameToPool("default") = pool - logInfo("Created a default pool with weight = 100") - } - else { - val xml = XML.loadFile(file) - for (poolNode <- (xml \\ "pool")) { - if((poolNode \ "weight").text != ""){ - val pool = new Pool((poolNode \ "@name").text,(poolNode \ "weight").text.toInt) - pools += pool - poolNameToPool((poolNode \ "@name").text) = pool - logInfo("Created pool "+ pool.name +"with weight = "+pool.weight) - } else { - val pool = new Pool((poolNode \ "@name").text,100) - pools += pool - poolNameToPool((poolNode \ "@name").text) = pool - logInfo("Created pool "+ pool.name +"with weight = 100") - } - } - if(!poolNameToPool.contains("default")) { - val pool = new Pool("default", 100) - pools += pool - poolNameToPool("default") = pool - logInfo("Created a default pool with weight = 100") - } - - } - } - - def taskFinished(manager: TaskSetManager) { - var poolName = "default" - if(manager.taskSet.properties != null) - poolName = manager.taskSet.properties.getProperty("spark.scheduler.cluster.fair.pool","default") - - this.synchronized { - //have to check that poolName exists - if(poolNameToPool.contains(poolName)) - { - poolNameToPool(poolName).numRunningTasks -= 1 - } - else - { - poolNameToPool("default").numRunningTasks -= 1 - } - } - } - - override def submitTasks(taskSet: TaskSet) { - val tasks = taskSet.tasks - - - var poolName = "default" - if(taskSet.properties != null) - poolName = taskSet.properties.getProperty("spark.scheduler.cluster.fair.pool","default") - - this.synchronized { - if(poolNameToPool.contains(poolName)) - { - val manager = new FairTaskSetManager(this, taskSet) - poolNameToPool(poolName).activeTaskSetsQueue += manager - activeTaskSets(taskSet.id) = manager - //activeTaskSetsQueue += manager - taskSetTaskIds(taskSet.id) = new HashSet[Long]() - logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks to pool "+poolName) - } - else //If the pool name does not exists, where do we put them? We put them in default - { - val manager = new FairTaskSetManager(this, taskSet) - poolNameToPool("default").activeTaskSetsQueue += manager - activeTaskSets(taskSet.id) = manager - //activeTaskSetsQueue += manager - taskSetTaskIds(taskSet.id) = new HashSet[Long]() - logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks to pool default") - } - if (hasReceivedTask == false) { - starvationTimer.scheduleAtFixedRate(new TimerTask() { - override def run() { - if (!hasLaunchedTask) { - logWarning("Initial job has not accepted any resources; " + - "check your cluster UI to ensure that workers are registered") - } else { - this.cancel() - } - } - }, STARVATION_TIMEOUT, STARVATION_TIMEOUT) - } - hasReceivedTask = true; - - } - backend.reviveOffers() - } - - override def taskSetFinished(manager: TaskSetManager) { - - var poolName = "default" - if(manager.taskSet.properties != null) - poolName = manager.taskSet.properties.getProperty("spark.scheduler.cluster.fair.pool","default") - - - this.synchronized { - //have to check that poolName exists - if(poolNameToPool.contains(poolName)) - { - poolNameToPool(poolName).activeTaskSetsQueue -= manager - } - else - { - poolNameToPool("default").activeTaskSetsQueue -= manager - } - //activeTaskSetsQueue -= manager - activeTaskSets -= manager.taskSet.id - taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id) - taskIdToExecutorId --= taskSetTaskIds(manager.taskSet.id) - taskSetTaskIds.remove(manager.taskSet.id) - } - //backend.reviveOffers() - } - - /** - * This is the comparison function used for sorting to determine which - * pool to allocate next based on fairness. - * The algorithm is as follows: we sort by the pool's running tasks to weight ratio - * (pools number running tast / pool's weight) - */ - def poolFairCompFn(pool1: Pool, pool2: Pool): Boolean = { - val tasksToWeightRatio1 = pool1.numRunningTasks.toDouble / pool1.weight.toDouble - val tasksToWeightRatio2 = pool2.numRunningTasks.toDouble / pool2.weight.toDouble - var res = Math.signum(tasksToWeightRatio1 - tasksToWeightRatio2) - if (res == 0) { - //Jobs are tied in fairness ratio. We break the tie by name - res = pool1.name.compareTo(pool2.name) - } - if (res < 0) - return true - else - return false - } - - /** - * Called by cluster manager to offer resources on slaves. We respond by asking our active task - * sets for tasks in order of priority. We fill each node with tasks in a fair manner so - * that tasks are balanced across the cluster. - */ - override def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = { - synchronized { - SparkEnv.set(sc.env) - // Mark each slave as alive and remember its hostname - for (o <- offers) { - executorIdToHost(o.executorId) = o.hostname - if (!executorsByHost.contains(o.hostname)) { - executorsByHost(o.hostname) = new HashSet() - } - } - // Build a list of tasks to assign to each slave - val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) - val availableCpus = offers.map(o => o.cores).toArray - var launchedTask = false - - for (i <- 0 until offers.size) { //we loop through the list of offers - val execId = offers(i).executorId - val host = offers(i).hostname - var breakOut = false - while(availableCpus(i) > 0 && !breakOut) { - breakable{ - launchedTask = false - for (pool <- pools.sortWith(poolFairCompFn)) { //we loop through the list of pools - if(!pool.activeTaskSetsQueue.isEmpty) { - //sort the tasksetmanager in the pool - pool.activeTaskSetsQueue.sortBy(m => (m.taskSet.priority, m.taskSet.stageId)) - for(manager <- pool.activeTaskSetsQueue) { //we loop through the activeTaskSets in this pool -// val manager = pool.activeTaskSetsQueue.head - //Make an offer - manager.slaveOffer(execId, host, availableCpus(i)) match { - case Some(task) => - tasks(i) += task - val tid = task.taskId - taskIdToTaskSetId(tid) = manager.taskSet.id - taskSetTaskIds(manager.taskSet.id) += tid - taskIdToExecutorId(tid) = execId - activeExecutorIds += execId - executorsByHost(host) += execId - availableCpus(i) -= 1 - pool.numRunningTasks += 1 - launchedTask = true - logInfo("launched task for pool"+pool.name); - break - case None => {} - } - } - } - } - //If there is not one pool that can assign the task then we have to exit the outer loop and continue to the next offer - if(!launchedTask){ - breakOut = true - } - } - } - } - if (tasks.size > 0) { - hasLaunchedTask = true - } - return tasks - } - } - - override def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { - var taskSetToUpdate: Option[TaskSetManager] = None - var failedExecutor: Option[String] = None - var taskFailed = false - synchronized { - try { - if (state == TaskState.LOST && taskIdToExecutorId.contains(tid)) { - // We lost this entire executor, so remember that it's gone - val execId = taskIdToExecutorId(tid) - if (activeExecutorIds.contains(execId)) { - removeExecutor(execId) - failedExecutor = Some(execId) - } - } - taskIdToTaskSetId.get(tid) match { - case Some(taskSetId) => - if (activeTaskSets.contains(taskSetId)) { - taskSetToUpdate = Some(activeTaskSets(taskSetId)) - } - if (TaskState.isFinished(state)) { - taskIdToTaskSetId.remove(tid) - if (taskSetTaskIds.contains(taskSetId)) { - taskSetTaskIds(taskSetId) -= tid - } - taskIdToExecutorId.remove(tid) - } - if (state == TaskState.FAILED) { - taskFailed = true - } - case None => - logInfo("Ignoring update from TID " + tid + " because its task set is gone") - } - } catch { - case e: Exception => logError("Exception in statusUpdate", e) - } - } - // Update the task set and DAGScheduler without holding a lock on this, since that can deadlock - if (taskSetToUpdate != None) { - taskSetToUpdate.get.statusUpdate(tid, state, serializedData) - } - if (failedExecutor != None) { - listener.executorLost(failedExecutor.get) - backend.reviveOffers() - } - if (taskFailed) { - // Also revive offers if a task had failed for some reason other than host lost - backend.reviveOffers() - } - } - - // Check for speculatable tasks in all our active jobs. - override def checkSpeculatableTasks() { - var shouldRevive = false - synchronized { - for (pool <- pools) { - for (ts <- pool.activeTaskSetsQueue) { - shouldRevive |= ts.checkSpeculatableTasks() - } - } - } - if (shouldRevive) { - backend.reviveOffers() - } - } - - /** Remove an executor from all our data structures and mark it as lost */ - private def removeExecutor(executorId: String) { - activeExecutorIds -= executorId - val host = executorIdToHost(executorId) - val execs = executorsByHost.getOrElse(host, new HashSet) - execs -= executorId - if (execs.isEmpty) { - executorsByHost -= host - } - executorIdToHost -= executorId - for (pool <- pools) { - pool.activeTaskSetsQueue.foreach(_.executorLost(executorId, host)) - } - } - -} - -/** - * An internal representation of a pool. It contains an ArrayBuffer of TaskSets and also weight and minshare - */ -class Pool(val name: String, val weight: Int) -{ - var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] - var numRunningTasks: Int = 0 -} diff --git a/core/src/main/scala/spark/scheduler/cluster/fair/FairTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/fair/FairTaskSetManager.scala deleted file mode 100644 index 4b0277d2d5..0000000000 --- a/core/src/main/scala/spark/scheduler/cluster/fair/FairTaskSetManager.scala +++ /dev/null @@ -1,130 +0,0 @@ -package spark.scheduler.cluster.fair - -import scala.collection.mutable.ArrayBuffer - -import spark._ -import spark.scheduler._ -import spark.scheduler.cluster._ -import spark.TaskState.TaskState -import java.nio.ByteBuffer - -/** - * Schedules the tasks within a single TaskSet in the FairClusterScheduler. - */ -private[spark] class FairTaskSetManager(sched: FairClusterScheduler, override val taskSet: TaskSet) extends TaskSetManager(sched, taskSet) with Logging { - - // Add a task to all the pending-task lists that it should be on. - private def addPendingTask(index: Int) { - val locations = tasks(index).preferredLocations.toSet & sched.hostsAlive - if (locations.size == 0) { - pendingTasksWithNoPrefs += index - } else { - for (host <- locations) { - val list = pendingTasksForHost.getOrElseUpdate(host, ArrayBuffer()) - list += index - } - } - allPendingTasks += index - } - - override def taskFinished(tid: Long, state: TaskState, serializedData: ByteBuffer) { - val info = taskInfos(tid) - if (info.failed) { - // We might get two task-lost messages for the same task in coarse-grained Mesos mode, - // or even from Mesos itself when acks get delayed. - return - } - val index = info.index - info.markSuccessful() - sched.taskFinished(this) - if (!finished(index)) { - tasksFinished += 1 - logInfo("Finished TID %s in %d ms (progress: %d/%d)".format( - tid, info.duration, tasksFinished, numTasks)) - // Deserialize task result and pass it to the scheduler - val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) - result.metrics.resultSize = serializedData.limit() - sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) - // Mark finished and stop if we've finished all the tasks - finished(index) = true - if (tasksFinished == numTasks) { - sched.taskSetFinished(this) - } - } else { - logInfo("Ignoring task-finished event for TID " + tid + - " because task " + index + " is already finished") - } - } - - override def taskLost(tid: Long, state: TaskState, serializedData: ByteBuffer) { - val info = taskInfos(tid) - if (info.failed) { - // We might get two task-lost messages for the same task in coarse-grained Mesos mode, - // or even from Mesos itself when acks get delayed. - return - } - val index = info.index - info.markFailed() - //Bookkeeping necessary for the pools in the scheduler - sched.taskFinished(this) - if (!finished(index)) { - logInfo("Lost TID %s (task %s:%d)".format(tid, taskSet.id, index)) - copiesRunning(index) -= 1 - // Check if the problem is a map output fetch failure. In that case, this - // task will never succeed on any node, so tell the scheduler about it. - if (serializedData != null && serializedData.limit() > 0) { - val reason = ser.deserialize[TaskEndReason](serializedData, getClass.getClassLoader) - reason match { - case fetchFailed: FetchFailed => - logInfo("Loss was due to fetch failure from " + fetchFailed.bmAddress) - sched.listener.taskEnded(tasks(index), fetchFailed, null, null, info, null) - finished(index) = true - tasksFinished += 1 - sched.taskSetFinished(this) - return - - case ef: ExceptionFailure => - val key = ef.exception.toString - val now = System.currentTimeMillis - val (printFull, dupCount) = { - if (recentExceptions.contains(key)) { - val (dupCount, printTime) = recentExceptions(key) - if (now - printTime > EXCEPTION_PRINT_INTERVAL) { - recentExceptions(key) = (0, now) - (true, 0) - } else { - recentExceptions(key) = (dupCount + 1, printTime) - (false, dupCount + 1) - } - } else { - recentExceptions(key) = (0, now) - (true, 0) - } - } - if (printFull) { - val locs = ef.exception.getStackTrace.map(loc => "\tat %s".format(loc.toString)) - logInfo("Loss was due to %s\n%s".format(ef.exception.toString, locs.mkString("\n"))) - } else { - logInfo("Loss was due to %s [duplicate %d]".format(ef.exception.toString, dupCount)) - } - - case _ => {} - } - } - // On non-fetch failures, re-enqueue the task as pending for a max number of retries - addPendingTask(index) - // Count failed attempts only on FAILED and LOST state (not on KILLED) - if (state == TaskState.FAILED || state == TaskState.LOST) { - numFailures(index) += 1 - if (numFailures(index) > MAX_TASK_FAILURES) { - logError("Task %s:%d failed more than %d times; aborting job".format( - taskSet.id, index, MAX_TASK_FAILURES)) - abort("Task %s:%d failed more than %d times".format(taskSet.id, index, MAX_TASK_FAILURES)) - } - } - } else { - logInfo("Ignoring task-lost event for TID " + tid + - " because task " + index + " is already finished") - } - } -} \ No newline at end of file -- cgit v1.2.3 From f5b1fecb9fc2e7d389810bfff5298f42b313f208 Mon Sep 17 00:00:00 2001 From: Harold Lim Date: Fri, 8 Mar 2013 15:38:32 -0500 Subject: Cleaned up the code --- .../spark/scheduler/cluster/ClusterScheduler.scala | 29 +--------------------- 1 file changed, 1 insertion(+), 28 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 0b5bf7a86c..5e960eb59d 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -27,7 +27,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val STARVATION_TIMEOUT = System.getProperty("spark.starvation.timeout", "15000").toLong val activeTaskSets = new HashMap[String, TaskSetManager] - // var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] val taskIdToTaskSetId = new HashMap[Long, String] val taskIdToExecutorId = new HashMap[Long, String] @@ -171,33 +170,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) executorsByHost(host) += execId } } - - /*val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) - val availableCpus = offers.map(o => o.cores).toArray - var launchedTask = false - for (manager <- activeTaskSetsQueue.sortBy(m => (m.taskSet.priority, m.taskSet.stageId))) { - do { - launchedTask = false - for (i <- 0 until offers.size) { - val execId = offers(i).executorId - val host = offers(i).hostname - manager.slaveOffer(execId, host, availableCpus(i)) match { - case Some(task) => - tasks(i) += task - val tid = task.taskId - taskIdToTaskSetId(tid) = manager.taskSet.id - taskSetTaskIds(manager.taskSet.id) += tid - taskIdToExecutorId(tid) = execId - activeExecutorIds += execId - executorsByHost(host) += execId - availableCpus(i) -= 1 - launchedTask = true - - case None => {} - } - } - } while (launchedTask) - }*/ + if (tasks.size > 0) { hasLaunchedTask = true } -- cgit v1.2.3 From 0b64e5f1ac0492aac6fca383c7877fbfce7d4cf1 Mon Sep 17 00:00:00 2001 From: Harold Lim Date: Fri, 8 Mar 2013 15:44:36 -0500 Subject: Removed some commented code --- core/src/main/scala/spark/SparkContext.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index f6ee399898..6eccb501c7 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -144,7 +144,7 @@ class SparkContext( new LocalScheduler(threads.toInt, maxFailures.toInt, this) case SPARK_REGEX(sparkUrl) => - val scheduler = new ClusterScheduler(this)//Class.forName(System.getProperty("spark.cluster.taskscheduler")).getConstructors()(0).newInstance(Array[AnyRef](this):_*).asInstanceOf[ClusterScheduler] + val scheduler = new ClusterScheduler(this) val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName) val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")).newInstance().asInstanceOf[TaskSetQueuesManager] scheduler.initialize(backend, taskSetQueuesManager) @@ -161,7 +161,7 @@ class SparkContext( memoryPerSlaveInt, sparkMemEnvInt)) } - val scheduler = new ClusterScheduler(this)//Class.forName(System.getProperty("spark.cluster.taskscheduler")).getConstructors()(0).newInstance(Array[AnyRef](this):_*).asInstanceOf[ClusterScheduler] + val scheduler = new ClusterScheduler(this) val localCluster = new LocalSparkCluster( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) val sparkUrl = localCluster.start() @@ -178,7 +178,7 @@ class SparkContext( logWarning("Master %s does not match expected format, parsing as Mesos URL".format(master)) } MesosNativeLibrary.load() - val scheduler = new ClusterScheduler(this)//Class.forName(System.getProperty("spark.cluster.taskscheduler")).getConstructors()(0).newInstance(Array[AnyRef](this):_*).asInstanceOf[ClusterScheduler] + val scheduler = new ClusterScheduler(this) val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean val masterWithoutProtocol = master.replaceFirst("^mesos://", "") // Strip initial mesos:// val backend = if (coarseGrained) { -- cgit v1.2.3 From 42822cf95de71039988e22d8690ba6a4bd639227 Mon Sep 17 00:00:00 2001 From: seanm Date: Wed, 13 Mar 2013 11:40:42 -0600 Subject: changing streaming resolver for akka --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 44c8058e9d..7e65979a5d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -163,7 +163,7 @@ object SparkBuild extends Build { def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", resolvers ++= Seq( - "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/" + "Akka Repository" at "http://repo.akka.io/releases/" ), libraryDependencies ++= Seq( "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile", -- cgit v1.2.3 From cfa8e769a86664722f47182fa572179e8beadcb7 Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 11 Mar 2013 17:16:15 -0600 Subject: KafkaInputDStream improvements. Allows more Kafka configurability --- .../scala/spark/streaming/StreamingContext.scala | 22 +++++++++- .../streaming/dstream/KafkaInputDStream.scala | 48 ++++++++++++++-------- 2 files changed, 51 insertions(+), 19 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 25c67b279b..4e1732adf5 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -199,7 +199,7 @@ class StreamingContext private ( } /** - * Create an input stream that pulls messages form a Kafka Broker. + * Create an input stream that pulls messages from a Kafka Broker. * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed @@ -216,7 +216,25 @@ class StreamingContext private ( initialOffsets: Map[KafkaPartitionKey, Long] = Map[KafkaPartitionKey, Long](), storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2 ): DStream[T] = { - val inputStream = new KafkaInputDStream[T](this, zkQuorum, groupId, topics, initialOffsets, storageLevel) + val kafkaParams = Map[String, String]("zk.connect" -> zkQuorum, "groupid" -> groupId, "zk.connectiontimeout.ms" -> "10000"); + kafkaStream[T](kafkaParams, topics, initialOffsets, storageLevel) + } + + /** + * Create an input stream that pulls messages from a Kafka Broker. + * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param initialOffsets Optional initial offsets for each of the partitions to consume. + * @param storageLevel Storage level to use for storing the received objects + */ + def kafkaStream[T: ClassManifest]( + kafkaParams: Map[String, String], + topics: Map[String, Int], + initialOffsets: Map[KafkaPartitionKey, Long], + storageLevel: StorageLevel + ): DStream[T] = { + val inputStream = new KafkaInputDStream[T](this, kafkaParams, topics, initialOffsets, storageLevel) registerInputStream(inputStream) inputStream } diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index dc7139cc27..f769fc1cc3 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -12,6 +12,8 @@ import kafka.message.{Message, MessageSet, MessageAndMetadata} import kafka.serializer.StringDecoder import kafka.utils.{Utils, ZKGroupTopicDirs} import kafka.utils.ZkUtils._ +import kafka.utils.ZKStringSerializer +import org.I0Itec.zkclient._ import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ @@ -23,8 +25,7 @@ case class KafkaPartitionKey(brokerId: Int, topic: String, groupId: String, part /** * Input stream that pulls messages from a Kafka Broker. * - * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). - * @param groupId The group id for this consumer. + * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. * @param initialOffsets Optional initial offsets for each of the partitions to consume. @@ -34,8 +35,7 @@ case class KafkaPartitionKey(brokerId: Int, topic: String, groupId: String, part private[streaming] class KafkaInputDStream[T: ClassManifest]( @transient ssc_ : StreamingContext, - zkQuorum: String, - groupId: String, + kafkaParams: Map[String, String], topics: Map[String, Int], initialOffsets: Map[KafkaPartitionKey, Long], storageLevel: StorageLevel @@ -43,19 +43,16 @@ class KafkaInputDStream[T: ClassManifest]( def getReceiver(): NetworkReceiver[T] = { - new KafkaReceiver(zkQuorum, groupId, topics, initialOffsets, storageLevel) + new KafkaReceiver(kafkaParams, topics, initialOffsets, storageLevel) .asInstanceOf[NetworkReceiver[T]] } } private[streaming] -class KafkaReceiver(zkQuorum: String, groupId: String, +class KafkaReceiver(kafkaParams: Map[String, String], topics: Map[String, Int], initialOffsets: Map[KafkaPartitionKey, Long], storageLevel: StorageLevel) extends NetworkReceiver[Any] { - // Timeout for establishing a connection to Zookeper in ms. - val ZK_TIMEOUT = 10000 - // Handles pushing data into the BlockManager lazy protected val blockGenerator = new BlockGenerator(storageLevel) // Connection to Kafka @@ -72,20 +69,24 @@ class KafkaReceiver(zkQuorum: String, groupId: String, // In case we are using multiple Threads to handle Kafka Messages val executorPool = Executors.newFixedThreadPool(topics.values.reduce(_ + _)) - logInfo("Starting Kafka Consumer Stream with group: " + groupId) + logInfo("Starting Kafka Consumer Stream with group: " + kafkaParams("groupid")) logInfo("Initial offsets: " + initialOffsets.toString) - // Zookeper connection properties + // Kafka connection properties val props = new Properties() - props.put("zk.connect", zkQuorum) - props.put("zk.connectiontimeout.ms", ZK_TIMEOUT.toString) - props.put("groupid", groupId) + kafkaParams.foreach(param => props.put(param._1, param._2)) // Create the connection to the cluster - logInfo("Connecting to Zookeper: " + zkQuorum) + logInfo("Connecting to Zookeper: " + kafkaParams("zk.connect")) val consumerConfig = new ConsumerConfig(props) consumerConnector = Consumer.create(consumerConfig).asInstanceOf[ZookeeperConsumerConnector] - logInfo("Connected to " + zkQuorum) + logInfo("Connected to " + kafkaParams("zk.connect")) + + // When autooffset.reset is 'smallest', it is our responsibility to try and whack the + // consumer group zk node. + if (kafkaParams.get("autooffset.reset").exists(_ == "smallest")) { + tryZookeeperConsumerGroupCleanup(kafkaParams("zk.connect"), kafkaParams("groupid")) + } // If specified, set the topic offset setOffsets(initialOffsets) @@ -97,7 +98,6 @@ class KafkaReceiver(zkQuorum: String, groupId: String, topicMessageStreams.values.foreach { streams => streams.foreach { stream => executorPool.submit(new MessageHandler(stream)) } } - } // Overwrites the offets in Zookeper. @@ -122,4 +122,18 @@ class KafkaReceiver(zkQuorum: String, groupId: String, } } } + + // Handles cleanup of consumer group znode. Lifted with love from Kafka's + // ConsumerConsole.scala tryCleanupZookeeper() + private def tryZookeeperConsumerGroupCleanup(zkUrl: String, groupId: String) { + try { + val dir = "/consumers/" + groupId + logInfo("Cleaning up temporary zookeeper data under " + dir + ".") + val zk = new ZkClient(zkUrl, 30*1000, 30*1000, ZKStringSerializer) + zk.deleteRecursive(dir) + zk.close() + } catch { + case _ => // swallow + } + } } -- cgit v1.2.3 From d06928321194b11e082986cd2bb2737d9bc3b698 Mon Sep 17 00:00:00 2001 From: seanm Date: Thu, 14 Mar 2013 23:25:35 -0600 Subject: fixing memory leak in kafka MessageHandler --- .../src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index f769fc1cc3..d674b6ee87 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -114,11 +114,8 @@ class KafkaReceiver(kafkaParams: Map[String, String], private class MessageHandler(stream: KafkaStream[String]) extends Runnable { def run() { logInfo("Starting MessageHandler.") - stream.takeWhile { msgAndMetadata => + for (msgAndMetadata <- stream) { blockGenerator += msgAndMetadata.message - // Keep on handling messages - - true } } } -- cgit v1.2.3 From 33fa1e7e4aca4d9e0edf65d2b768b569305fd044 Mon Sep 17 00:00:00 2001 From: seanm Date: Thu, 14 Mar 2013 23:32:52 -0600 Subject: removing dependency on ZookeeperConsumerConnector + purging last relic of kafka reliability that never solidified (ie- setOffsets) --- .../scala/spark/streaming/StreamingContext.scala | 9 ++----- .../streaming/api/java/JavaStreamingContext.scala | 28 ---------------------- .../streaming/dstream/KafkaInputDStream.scala | 28 ++++------------------ 3 files changed, 6 insertions(+), 59 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 4e1732adf5..bb7f216ca7 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -204,8 +204,6 @@ class StreamingContext private ( * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. - * @param initialOffsets Optional initial offsets for each of the partitions to consume. - * By default the value is pulled from zookeper. * @param storageLevel Storage level to use for storing the received objects * (default: StorageLevel.MEMORY_AND_DISK_SER_2) */ @@ -213,11 +211,10 @@ class StreamingContext private ( zkQuorum: String, groupId: String, topics: Map[String, Int], - initialOffsets: Map[KafkaPartitionKey, Long] = Map[KafkaPartitionKey, Long](), storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2 ): DStream[T] = { val kafkaParams = Map[String, String]("zk.connect" -> zkQuorum, "groupid" -> groupId, "zk.connectiontimeout.ms" -> "10000"); - kafkaStream[T](kafkaParams, topics, initialOffsets, storageLevel) + kafkaStream[T](kafkaParams, topics, storageLevel) } /** @@ -225,16 +222,14 @@ class StreamingContext private ( * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. - * @param initialOffsets Optional initial offsets for each of the partitions to consume. * @param storageLevel Storage level to use for storing the received objects */ def kafkaStream[T: ClassManifest]( kafkaParams: Map[String, String], topics: Map[String, Int], - initialOffsets: Map[KafkaPartitionKey, Long], storageLevel: StorageLevel ): DStream[T] = { - val inputStream = new KafkaInputDStream[T](this, kafkaParams, topics, initialOffsets, storageLevel) + val inputStream = new KafkaInputDStream[T](this, kafkaParams, topics, storageLevel) registerInputStream(inputStream) inputStream } diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index f3b40b5b88..2373f4824a 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -84,39 +84,12 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. - * @param initialOffsets Optional initial offsets for each of the partitions to consume. - * By default the value is pulled from zookeper. - */ - def kafkaStream[T]( - zkQuorum: String, - groupId: String, - topics: JMap[String, JInt], - initialOffsets: JMap[KafkaPartitionKey, JLong]) - : JavaDStream[T] = { - implicit val cmt: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] - ssc.kafkaStream[T]( - zkQuorum, - groupId, - Map(topics.mapValues(_.intValue()).toSeq: _*), - Map(initialOffsets.mapValues(_.longValue()).toSeq: _*)) - } - - /** - * Create an input stream that pulls messages form a Kafka Broker. - * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). - * @param groupId The group id for this consumer. - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param initialOffsets Optional initial offsets for each of the partitions to consume. - * By default the value is pulled from zookeper. * @param storageLevel RDD storage level. Defaults to memory-only */ def kafkaStream[T]( zkQuorum: String, groupId: String, topics: JMap[String, JInt], - initialOffsets: JMap[KafkaPartitionKey, JLong], storageLevel: StorageLevel) : JavaDStream[T] = { implicit val cmt: ClassManifest[T] = @@ -125,7 +98,6 @@ class JavaStreamingContext(val ssc: StreamingContext) { zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), - Map(initialOffsets.mapValues(_.longValue()).toSeq: _*), storageLevel) } diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index d674b6ee87..c6da1a7f70 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -19,17 +19,12 @@ import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ -// Key for a specific Kafka Partition: (broker, topic, group, part) -case class KafkaPartitionKey(brokerId: Int, topic: String, groupId: String, partId: Int) - /** * Input stream that pulls messages from a Kafka Broker. * * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. - * @param initialOffsets Optional initial offsets for each of the partitions to consume. - * By default the value is pulled from zookeper. * @param storageLevel RDD storage level. */ private[streaming] @@ -37,26 +32,25 @@ class KafkaInputDStream[T: ClassManifest]( @transient ssc_ : StreamingContext, kafkaParams: Map[String, String], topics: Map[String, Int], - initialOffsets: Map[KafkaPartitionKey, Long], storageLevel: StorageLevel ) extends NetworkInputDStream[T](ssc_ ) with Logging { def getReceiver(): NetworkReceiver[T] = { - new KafkaReceiver(kafkaParams, topics, initialOffsets, storageLevel) + new KafkaReceiver(kafkaParams, topics, storageLevel) .asInstanceOf[NetworkReceiver[T]] } } private[streaming] class KafkaReceiver(kafkaParams: Map[String, String], - topics: Map[String, Int], initialOffsets: Map[KafkaPartitionKey, Long], + topics: Map[String, Int], storageLevel: StorageLevel) extends NetworkReceiver[Any] { // Handles pushing data into the BlockManager lazy protected val blockGenerator = new BlockGenerator(storageLevel) // Connection to Kafka - var consumerConnector : ZookeeperConsumerConnector = null + var consumerConnector : ConsumerConnector = null def onStop() { blockGenerator.stop() @@ -70,7 +64,6 @@ class KafkaReceiver(kafkaParams: Map[String, String], val executorPool = Executors.newFixedThreadPool(topics.values.reduce(_ + _)) logInfo("Starting Kafka Consumer Stream with group: " + kafkaParams("groupid")) - logInfo("Initial offsets: " + initialOffsets.toString) // Kafka connection properties val props = new Properties() @@ -79,7 +72,7 @@ class KafkaReceiver(kafkaParams: Map[String, String], // Create the connection to the cluster logInfo("Connecting to Zookeper: " + kafkaParams("zk.connect")) val consumerConfig = new ConsumerConfig(props) - consumerConnector = Consumer.create(consumerConfig).asInstanceOf[ZookeeperConsumerConnector] + consumerConnector = Consumer.create(consumerConfig) logInfo("Connected to " + kafkaParams("zk.connect")) // When autooffset.reset is 'smallest', it is our responsibility to try and whack the @@ -88,9 +81,6 @@ class KafkaReceiver(kafkaParams: Map[String, String], tryZookeeperConsumerGroupCleanup(kafkaParams("zk.connect"), kafkaParams("groupid")) } - // If specified, set the topic offset - setOffsets(initialOffsets) - // Create Threads for each Topic/Message Stream we are listening val topicMessageStreams = consumerConnector.createMessageStreams(topics, new StringDecoder()) @@ -100,16 +90,6 @@ class KafkaReceiver(kafkaParams: Map[String, String], } } - // Overwrites the offets in Zookeper. - private def setOffsets(offsets: Map[KafkaPartitionKey, Long]) { - offsets.foreach { case(key, offset) => - val topicDirs = new ZKGroupTopicDirs(key.groupId, key.topic) - val partitionName = key.brokerId + "-" + key.partId - updatePersistentPath(consumerConnector.zkClient, - topicDirs.consumerOffsetDir + "/" + partitionName, offset.toString) - } - } - // Handles Kafka Messages private class MessageHandler(stream: KafkaStream[String]) extends Runnable { def run() { -- cgit v1.2.3 From 5892393140eb024a32585b6d5b51146ddde8f63a Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Sat, 16 Mar 2013 11:13:38 +0800 Subject: refactor fair scheduler implementation 1.Chage "pool" properties to be the memeber of ActiveJob 2.Abstract the Schedulable of Pool and TaskSetManager 3.Abstract the FIFO and FS comparator algorithm 4.Miscellaneous changing of class define and construction --- .../src/main/scala/spark/scheduler/ActiveJob.scala | 5 +- .../main/scala/spark/scheduler/DAGScheduler.scala | 32 +-- core/src/main/scala/spark/scheduler/Stage.scala | 4 +- .../spark/scheduler/cluster/ClusterScheduler.scala | 48 ++-- .../cluster/FIFOTaskSetQueuesManager.scala | 37 ++- .../cluster/FairTaskSetQueuesManager.scala | 248 ++++++++++----------- .../main/scala/spark/scheduler/cluster/Pool.scala | 92 ++++++++ .../spark/scheduler/cluster/Schedulable.scala | 21 ++ .../scheduler/cluster/SchedulingAlgorithm.scala | 69 ++++++ .../spark/scheduler/cluster/SchedulingMode.scala | 8 + .../spark/scheduler/cluster/TaskDescription.scala | 1 + .../spark/scheduler/cluster/TaskSetManager.scala | 38 +++- .../scheduler/cluster/TaskSetQueuesManager.scala | 6 +- 13 files changed, 415 insertions(+), 194 deletions(-) create mode 100644 core/src/main/scala/spark/scheduler/cluster/Pool.scala create mode 100644 core/src/main/scala/spark/scheduler/cluster/Schedulable.scala create mode 100644 core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala create mode 100644 core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala diff --git a/core/src/main/scala/spark/scheduler/ActiveJob.scala b/core/src/main/scala/spark/scheduler/ActiveJob.scala index 5a4e9a582d..b6d3c2c089 100644 --- a/core/src/main/scala/spark/scheduler/ActiveJob.scala +++ b/core/src/main/scala/spark/scheduler/ActiveJob.scala @@ -2,6 +2,8 @@ package spark.scheduler import spark.TaskContext +import java.util.Properties + /** * Tracks information about an active job in the DAGScheduler. */ @@ -11,7 +13,8 @@ private[spark] class ActiveJob( val func: (TaskContext, Iterator[_]) => _, val partitions: Array[Int], val callSite: String, - val listener: JobListener) { + val listener: JobListener, + val properties: Properties) { val numPartitions = partitions.length val finished = Array.fill[Boolean](numPartitions)(false) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 2ad73f3232..717cc27739 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -89,6 +89,8 @@ class DAGScheduler( // stray messages to detect. val failedGeneration = new HashMap[String, Long] + val idToActiveJob = new HashMap[Int, ActiveJob] + val waiting = new HashSet[Stage] // Stages we need to run whose parents aren't done val running = new HashSet[Stage] // Stages we are running right now val failed = new HashSet[Stage] // Stages that must be resubmitted due to fetch failures @@ -129,11 +131,11 @@ class DAGScheduler( * The priority value passed in will be used if the stage doesn't already exist with * a lower priority (we assume that priorities always increase across jobs for now). */ - private def getShuffleMapStage(shuffleDep: ShuffleDependency[_,_], priority: Int, properties: Properties): Stage = { + private def getShuffleMapStage(shuffleDep: ShuffleDependency[_,_], priority: Int): Stage = { shuffleToMapStage.get(shuffleDep.shuffleId) match { case Some(stage) => stage case None => - val stage = newStage(shuffleDep.rdd, Some(shuffleDep), priority, properties) + val stage = newStage(shuffleDep.rdd, Some(shuffleDep), priority) shuffleToMapStage(shuffleDep.shuffleId) = stage stage } @@ -144,7 +146,7 @@ class DAGScheduler( * as a result stage for the final RDD used directly in an action. The stage will also be given * the provided priority. */ - private def newStage(rdd: RDD[_], shuffleDep: Option[ShuffleDependency[_,_]], priority: Int, properties: Properties): Stage = { + private def newStage(rdd: RDD[_], shuffleDep: Option[ShuffleDependency[_,_]], priority: Int): Stage = { if (shuffleDep != None) { // Kind of ugly: need to register RDDs with the cache and map output tracker here // since we can't do it in the RDD constructor because # of partitions is unknown @@ -152,7 +154,7 @@ class DAGScheduler( mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.partitions.size) } val id = nextStageId.getAndIncrement() - val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority, properties), priority, properties) + val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority), priority) idToStage(id) = stage stageToInfos(stage) = StageInfo(stage) stage @@ -162,7 +164,7 @@ class DAGScheduler( * Get or create the list of parent stages for a given RDD. The stages will be assigned the * provided priority if they haven't already been created with a lower priority. */ - private def getParentStages(rdd: RDD[_], priority: Int, properties: Properties): List[Stage] = { + private def getParentStages(rdd: RDD[_], priority: Int): List[Stage] = { val parents = new HashSet[Stage] val visited = new HashSet[RDD[_]] def visit(r: RDD[_]) { @@ -173,7 +175,7 @@ class DAGScheduler( for (dep <- r.dependencies) { dep match { case shufDep: ShuffleDependency[_,_] => - parents += getShuffleMapStage(shufDep, priority, properties) + parents += getShuffleMapStage(shufDep, priority) case _ => visit(dep.rdd) } @@ -194,7 +196,7 @@ class DAGScheduler( for (dep <- rdd.dependencies) { dep match { case shufDep: ShuffleDependency[_,_] => - val mapStage = getShuffleMapStage(shufDep, stage.priority, stage.properties) + val mapStage = getShuffleMapStage(shufDep, stage.priority) if (!mapStage.isAvailable) { missing += mapStage } @@ -239,7 +241,8 @@ class DAGScheduler( partitions: Seq[Int], callSite: String, allowLocal: Boolean, - resultHandler: (Int, U) => Unit, properties: Properties = null) + resultHandler: (Int, U) => Unit, + properties: Properties = null) { if (partitions.size == 0) { return @@ -260,7 +263,8 @@ class DAGScheduler( func: (TaskContext, Iterator[T]) => U, evaluator: ApproximateEvaluator[U, R], callSite: String, - timeout: Long, properties: Properties = null) + timeout: Long, + properties: Properties = null) : PartialResult[R] = { val listener = new ApproximateActionListener(rdd, func, evaluator, timeout) @@ -278,8 +282,8 @@ class DAGScheduler( event match { case JobSubmitted(finalRDD, func, partitions, allowLocal, callSite, listener, properties) => val runId = nextRunId.getAndIncrement() - val finalStage = newStage(finalRDD, None, runId, properties) - val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener) + val finalStage = newStage(finalRDD, None, runId) + val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener, properties) clearCacheLocs() logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length + " output partitions (allowLocal=" + allowLocal + ")") @@ -290,6 +294,7 @@ class DAGScheduler( // Compute very short actions like first() or take() with no parent stages locally. runLocally(job) } else { + idToActiveJob(runId) = job activeJobs += job resultStageToJob(finalStage) = job submitStage(finalStage) @@ -459,8 +464,9 @@ class DAGScheduler( logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks logDebug("New pending tasks: " + myPending) + val properties = idToActiveJob(stage.priority).properties taskSched.submitTasks( - new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.priority, stage.properties)) + new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.priority, properties)) if (!stage.submissionTime.isDefined) { stage.submissionTime = Some(System.currentTimeMillis()) } @@ -665,7 +671,7 @@ class DAGScheduler( for (dep <- rdd.dependencies) { dep match { case shufDep: ShuffleDependency[_,_] => - val mapStage = getShuffleMapStage(shufDep, stage.priority, stage.properties) + val mapStage = getShuffleMapStage(shufDep, stage.priority) if (!mapStage.isAvailable) { visitedStages += mapStage visit(mapStage.rdd) diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala index 97afa27a60..bc54cd601d 100644 --- a/core/src/main/scala/spark/scheduler/Stage.scala +++ b/core/src/main/scala/spark/scheduler/Stage.scala @@ -1,7 +1,6 @@ package spark.scheduler import java.net.URI -import java.util.Properties import spark._ import spark.storage.BlockManagerId @@ -26,8 +25,7 @@ private[spark] class Stage( val rdd: RDD[_], val shuffleDep: Option[ShuffleDependency[_,_]], // Output shuffle if stage is a map stage val parents: List[Stage], - val priority: Int, - val properties: Properties = null) + val priority: Int) extends Logging { val isShuffleMap = shuffleDep != None diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 5e960eb59d..092b0a0cfc 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -143,7 +143,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) * that tasks are balanced across the cluster. */ def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = { - synchronized { + synchronized { + SparkEnv.set(sc.env) // Mark each slave as alive and remember its hostname for (o <- offers) { @@ -152,25 +153,33 @@ private[spark] class ClusterScheduler(val sc: SparkContext) executorsByHost(o.hostname) = new HashSet() } } - // Build a list of tasks to assign to each slave val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) - val taskSetIds = taskSetQueuesManager.receiveOffer(tasks, offers) - //We populate the necessary bookkeeping structures - for (i <- 0 until offers.size) { - val execId = offers(i).executorId - val host = offers(i).hostname - for(j <- 0 until tasks(i).size) { - val tid = tasks(i)(j).taskId - val taskSetid = taskSetIds(i)(j) - taskIdToTaskSetId(tid) = taskSetid - taskSetTaskIds(taskSetid) += tid - taskIdToExecutorId(tid) = execId - activeExecutorIds += execId - executorsByHost(host) += execId - } + val availableCpus = offers.map(o => o.cores).toArray + for (i <- 0 until offers.size) + { + var launchedTask = true + val execId = offers(i).executorId + val host = offers(i).hostname + while (availableCpus(i) > 0 && launchedTask) + { + launchedTask = false + taskSetQueuesManager.receiveOffer(execId,host,availableCpus(i)) match { + case Some(task) => + tasks(i) += task + val tid = task.taskId + taskIdToTaskSetId(tid) = task.taskSetId + taskSetTaskIds(task.taskSetId) += tid + taskIdToExecutorId(tid) = execId + activeExecutorIds += execId + executorsByHost(host) += execId + availableCpus(i) -= 1 + launchedTask = true + + case None => {} + } + } } - if (tasks.size > 0) { hasLaunchedTask = true } @@ -219,10 +228,11 @@ private[spark] class ClusterScheduler(val sc: SparkContext) taskSetToUpdate.get.statusUpdate(tid, state, serializedData) } if (failedExecutor != None) { - listener.executorLost(failedExecutor.get) + listener.executorLost(failedExecutor.get) backend.reviveOffers() } if (taskFailed) { + // Also revive offers if a task had failed for some reason other than host lost backend.reviveOffers() } @@ -289,7 +299,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } // Call listener.executorLost without holding the lock on this to prevent deadlock if (failedExecutor != None) { - listener.executorLost(failedExecutor.get) + listener.executorLost(failedExecutor.get) backend.reviveOffers() } } diff --git a/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala index 99a9c94222..868b11c8d6 100644 --- a/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala @@ -10,6 +10,7 @@ import spark.Logging private[spark] class FIFOTaskSetQueuesManager extends TaskSetQueuesManager with Logging { var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] + val tasksetSchedulingAlgorithm = new FIFOSchedulingAlgorithm() override def addTaskSetManager(manager: TaskSetManager) { activeTaskSetsQueue += manager @@ -27,31 +28,19 @@ private[spark] class FIFOTaskSetQueuesManager extends TaskSetQueuesManager with activeTaskSetsQueue.foreach(_.executorLost(executorId, host)) } - override def receiveOffer(tasks: Seq[ArrayBuffer[TaskDescription]], offers: Seq[WorkerOffer]): Seq[Seq[String]] = { - val taskSetIds = offers.map(o => new ArrayBuffer[String](o.cores)) - val availableCpus = offers.map(o => o.cores).toArray - var launchedTask = false - for (manager <- activeTaskSetsQueue.sortBy(m => (m.taskSet.priority, m.taskSet.stageId))) { - do { - launchedTask = false - for (i <- 0 until offers.size) { - val execId = offers(i).executorId - val host = offers(i).hostname - manager.slaveOffer(execId, host, availableCpus(i)) match { - case Some(task) => - tasks(i) += task - taskSetIds(i) += manager.taskSet.id - availableCpus(i) -= 1 - launchedTask = true - - case None => {} - } - } - } while (launchedTask) + override def receiveOffer(execId:String, host:String,avaiableCpus:Double):Option[TaskDescription] = + { + for(manager <- activeTaskSetsQueue.sortWith(tasksetSchedulingAlgorithm.comparator)) + { + val task = manager.slaveOffer(execId,host,avaiableCpus) + if (task != None) + { + return task + } } - return taskSetIds + return None } - + override def checkSpeculatableTasks(): Boolean = { var shouldRevive = false for (ts <- activeTaskSetsQueue) { @@ -60,4 +49,4 @@ private[spark] class FIFOTaskSetQueuesManager extends TaskSetQueuesManager with return shouldRevive } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala index ca308a5229..4e26cedfda 100644 --- a/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala @@ -10,174 +10,166 @@ import scala.util.control.Breaks._ import scala.xml._ import spark.Logging +import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * A Fair Implementation of the TaskSetQueuesManager * - * The current implementation makes the following assumptions: A pool has a fixed configuration of weight. - * Within a pool, it just uses FIFO. - * Also, currently we assume that pools are statically defined - * We currently don't support min shares + * Currently we support minShare,weight for fair scheduler between pools + * Within a pool, it supports FIFO or FS + * Also, currently we could allocate pools dynamically + * */ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with Logging { val schedulerAllocFile = System.getProperty("spark.fairscheduler.allocation.file","unspecified") val poolNameToPool= new HashMap[String, Pool] var pools = new ArrayBuffer[Pool] + val poolScheduleAlgorithm = new FairSchedulingAlgorithm() + val POOL_FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.cluster.fair.pool" + val POOL_DEFAULT_POOL_NAME = "default" + val POOL_MINIMUM_SHARES_PROPERTY = "minShares" + val POOL_SCHEDULING_MODE_PROPERTY = "schedulingMode" + val POOL_WEIGHT_PROPERTY = "weight" + val POOL_POOL_NAME_PROPERTY = "@name" + val POOL_POOLS_PROPERTY = "pool" + val POOL_DEFAULT_SCHEDULING_MODE = SchedulingMode.FIFO + val POOL_DEFAULT_MINIMUM_SHARES = 2 + val POOL_DEFAULT_WEIGHT = 1 loadPoolProperties() def loadPoolProperties() { //first check if the file exists val file = new File(schedulerAllocFile) - if(!file.exists()) { - //if file does not exist, we just create 1 pool, default - val pool = new Pool("default",100) - pools += pool - poolNameToPool("default") = pool - logInfo("Created a default pool with weight = 100") - } - else { + if(file.exists()) + { val xml = XML.loadFile(file) - for (poolNode <- (xml \\ "pool")) { - if((poolNode \ "weight").text != ""){ - val pool = new Pool((poolNode \ "@name").text,(poolNode \ "weight").text.toInt) - pools += pool - poolNameToPool((poolNode \ "@name").text) = pool - logInfo("Created pool "+ pool.name +"with weight = "+pool.weight) - } else { - val pool = new Pool((poolNode \ "@name").text,100) - pools += pool - poolNameToPool((poolNode \ "@name").text) = pool - logInfo("Created pool "+ pool.name +"with weight = 100") + for (poolNode <- (xml \\ POOL_POOLS_PROPERTY)) { + + val poolName = (poolNode \ POOL_POOL_NAME_PROPERTY).text + var schedulingMode = POOL_DEFAULT_SCHEDULING_MODE + var minShares = POOL_DEFAULT_MINIMUM_SHARES + var weight = POOL_DEFAULT_WEIGHT + + + val xmlSchedulingMode = (poolNode \ POOL_SCHEDULING_MODE_PROPERTY).text + if( xmlSchedulingMode != "") + { + try + { + schedulingMode = SchedulingMode.withName(xmlSchedulingMode) + } + catch{ + case e:Exception => logInfo("Error xml schedulingMode, using default schedulingMode") + } } - } - if(!poolNameToPool.contains("default")) { - val pool = new Pool("default", 100) + + val xmlMinShares = (poolNode \ POOL_MINIMUM_SHARES_PROPERTY).text + if(xmlMinShares != "") + { + minShares = xmlMinShares.toInt + } + + val xmlWeight = (poolNode \ POOL_WEIGHT_PROPERTY).text + if(xmlWeight != "") + { + weight = xmlWeight.toInt + } + + val pool = new Pool(poolName,schedulingMode,minShares,weight) pools += pool - poolNameToPool("default") = pool - logInfo("Created a default pool with weight = 100") + poolNameToPool(poolName) = pool + logInfo("Create new pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format(poolName,schedulingMode,minShares,weight)) } - - } + } + + if(!poolNameToPool.contains(POOL_DEFAULT_POOL_NAME)) + { + val pool = new Pool(POOL_DEFAULT_POOL_NAME, POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT) + pools += pool + poolNameToPool(POOL_DEFAULT_POOL_NAME) = pool + logInfo("Create default pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format(POOL_DEFAULT_POOL_NAME,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) + } } override def addTaskSetManager(manager: TaskSetManager) { - var poolName = "default" - if(manager.taskSet.properties != null) - poolName = manager.taskSet.properties.getProperty("spark.scheduler.cluster.fair.pool","default") - if(poolNameToPool.contains(poolName)) - poolNameToPool(poolName).activeTaskSetsQueue += manager - else - poolNameToPool("default").activeTaskSetsQueue += manager - logInfo("Added task set " + manager.taskSet.id + " tasks to pool "+poolName) - + var poolName = POOL_DEFAULT_POOL_NAME + if(manager.taskSet.properties != null) + { + poolName = manager.taskSet.properties.getProperty(POOL_FAIR_SCHEDULER_PROPERTIES,POOL_DEFAULT_POOL_NAME) + if(!poolNameToPool.contains(poolName)) + { + //we will create a new pool that user has configured in app,but not contained in xml file + val pool = new Pool(poolName,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT) + pools += pool + poolNameToPool(poolName) = pool + logInfo("Create pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format(poolName,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) + } + } + poolNameToPool(poolName).addTaskSetManager(manager) + logInfo("Added task set " + manager.taskSet.id + " tasks to pool "+poolName) } override def removeTaskSetManager(manager: TaskSetManager) { - var poolName = "default" - if(manager.taskSet.properties != null) - poolName = manager.taskSet.properties.getProperty("spark.scheduler.cluster.fair.pool","default") - if(poolNameToPool.contains(poolName)) - poolNameToPool(poolName).activeTaskSetsQueue -= manager - else - poolNameToPool("default").activeTaskSetsQueue -= manager + + var poolName = POOL_DEFAULT_POOL_NAME + if(manager.taskSet.properties != null) + { + poolName = manager.taskSet.properties.getProperty(POOL_FAIR_SCHEDULER_PROPERTIES,POOL_DEFAULT_POOL_NAME) + } + logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id,poolName)) + val pool = poolNameToPool(poolName) + pool.removeTaskSetManager(manager) + pool.setRunningTasks(pool.getRunningTasks() - manager.getRunningTasks()) + } override def taskFinished(manager: TaskSetManager) { - var poolName = "default" - if(manager.taskSet.properties != null) - poolName = manager.taskSet.properties.getProperty("spark.scheduler.cluster.fair.pool","default") - if(poolNameToPool.contains(poolName)) - poolNameToPool(poolName).numRunningTasks -= 1 - else - poolNameToPool("default").numRunningTasks -= 1 + var poolName = POOL_DEFAULT_POOL_NAME + if(manager.taskSet.properties != null) + { + poolName = manager.taskSet.properties.getProperty(POOL_FAIR_SCHEDULER_PROPERTIES,POOL_DEFAULT_POOL_NAME) + } + val pool = poolNameToPool(poolName) + pool.setRunningTasks(pool.getRunningTasks() - 1) + manager.setRunningTasks(manager.getRunningTasks() - 1) } override def removeExecutor(executorId: String, host: String) { for (pool <- pools) { - pool.activeTaskSetsQueue.foreach(_.executorLost(executorId, host)) + pool.removeExecutor(executorId,host) } } - /** - * This is the comparison function used for sorting to determine which - * pool to allocate next based on fairness. - * The algorithm is as follows: we sort by the pool's running tasks to weight ratio - * (pools number running tast / pool's weight) - */ - def poolFairCompFn(pool1: Pool, pool2: Pool): Boolean = { - val tasksToWeightRatio1 = pool1.numRunningTasks.toDouble / pool1.weight.toDouble - val tasksToWeightRatio2 = pool2.numRunningTasks.toDouble / pool2.weight.toDouble - var res = Math.signum(tasksToWeightRatio1 - tasksToWeightRatio2) - if (res == 0) { - //Jobs are tied in fairness ratio. We break the tie by name - res = pool1.name.compareTo(pool2.name) - } - if (res < 0) - return true - else - return false + override def receiveOffer(execId: String,host:String,avaiableCpus:Double):Option[TaskDescription] = + { + + val sortedPools = pools.sortWith(poolScheduleAlgorithm.comparator) + for(pool <- sortedPools) + { + logDebug("poolName:%s,tasksetNum:%d,minShares:%d,runningTasks:%d".format(pool.poolName,pool.activeTaskSetsQueue.length,pool.getMinShare(),pool.getRunningTasks())) + } + for (pool <- sortedPools) + { + val task = pool.receiveOffer(execId,host,avaiableCpus) + if(task != None) + { + pool.setRunningTasks(pool.getRunningTasks() + 1) + return task + } + } + return None } - override def receiveOffer(tasks: Seq[ArrayBuffer[TaskDescription]], offers: Seq[WorkerOffer]): Seq[Seq[String]] = { - val taskSetIds = offers.map(o => new ArrayBuffer[String](o.cores)) - val availableCpus = offers.map(o => o.cores).toArray - var launchedTask = false - - for (i <- 0 until offers.size) { //we loop through the list of offers - val execId = offers(i).executorId - val host = offers(i).hostname - var breakOut = false - while(availableCpus(i) > 0 && !breakOut) { - breakable{ - launchedTask = false - for (pool <- pools.sortWith(poolFairCompFn)) { //we loop through the list of pools - if(!pool.activeTaskSetsQueue.isEmpty) { - //sort the tasksetmanager in the pool - pool.activeTaskSetsQueue.sortBy(m => (m.taskSet.priority, m.taskSet.stageId)) - for(manager <- pool.activeTaskSetsQueue) { //we loop through the activeTaskSets in this pool - //Make an offer - manager.slaveOffer(execId, host, availableCpus(i)) match { - case Some(task) => - tasks(i) += task - taskSetIds(i) += manager.taskSet.id - availableCpus(i) -= 1 - pool.numRunningTasks += 1 - launchedTask = true - logInfo("launched task for pool"+pool.name); - break - case None => {} - } - } - } - } - //If there is not one pool that can assign the task then we have to exit the outer loop and continue to the next offer - if(!launchedTask){ - breakOut = true - } - } - } - } - return taskSetIds - } - - override def checkSpeculatableTasks(): Boolean = { + override def checkSpeculatableTasks(): Boolean = + { var shouldRevive = false - for (pool <- pools) { - for (ts <- pool.activeTaskSetsQueue) { - shouldRevive |= ts.checkSpeculatableTasks() - } + for (pool <- pools) + { + shouldRevive |= pool.checkSpeculatableTasks() } return shouldRevive } -} -/** - * An internal representation of a pool. It contains an ArrayBuffer of TaskSets and also weight - */ -class Pool(val name: String, val weight: Int) -{ - var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] - var numRunningTasks: Int = 0 -} \ No newline at end of file + } diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/spark/scheduler/cluster/Pool.scala new file mode 100644 index 0000000000..7b58a99582 --- /dev/null +++ b/core/src/main/scala/spark/scheduler/cluster/Pool.scala @@ -0,0 +1,92 @@ +package spark.scheduler.cluster + +import scala.collection.mutable.ArrayBuffer + +import spark.Logging +import spark.scheduler.cluster.SchedulingMode.SchedulingMode +/** + * An interface for + * + */ +private[spark] class Pool(val poolName: String, schedulingMode: SchedulingMode,val minShare:Int, val weight:Int) extends Schedulable with Logging { + + var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] + var numRunningTasks: Int = 0 + var taskSetSchedulingAlgorithm: SchedulingAlgorithm = + { + schedulingMode match + { + case SchedulingMode.FAIR => + val schedule = new FairSchedulingAlgorithm() + schedule + case SchedulingMode.FIFO => + val schedule = new FIFOSchedulingAlgorithm() + schedule + } + } + + override def getMinShare():Int = + { + return minShare + } + + override def getRunningTasks():Int = + { + return numRunningTasks + } + + def setRunningTasks(taskNum : Int) + { + numRunningTasks = taskNum + } + + override def getWeight(): Int = + { + return weight + } + + def addTaskSetManager(manager:TaskSetManager) + { + activeTaskSetsQueue += manager + } + + def removeTaskSetManager(manager:TaskSetManager) + { + activeTaskSetsQueue -= manager + } + + def removeExecutor(executorId: String, host: String) + { + activeTaskSetsQueue.foreach(_.executorLost(executorId,host)) + } + + def checkSpeculatableTasks(): Boolean = + { + var shouldRevive = false + for(ts <- activeTaskSetsQueue) + { + shouldRevive |= ts.checkSpeculatableTasks() + } + return shouldRevive + } + + def receiveOffer(execId:String,host:String,availableCpus:Double):Option[TaskDescription] = + { + val sortedActiveTasksSetQueue = activeTaskSetsQueue.sortWith(taskSetSchedulingAlgorithm.comparator) + for(manager <- sortedActiveTasksSetQueue) + { + + logDebug("taskSetId:%s,taskNum:%d,minShares:%d,weight:%d,runningTasks:%d".format(manager.taskSet.id,manager.numTasks,manager.getMinShare(),manager.getWeight(),manager.getRunningTasks())) + } + for(manager <- sortedActiveTasksSetQueue) + { + val task = manager.slaveOffer(execId,host,availableCpus) + if (task != None) + { + manager.setRunningTasks(manager.getRunningTasks() + 1) + return task + } + } + return None + } +} diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala new file mode 100644 index 0000000000..837f9c4983 --- /dev/null +++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala @@ -0,0 +1,21 @@ +package spark.scheduler.cluster + +import scala.collection.mutable.ArrayBuffer + +/** + * An interface for schedulable entities, there are two type Schedulable entities(Pools and TaskSetManagers) + */ +private[spark] trait Schedulable { + + def getMinShare(): Int + def getRunningTasks(): Int + def getPriority(): Int = + { + return 0 + } + def getWeight(): Int + def getStageId(): Int = + { + return 0 + } +} diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala new file mode 100644 index 0000000000..f8919e7374 --- /dev/null +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala @@ -0,0 +1,69 @@ +package spark.scheduler.cluster + +/** + * An interface for sort algorithm + * FIFO: FIFO algorithm for TaskSetManagers + * FS: FS algorithm for Pools, and FIFO or FS for TaskSetManagers + */ +private[spark] trait SchedulingAlgorithm { + def comparator(s1: Schedulable,s2: Schedulable): Boolean +} + +private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm +{ + override def comparator(s1: Schedulable, s2: Schedulable): Boolean = + { + val priority1 = s1.getPriority() + val priority2 = s2.getPriority() + var res = Math.signum(priority1 - priority2) + if (res == 0) + { + val stageId1 = s1.getStageId() + val stageId2 = s2.getStageId() + res = Math.signum(stageId1 - stageId2) + } + if (res < 0) + { + return true + } + else + { + return false + } + } +} + +private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm +{ + def comparator(s1: Schedulable, s2:Schedulable): Boolean = + { + val minShare1 = s1.getMinShare() + val minShare2 = s2.getMinShare() + val s1Needy = s1.getRunningTasks() < minShare1 + val s2Needy = s2.getRunningTasks() < minShare2 + val minShareRatio1 = s1.getRunningTasks().toDouble / Math.max(minShare1,1.0).toDouble + val minShareRatio2 = s2.getRunningTasks().toDouble / Math.max(minShare2,1.0).toDouble + val taskToWeightRatio1 = s1.getRunningTasks().toDouble / s1.getWeight().toDouble + val taskToWeightRatio2 = s2.getRunningTasks().toDouble / s2.getWeight().toDouble + var res:Boolean = true + + if(s1Needy && !s2Needy) + { + res = true + } + else if(!s1Needy && s2Needy) + { + res = false + } + else if (s1Needy && s2Needy) + { + res = minShareRatio1 <= minShareRatio2 + } + else + { + res = taskToWeightRatio1 <= taskToWeightRatio2 + } + return res + } +} + diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala new file mode 100644 index 0000000000..6be4f3cd84 --- /dev/null +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala @@ -0,0 +1,8 @@ +package spark.scheduler.cluster + +object SchedulingMode extends Enumeration("FAIR","FIFO") +{ + type SchedulingMode = Value + + val FAIR,FIFO = Value +} diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala index b41e951be9..cdd004c94b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala @@ -5,6 +5,7 @@ import spark.util.SerializableBuffer private[spark] class TaskDescription( val taskId: Long, + val taskSetId: String, val executorId: String, val name: String, _serializedTask: ByteBuffer) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 015092b60b..723c3b46bd 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -17,7 +17,7 @@ import java.nio.ByteBuffer /** * Schedules the tasks within a single TaskSet in the ClusterScheduler. */ -private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSet) extends Logging { +private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSet) extends Schedulable with Logging { // Maximum time to wait to run a task in a preferred location (in ms) val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong @@ -28,6 +28,9 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // Maximum times a task is allowed to fail before failing the job val MAX_TASK_FAILURES = 4 + val TASKSET_MINIMIUM_SHARES = 1 + + val TASKSET_WEIGHT = 1 // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble val SPECULATION_MULTIPLIER = System.getProperty("spark.speculation.multiplier", "1.5").toDouble @@ -43,6 +46,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe val numFailures = new Array[Int](numTasks) val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) var tasksFinished = 0 + var numRunningTasks =0; // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis @@ -96,6 +100,36 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe addPendingTask(i) } + override def getMinShare(): Int = + { + return TASKSET_MINIMIUM_SHARES + } + + override def getRunningTasks(): Int = + { + return numRunningTasks + } + + def setRunningTasks(taskNum :Int) + { + numRunningTasks = taskNum + } + + override def getPriority(): Int = + { + return priority + } + + override def getWeight(): Int = + { + return TASKSET_WEIGHT + } + + override def getStageId(): Int = + { + return taskSet.stageId + } + // Add a task to all the pending-task lists that it should be on. private def addPendingTask(index: Int) { val locations = tasks(index).preferredLocations.toSet & sched.hostsAlive @@ -222,7 +256,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe logInfo("Serialized task %s:%d as %d bytes in %d ms".format( taskSet.id, index, serializedTask.limit, timeTaken)) val taskName = "task %s:%d".format(taskSet.id, index) - return Some(new TaskDescription(taskId, execId, taskName, serializedTask)) + return Some(new TaskDescription(taskId,taskSet.id,execId, taskName, serializedTask)) } case _ => } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala index b0c30e9e8b..c117ee7a85 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala @@ -12,8 +12,6 @@ private[spark] trait TaskSetQueuesManager { def removeTaskSetManager(manager: TaskSetManager): Unit def taskFinished(manager: TaskSetManager): Unit def removeExecutor(executorId: String, host: String): Unit - //The receiveOffers function, accepts tasks and offers. It populates the tasks to the actual task from TaskSet - //It returns a list of TaskSet ID that corresponds to each assigned tasks - def receiveOffer(tasks: Seq[ArrayBuffer[TaskDescription]], offers: Seq[WorkerOffer]): Seq[Seq[String]] + def receiveOffer(execId: String, host:String, avaiableCpus:Double):Option[TaskDescription] def checkSpeculatableTasks(): Boolean -} \ No newline at end of file +} -- cgit v1.2.3 From d61978d0abad30a148680c8a63df33e40e469525 Mon Sep 17 00:00:00 2001 From: seanm Date: Fri, 15 Mar 2013 23:36:52 -0600 Subject: keeping JavaStreamingContext in sync with StreamingContext + adding comments for better clarity --- .../main/scala/spark/streaming/api/java/JavaStreamingContext.scala | 7 +++---- .../src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala | 6 ++++-- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index 2373f4824a..7a8864614c 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -80,6 +80,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Create an input stream that pulls messages form a Kafka Broker. + * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed @@ -87,16 +88,14 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param storageLevel RDD storage level. Defaults to memory-only */ def kafkaStream[T]( - zkQuorum: String, - groupId: String, + kafkaParams: JMap[String, String], topics: JMap[String, JInt], storageLevel: StorageLevel) : JavaDStream[T] = { implicit val cmt: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] ssc.kafkaStream[T]( - zkQuorum, - groupId, + kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) } diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index c6da1a7f70..85693808d1 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -100,8 +100,10 @@ class KafkaReceiver(kafkaParams: Map[String, String], } } - // Handles cleanup of consumer group znode. Lifted with love from Kafka's - // ConsumerConsole.scala tryCleanupZookeeper() + // Delete consumer group from zookeeper. This effectivly resets the group so we can consume from the beginning again. + // The kafka high level consumer doesn't expose setting offsets currently, this is a trick copied from Kafkas' + // ConsoleConsumer. See code related to 'autooffset.reset' when it is set to 'smallest': + // https://github.com/apache/kafka/blob/0.7.2/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala private def tryZookeeperConsumerGroupCleanup(zkUrl: String, groupId: String) { try { val dir = "/consumers/" + groupId -- cgit v1.2.3 From d1d9bdaabe24cc60097f843e0bef92e57b404941 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Sat, 23 Mar 2013 07:25:30 +0800 Subject: Just update typo and comments --- core/src/main/scala/spark/scheduler/cluster/Pool.scala | 3 +-- core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala | 4 ++-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/spark/scheduler/cluster/Pool.scala index 7b58a99582..68e1d2a75a 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Pool.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Pool.scala @@ -5,8 +5,7 @@ import scala.collection.mutable.ArrayBuffer import spark.Logging import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** - * An interface for - * + * An Schedulable entity that represent collection of TaskSetManager */ private[spark] class Pool(val poolName: String, schedulingMode: SchedulingMode,val minShare:Int, val weight:Int) extends Schedulable with Logging { diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 723c3b46bd..064593f486 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -28,7 +28,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // Maximum times a task is allowed to fail before failing the job val MAX_TASK_FAILURES = 4 - val TASKSET_MINIMIUM_SHARES = 1 + val TASKSET_MINIMUM_SHARES = 1 val TASKSET_WEIGHT = 1 // Quantile of tasks at which to start speculation @@ -102,7 +102,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe override def getMinShare(): Int = { - return TASKSET_MINIMIUM_SHARES + return TASKSET_MINIMUM_SHARES } override def getRunningTasks(): Int = -- cgit v1.2.3 From 329ef34c2e04d28c2ad150cf6674d6e86d7511ce Mon Sep 17 00:00:00 2001 From: seanm Date: Tue, 26 Mar 2013 23:56:15 -0600 Subject: fixing autooffset.reset behavior when set to 'largest' --- .../main/scala/spark/streaming/dstream/KafkaInputDStream.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index 85693808d1..17a5be3420 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -75,9 +75,9 @@ class KafkaReceiver(kafkaParams: Map[String, String], consumerConnector = Consumer.create(consumerConfig) logInfo("Connected to " + kafkaParams("zk.connect")) - // When autooffset.reset is 'smallest', it is our responsibility to try and whack the + // When autooffset.reset is defined, it is our responsibility to try and whack the // consumer group zk node. - if (kafkaParams.get("autooffset.reset").exists(_ == "smallest")) { + if (kafkaParams.contains("autooffset.reset")) { tryZookeeperConsumerGroupCleanup(kafkaParams("zk.connect"), kafkaParams("groupid")) } @@ -100,9 +100,11 @@ class KafkaReceiver(kafkaParams: Map[String, String], } } - // Delete consumer group from zookeeper. This effectivly resets the group so we can consume from the beginning again. + // It is our responsibility to delete the consumer group when specifying autooffset.reset. This is because + // Kafka 0.7.2 only honors this param when the group is not in zookeeper. + // // The kafka high level consumer doesn't expose setting offsets currently, this is a trick copied from Kafkas' - // ConsoleConsumer. See code related to 'autooffset.reset' when it is set to 'smallest': + // ConsoleConsumer. See code related to 'autooffset.reset' when it is set to 'smallest'/'largest': // https://github.com/apache/kafka/blob/0.7.2/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala private def tryZookeeperConsumerGroupCleanup(zkUrl: String, groupId: String) { try { -- cgit v1.2.3 From def3d1c84a3e0d1371239e9358294a4b4ad46b9f Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Fri, 29 Mar 2013 08:20:35 +0800 Subject: 1.remove redundant spacing in source code 2.replace get/set functions with val and var defination --- .../src/main/scala/spark/scheduler/ActiveJob.scala | 2 +- .../main/scala/spark/scheduler/DAGScheduler.scala | 4 +- .../spark/scheduler/cluster/ClusterScheduler.scala | 13 ++--- .../cluster/FIFOTaskSetQueuesManager.scala | 13 ++--- .../cluster/FairTaskSetQueuesManager.scala | 65 +++++++++++----------- .../main/scala/spark/scheduler/cluster/Pool.scala | 43 +++++--------- .../spark/scheduler/cluster/Schedulable.scala | 21 ++----- .../scheduler/cluster/SchedulingAlgorithm.scala | 34 +++++------ .../spark/scheduler/cluster/SchedulingMode.scala | 2 +- .../spark/scheduler/cluster/TaskSetManager.scala | 37 ++---------- .../scheduler/cluster/TaskSetQueuesManager.scala | 1 - 11 files changed, 91 insertions(+), 144 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/ActiveJob.scala b/core/src/main/scala/spark/scheduler/ActiveJob.scala index b6d3c2c089..105eaecb22 100644 --- a/core/src/main/scala/spark/scheduler/ActiveJob.scala +++ b/core/src/main/scala/spark/scheduler/ActiveJob.scala @@ -13,7 +13,7 @@ private[spark] class ActiveJob( val func: (TaskContext, Iterator[_]) => _, val partitions: Array[Int], val callSite: String, - val listener: JobListener, + val listener: JobListener, val properties: Properties) { val numPartitions = partitions.length diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 717cc27739..0a64a4f041 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -241,7 +241,7 @@ class DAGScheduler( partitions: Seq[Int], callSite: String, allowLocal: Boolean, - resultHandler: (Int, U) => Unit, + resultHandler: (Int, U) => Unit, properties: Properties = null) { if (partitions.size == 0) { @@ -263,7 +263,7 @@ class DAGScheduler( func: (TaskContext, Iterator[T]) => U, evaluator: ApproximateEvaluator[U, R], callSite: String, - timeout: Long, + timeout: Long, properties: Properties = null) : PartialResult[R] = { diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 092b0a0cfc..be0d480aa0 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -60,7 +60,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) var backend: SchedulerBackend = null val mapOutputTracker = SparkEnv.get.mapOutputTracker - + var taskSetQueuesManager: TaskSetQueuesManager = null override def setListener(listener: TaskSchedulerListener) { @@ -131,11 +131,11 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } } - def taskFinished(manager: TaskSetManager) { + def taskFinished(manager: TaskSetManager) { this.synchronized { - taskSetQueuesManager.taskFinished(manager) + taskSetQueuesManager.taskFinished(manager) } - } + } /** * Called by cluster manager to offer resources on slaves. We respond by asking our active task @@ -144,7 +144,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext) */ def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = { synchronized { - SparkEnv.set(sc.env) // Mark each slave as alive and remember its hostname for (o <- offers) { @@ -228,7 +227,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) taskSetToUpdate.get.statusUpdate(tid, state, serializedData) } if (failedExecutor != None) { - listener.executorLost(failedExecutor.get) + listener.executorLost(failedExecutor.get) backend.reviveOffers() } if (taskFailed) { @@ -299,7 +298,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } // Call listener.executorLost without holding the lock on this to prevent deadlock if (failedExecutor != None) { - listener.executorLost(failedExecutor.get) + listener.executorLost(failedExecutor.get) backend.reviveOffers() } } diff --git a/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala index 868b11c8d6..5949ee773f 100644 --- a/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala @@ -8,26 +8,26 @@ import spark.Logging * A FIFO Implementation of the TaskSetQueuesManager */ private[spark] class FIFOTaskSetQueuesManager extends TaskSetQueuesManager with Logging { - + var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] val tasksetSchedulingAlgorithm = new FIFOSchedulingAlgorithm() - + override def addTaskSetManager(manager: TaskSetManager) { activeTaskSetsQueue += manager } - + override def removeTaskSetManager(manager: TaskSetManager) { activeTaskSetsQueue -= manager } - + override def taskFinished(manager: TaskSetManager) { //do nothing } - + override def removeExecutor(executorId: String, host: String) { activeTaskSetsQueue.foreach(_.executorLost(executorId, host)) } - + override def receiveOffer(execId:String, host:String,avaiableCpus:Double):Option[TaskDescription] = { for(manager <- activeTaskSetsQueue.sortWith(tasksetSchedulingAlgorithm.comparator)) @@ -48,5 +48,4 @@ private[spark] class FIFOTaskSetQueuesManager extends TaskSetQueuesManager with } return shouldRevive } - } diff --git a/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala index 4e26cedfda..0609600f35 100644 --- a/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala @@ -14,15 +14,14 @@ import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * A Fair Implementation of the TaskSetQueuesManager - * + * * Currently we support minShare,weight for fair scheduler between pools * Within a pool, it supports FIFO or FS * Also, currently we could allocate pools dynamically - * */ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with Logging { - - val schedulerAllocFile = System.getProperty("spark.fairscheduler.allocation.file","unspecified") + + val schedulerAllocFile = System.getProperty("spark.fairscheduler.allocation.file","unspecified") val poolNameToPool= new HashMap[String, Pool] var pools = new ArrayBuffer[Pool] val poolScheduleAlgorithm = new FairSchedulingAlgorithm() @@ -36,9 +35,9 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with val POOL_DEFAULT_SCHEDULING_MODE = SchedulingMode.FIFO val POOL_DEFAULT_MINIMUM_SHARES = 2 val POOL_DEFAULT_WEIGHT = 1 - + loadPoolProperties() - + def loadPoolProperties() { //first check if the file exists val file = new File(schedulerAllocFile) @@ -51,26 +50,25 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with var schedulingMode = POOL_DEFAULT_SCHEDULING_MODE var minShares = POOL_DEFAULT_MINIMUM_SHARES var weight = POOL_DEFAULT_WEIGHT - - + val xmlSchedulingMode = (poolNode \ POOL_SCHEDULING_MODE_PROPERTY).text if( xmlSchedulingMode != "") { - try + try { schedulingMode = SchedulingMode.withName(xmlSchedulingMode) } catch{ - case e:Exception => logInfo("Error xml schedulingMode, using default schedulingMode") + case e:Exception => logInfo("Error xml schedulingMode, using default schedulingMode") } } - + val xmlMinShares = (poolNode \ POOL_MINIMUM_SHARES_PROPERTY).text if(xmlMinShares != "") { minShares = xmlMinShares.toInt } - + val xmlWeight = (poolNode \ POOL_WEIGHT_PROPERTY).text if(xmlWeight != "") { @@ -84,15 +82,15 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with } } - if(!poolNameToPool.contains(POOL_DEFAULT_POOL_NAME)) + if(!poolNameToPool.contains(POOL_DEFAULT_POOL_NAME)) { val pool = new Pool(POOL_DEFAULT_POOL_NAME, POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT) pools += pool poolNameToPool(POOL_DEFAULT_POOL_NAME) = pool logInfo("Create default pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format(POOL_DEFAULT_POOL_NAME,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) - } + } } - + override def addTaskSetManager(manager: TaskSetManager) { var poolName = POOL_DEFAULT_POOL_NAME if(manager.taskSet.properties != null) @@ -100,19 +98,19 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with poolName = manager.taskSet.properties.getProperty(POOL_FAIR_SCHEDULER_PROPERTIES,POOL_DEFAULT_POOL_NAME) if(!poolNameToPool.contains(poolName)) { - //we will create a new pool that user has configured in app,but not contained in xml file + //we will create a new pool that user has configured in app instead of being defined in xml file val pool = new Pool(poolName,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT) pools += pool poolNameToPool(poolName) = pool - logInfo("Create pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format(poolName,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) + logInfo("Create pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format(poolName,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) } } poolNameToPool(poolName).addTaskSetManager(manager) - logInfo("Added task set " + manager.taskSet.id + " tasks to pool "+poolName) + logInfo("Added task set " + manager.taskSet.id + " tasks to pool "+poolName) } - + override def removeTaskSetManager(manager: TaskSetManager) { - + var poolName = POOL_DEFAULT_POOL_NAME if(manager.taskSet.properties != null) { @@ -121,10 +119,9 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id,poolName)) val pool = poolNameToPool(poolName) pool.removeTaskSetManager(manager) - pool.setRunningTasks(pool.getRunningTasks() - manager.getRunningTasks()) - + pool.runningTasks -= manager.runningTasks } - + override def taskFinished(manager: TaskSetManager) { var poolName = POOL_DEFAULT_POOL_NAME if(manager.taskSet.properties != null) @@ -132,40 +129,40 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with poolName = manager.taskSet.properties.getProperty(POOL_FAIR_SCHEDULER_PROPERTIES,POOL_DEFAULT_POOL_NAME) } val pool = poolNameToPool(poolName) - pool.setRunningTasks(pool.getRunningTasks() - 1) - manager.setRunningTasks(manager.getRunningTasks() - 1) + pool.runningTasks -= 1 + manager.runningTasks -=1 } - + override def removeExecutor(executorId: String, host: String) { for (pool <- pools) { - pool.removeExecutor(executorId,host) - } + pool.removeExecutor(executorId,host) + } } - + override def receiveOffer(execId: String,host:String,avaiableCpus:Double):Option[TaskDescription] = { val sortedPools = pools.sortWith(poolScheduleAlgorithm.comparator) for(pool <- sortedPools) { - logDebug("poolName:%s,tasksetNum:%d,minShares:%d,runningTasks:%d".format(pool.poolName,pool.activeTaskSetsQueue.length,pool.getMinShare(),pool.getRunningTasks())) + logDebug("poolName:%s,tasksetNum:%d,minShares:%d,runningTasks:%d".format(pool.poolName,pool.activeTaskSetsQueue.length,pool.minShare,pool.runningTasks)) } for (pool <- sortedPools) { val task = pool.receiveOffer(execId,host,avaiableCpus) if(task != None) { - pool.setRunningTasks(pool.getRunningTasks() + 1) + pool.runningTasks += 1 return task } } return None } - - override def checkSpeculatableTasks(): Boolean = + + override def checkSpeculatableTasks(): Boolean = { var shouldRevive = false - for (pool <- pools) + for (pool <- pools) { shouldRevive |= pool.checkSpeculatableTasks() } diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/spark/scheduler/cluster/Pool.scala index 68e1d2a75a..8fdca5d2b4 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Pool.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Pool.scala @@ -7,13 +7,21 @@ import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * An Schedulable entity that represent collection of TaskSetManager */ -private[spark] class Pool(val poolName: String, schedulingMode: SchedulingMode,val minShare:Int, val weight:Int) extends Schedulable with Logging { - +private[spark] class Pool(val poolName: String,val schedulingMode: SchedulingMode, initMinShare:Int, initWeight:Int) extends Schedulable with Logging +{ + var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] - var numRunningTasks: Int = 0 - var taskSetSchedulingAlgorithm: SchedulingAlgorithm = + + var weight = initWeight + var minShare = initMinShare + var runningTasks = 0 + + val priority = 0 + val stageId = 0 + + var taskSetSchedulingAlgorithm: SchedulingAlgorithm = { - schedulingMode match + schedulingMode match { case SchedulingMode.FAIR => val schedule = new FairSchedulingAlgorithm() @@ -23,26 +31,6 @@ private[spark] class Pool(val poolName: String, schedulingMode: SchedulingMode,v schedule } } - - override def getMinShare():Int = - { - return minShare - } - - override def getRunningTasks():Int = - { - return numRunningTasks - } - - def setRunningTasks(taskNum : Int) - { - numRunningTasks = taskNum - } - - override def getWeight(): Int = - { - return weight - } def addTaskSetManager(manager:TaskSetManager) { @@ -74,15 +62,14 @@ private[spark] class Pool(val poolName: String, schedulingMode: SchedulingMode,v val sortedActiveTasksSetQueue = activeTaskSetsQueue.sortWith(taskSetSchedulingAlgorithm.comparator) for(manager <- sortedActiveTasksSetQueue) { - - logDebug("taskSetId:%s,taskNum:%d,minShares:%d,weight:%d,runningTasks:%d".format(manager.taskSet.id,manager.numTasks,manager.getMinShare(),manager.getWeight(),manager.getRunningTasks())) + logDebug("poolname:%s,taskSetId:%s,taskNum:%d,minShares:%d,weight:%d,runningTasks:%d".format(poolName,manager.taskSet.id,manager.numTasks,manager.minShare,manager.weight,manager.runningTasks)) } for(manager <- sortedActiveTasksSetQueue) { val task = manager.slaveOffer(execId,host,availableCpus) if (task != None) { - manager.setRunningTasks(manager.getRunningTasks() + 1) + manager.runningTasks += 1 return task } } diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala index 837f9c4983..6f4f104f42 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala @@ -1,21 +1,12 @@ package spark.scheduler.cluster -import scala.collection.mutable.ArrayBuffer - /** - * An interface for schedulable entities, there are two type Schedulable entities(Pools and TaskSetManagers) + * An interface for schedulable entities, there are two type Schedulable entities(Pools and TaskSetManagers) */ private[spark] trait Schedulable { - - def getMinShare(): Int - def getRunningTasks(): Int - def getPriority(): Int = - { - return 0 - } - def getWeight(): Int - def getStageId(): Int = - { - return 0 - } + def weight:Int + def minShare:Int + def runningTasks:Int + def priority:Int + def stageId:Int } diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala index f8919e7374..2f8123587f 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala @@ -1,7 +1,7 @@ package spark.scheduler.cluster /** - * An interface for sort algorithm + * An interface for sort algorithm * FIFO: FIFO algorithm for TaskSetManagers * FS: FS algorithm for Pools, and FIFO or FS for TaskSetManagers */ @@ -13,13 +13,13 @@ private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm { override def comparator(s1: Schedulable, s2: Schedulable): Boolean = { - val priority1 = s1.getPriority() - val priority2 = s2.getPriority() + val priority1 = s1.priority + val priority2 = s2.priority var res = Math.signum(priority1 - priority2) if (res == 0) { - val stageId1 = s1.getStageId() - val stageId2 = s2.getStageId() + val stageId1 = s1.stageId + val stageId2 = s2.stageId res = Math.signum(stageId1 - stageId2) } if (res < 0) @@ -29,7 +29,7 @@ private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm else { return false - } + } } } @@ -37,16 +37,18 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { def comparator(s1: Schedulable, s2:Schedulable): Boolean = { - val minShare1 = s1.getMinShare() - val minShare2 = s2.getMinShare() - val s1Needy = s1.getRunningTasks() < minShare1 - val s2Needy = s2.getRunningTasks() < minShare2 - val minShareRatio1 = s1.getRunningTasks().toDouble / Math.max(minShare1,1.0).toDouble - val minShareRatio2 = s2.getRunningTasks().toDouble / Math.max(minShare2,1.0).toDouble - val taskToWeightRatio1 = s1.getRunningTasks().toDouble / s1.getWeight().toDouble - val taskToWeightRatio2 = s2.getRunningTasks().toDouble / s2.getWeight().toDouble + val minShare1 = s1.minShare + val minShare2 = s2.minShare + val runningTasks1 = s1.runningTasks + val runningTasks2 = s2.runningTasks + val s1Needy = runningTasks1 < minShare1 + val s2Needy = runningTasks2 < minShare2 + val minShareRatio1 = runningTasks1.toDouble / Math.max(minShare1,1.0).toDouble + val minShareRatio2 = runningTasks2.toDouble / Math.max(minShare2,1.0).toDouble + val taskToWeightRatio1 = runningTasks1.toDouble / s1.weight.toDouble + val taskToWeightRatio2 = runningTasks2.toDouble / s2.weight.toDouble var res:Boolean = true - + if(s1Needy && !s2Needy) { res = true @@ -57,7 +59,7 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm } else if (s1Needy && s2Needy) { - res = minShareRatio1 <= minShareRatio2 + res = minShareRatio1 <= minShareRatio2 } else { diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala index 6be4f3cd84..480af2c1a3 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala @@ -1,4 +1,4 @@ -package spark.scheduler.cluster +package spark.scheduler.cluster object SchedulingMode extends Enumeration("FAIR","FIFO") { diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 064593f486..ddc4fa6642 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -29,7 +29,6 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe val MAX_TASK_FAILURES = 4 val TASKSET_MINIMUM_SHARES = 1 - val TASKSET_WEIGHT = 1 // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble @@ -38,7 +37,12 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // Serializer for closures and tasks. val ser = SparkEnv.get.closureSerializer.newInstance() + var weight = TASKSET_WEIGHT + var minShare = TASKSET_MINIMUM_SHARES + var runningTasks = 0 val priority = taskSet.priority + val stageId = taskSet.stageId + val tasks = taskSet.tasks val numTasks = tasks.length val copiesRunning = new Array[Int](numTasks) @@ -46,7 +50,6 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe val numFailures = new Array[Int](numTasks) val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) var tasksFinished = 0 - var numRunningTasks =0; // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis @@ -100,36 +103,6 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe addPendingTask(i) } - override def getMinShare(): Int = - { - return TASKSET_MINIMUM_SHARES - } - - override def getRunningTasks(): Int = - { - return numRunningTasks - } - - def setRunningTasks(taskNum :Int) - { - numRunningTasks = taskNum - } - - override def getPriority(): Int = - { - return priority - } - - override def getWeight(): Int = - { - return TASKSET_WEIGHT - } - - override def getStageId(): Int = - { - return taskSet.stageId - } - // Add a task to all the pending-task lists that it should be on. private def addPendingTask(index: Int) { val locations = tasks(index).preferredLocations.toSet & sched.hostsAlive diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala index c117ee7a85..86971d47e6 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala @@ -5,7 +5,6 @@ import scala.collection.mutable.ArrayBuffer /** * An interface for managing TaskSet queue/s that allows plugging different policy for * offering tasks to resources - * */ private[spark] trait TaskSetQueuesManager { def addTaskSetManager(manager: TaskSetManager): Unit -- cgit v1.2.3 From 1a28f92711cb59ad99bc9e3dd84a5990181e572b Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Fri, 29 Mar 2013 08:34:28 +0800 Subject: change some typo and some spacing --- core/src/main/scala/spark/SparkContext.scala | 12 ++++++------ core/src/main/scala/spark/scheduler/DAGScheduler.scala | 4 ++-- core/src/main/scala/spark/scheduler/Stage.scala | 5 ++--- 3 files changed, 10 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 6eccb501c7..ed5f686379 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -74,7 +74,7 @@ class SparkContext( if (System.getProperty("spark.driver.port") == null) { System.setProperty("spark.driver.port", "0") } - + //Set the default task scheduler if (System.getProperty("spark.cluster.taskscheduler") == null) { System.setProperty("spark.cluster.taskscheduler", "spark.scheduler.cluster.FIFOTaskSetQueuesManager") @@ -119,7 +119,7 @@ class SparkContext( } } executorEnvs ++= environment - + // Create and start the scheduler private var taskScheduler: TaskScheduler = { // Regular expression used for local[N] master format @@ -216,14 +216,14 @@ class SparkContext( } private[spark] var checkpointDir: Option[String] = None - + // Thread Local variable that can be used by users to pass information down the stack private val localProperties = new DynamicVariable[Properties](null) - + def initLocalProperties() { localProperties.value = new Properties() } - + def addLocalProperties(key: String, value: String) { if(localProperties.value == null) { localProperties.value = new Properties() @@ -673,7 +673,7 @@ class SparkContext( val processFunc = (context: TaskContext, iter: Iterator[T]) => processPartition(iter) runJob[T, U](rdd, processFunc, 0 until rdd.partitions.size, false, resultHandler) } - + /** * Run a job that can return approximate results. */ diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 0a64a4f041..abc24c0270 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -333,7 +333,7 @@ class DAGScheduler( submitStage(stage) } } - + /** * Check for waiting or failed stages which are now eligible for resubmission. * Ordinarily run on every iteration of the event loop. @@ -720,7 +720,7 @@ class DAGScheduler( sizeBefore = shuffleToMapStage.size shuffleToMapStage.clearOldValues(cleanupTime) logInfo("shuffleToMapStage " + sizeBefore + " --> " + shuffleToMapStage.size) - + sizeBefore = pendingTasks.size pendingTasks.clearOldValues(cleanupTime) logInfo("pendingTasks " + sizeBefore + " --> " + pendingTasks.size) diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala index bc54cd601d..7fc9e13fd9 100644 --- a/core/src/main/scala/spark/scheduler/Stage.scala +++ b/core/src/main/scala/spark/scheduler/Stage.scala @@ -5,7 +5,6 @@ import java.net.URI import spark._ import spark.storage.BlockManagerId - /** * A stage is a set of independent tasks all computing the same function that need to run as part * of a Spark job, where all the tasks have the same shuffle dependencies. Each DAG of tasks run @@ -27,7 +26,7 @@ private[spark] class Stage( val parents: List[Stage], val priority: Int) extends Logging { - + val isShuffleMap = shuffleDep != None val numPartitions = rdd.partitions.size val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil) @@ -61,7 +60,7 @@ private[spark] class Stage( numAvailableOutputs -= 1 } } - + def removeOutputsOnExecutor(execId: String) { var becameUnavailable = false for (partition <- 0 until numPartitions) { -- cgit v1.2.3 From 2b373dd07a7b3f2906607d910c869e3290ca9d05 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Tue, 2 Apr 2013 12:11:14 +0800 Subject: add properties default value null to fix sbt/sbt test errors --- core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index 79588891e7..6f4e5cd83e 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -23,7 +23,7 @@ private[spark] case class JobSubmitted( partitions: Array[Int], allowLocal: Boolean, callSite: String, - listener: JobListener, properties: Properties) + listener: JobListener, properties: Properties = null) extends DAGSchedulerEvent private[spark] case class CompletionEvent( -- cgit v1.2.3 From df47b40b764e25cbd10ce49d7152e1d33f51a263 Mon Sep 17 00:00:00 2001 From: shane-huang Date: Wed, 20 Feb 2013 11:51:13 +0800 Subject: Shuffle Performance fix: Use netty embeded OIO file server instead of ConnectionManager Shuffle Performance Optimization: do not send 0-byte block requests to reduce network messages change reference from io.Source to scala.io.Source to avoid looking into io.netty package Signed-off-by: shane-huang --- .../main/java/spark/network/netty/FileClient.java | 89 +++++++ .../netty/FileClientChannelInitializer.java | 29 +++ .../spark/network/netty/FileClientHandler.java | 38 +++ .../main/java/spark/network/netty/FileServer.java | 59 +++++ .../netty/FileServerChannelInitializer.java | 33 +++ .../spark/network/netty/FileServerHandler.java | 68 ++++++ .../java/spark/network/netty/PathResolver.java | 12 + .../scala/spark/network/netty/FileHeader.scala | 57 +++++ .../scala/spark/network/netty/ShuffleCopier.scala | 88 +++++++ .../scala/spark/network/netty/ShuffleSender.scala | 50 ++++ .../main/scala/spark/storage/BlockManager.scala | 272 +++++++++++++++++---- core/src/main/scala/spark/storage/DiskStore.scala | 51 +++- project/SparkBuild.scala | 3 +- .../scala/spark/streaming/util/RawTextSender.scala | 2 +- 14 files changed, 795 insertions(+), 56 deletions(-) create mode 100644 core/src/main/java/spark/network/netty/FileClient.java create mode 100644 core/src/main/java/spark/network/netty/FileClientChannelInitializer.java create mode 100644 core/src/main/java/spark/network/netty/FileClientHandler.java create mode 100644 core/src/main/java/spark/network/netty/FileServer.java create mode 100644 core/src/main/java/spark/network/netty/FileServerChannelInitializer.java create mode 100644 core/src/main/java/spark/network/netty/FileServerHandler.java create mode 100755 core/src/main/java/spark/network/netty/PathResolver.java create mode 100644 core/src/main/scala/spark/network/netty/FileHeader.scala create mode 100644 core/src/main/scala/spark/network/netty/ShuffleCopier.scala create mode 100644 core/src/main/scala/spark/network/netty/ShuffleSender.scala diff --git a/core/src/main/java/spark/network/netty/FileClient.java b/core/src/main/java/spark/network/netty/FileClient.java new file mode 100644 index 0000000000..d0c5081dd2 --- /dev/null +++ b/core/src/main/java/spark/network/netty/FileClient.java @@ -0,0 +1,89 @@ +package spark.network.netty; + +import io.netty.bootstrap.Bootstrap; +import io.netty.channel.AbstractChannel; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelOption; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.nio.NioSocketChannel; +import io.netty.channel.oio.OioEventLoopGroup; +import io.netty.channel.socket.oio.OioSocketChannel; + +import java.util.Arrays; + +public class FileClient { + + private FileClientHandler handler = null; + private Channel channel = null; + private Bootstrap bootstrap = null; + + public FileClient(FileClientHandler handler){ + this.handler = handler; + } + + public void init(){ + bootstrap = new Bootstrap(); + bootstrap.group(new OioEventLoopGroup()) + .channel(OioSocketChannel.class) + .option(ChannelOption.SO_KEEPALIVE, true) + .option(ChannelOption.TCP_NODELAY, true) + .handler(new FileClientChannelInitializer(handler)); + } + + public static final class ChannelCloseListener implements ChannelFutureListener { + private FileClient fc = null; + public ChannelCloseListener(FileClient fc){ + this.fc = fc; + } + @Override + public void operationComplete(ChannelFuture future) { + if (fc.bootstrap!=null){ + fc.bootstrap.shutdown(); + fc.bootstrap = null; + } + } + } + + public void connect(String host, int port){ + try { + + // Start the connection attempt. + channel = bootstrap.connect(host, port).sync().channel(); + // ChannelFuture cf = channel.closeFuture(); + //cf.addListener(new ChannelCloseListener(this)); + } catch (InterruptedException e) { + close(); + } + } + + public void waitForClose(){ + try { + channel.closeFuture().sync(); + } catch (InterruptedException e){ + e.printStackTrace(); + } + } + + public void sendRequest(String file){ + //assert(file == null); + //assert(channel == null); + channel.write(file+"\r\n"); + } + + public void close(){ + if(channel != null) { + channel.close(); + channel = null; + } + if ( bootstrap!=null) { + bootstrap.shutdown(); + bootstrap = null; + } + } + + +} + + diff --git a/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java b/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java new file mode 100644 index 0000000000..50e5704619 --- /dev/null +++ b/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java @@ -0,0 +1,29 @@ +package spark.network.netty; + +import io.netty.buffer.BufType; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.socket.SocketChannel; +import io.netty.handler.codec.LengthFieldBasedFrameDecoder; +import io.netty.handler.codec.string.StringEncoder; +import io.netty.util.CharsetUtil; + +import io.netty.handler.logging.LoggingHandler; +import io.netty.handler.logging.LogLevel; + +public class FileClientChannelInitializer extends + ChannelInitializer { + + private FileClientHandler fhandler; + + public FileClientChannelInitializer(FileClientHandler handler) { + fhandler = handler; + } + + @Override + public void initChannel(SocketChannel channel) { + // file no more than 2G + channel.pipeline() + .addLast("encoder", new StringEncoder(BufType.BYTE)) + .addLast("handler", fhandler); + } +} diff --git a/core/src/main/java/spark/network/netty/FileClientHandler.java b/core/src/main/java/spark/network/netty/FileClientHandler.java new file mode 100644 index 0000000000..911c8b32b5 --- /dev/null +++ b/core/src/main/java/spark/network/netty/FileClientHandler.java @@ -0,0 +1,38 @@ +package spark.network.netty; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundByteHandlerAdapter; +import io.netty.util.CharsetUtil; + +import java.util.concurrent.atomic.AtomicInteger; +import java.util.logging.Logger; + +public abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter { + + private FileHeader currentHeader = null; + + public abstract void handle(ChannelHandlerContext ctx, ByteBuf in, FileHeader header); + + @Override + public ByteBuf newInboundBuffer(ChannelHandlerContext ctx) { + // Use direct buffer if possible. + return ctx.alloc().ioBuffer(); + } + + @Override + public void inboundBufferUpdated(ChannelHandlerContext ctx, ByteBuf in) { + // get header + if (currentHeader == null && in.readableBytes() >= FileHeader.HEADER_SIZE()) { + currentHeader = FileHeader.create(in.readBytes(FileHeader.HEADER_SIZE())); + } + // get file + if(in.readableBytes() >= currentHeader.fileLen()){ + handle(ctx,in,currentHeader); + currentHeader = null; + ctx.close(); + } + } + +} + diff --git a/core/src/main/java/spark/network/netty/FileServer.java b/core/src/main/java/spark/network/netty/FileServer.java new file mode 100644 index 0000000000..729e45f0a1 --- /dev/null +++ b/core/src/main/java/spark/network/netty/FileServer.java @@ -0,0 +1,59 @@ +package spark.network.netty; + +import java.io.File; +import io.netty.bootstrap.ServerBootstrap; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelOption; +import io.netty.channel.Channel; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.nio.NioServerSocketChannel; +import io.netty.channel.oio.OioEventLoopGroup; +import io.netty.channel.socket.oio.OioServerSocketChannel; +import io.netty.handler.logging.LogLevel; +import io.netty.handler.logging.LoggingHandler; + +/** + * Server that accept the path of a file an echo back its content. + */ +public class FileServer { + + private ServerBootstrap bootstrap = null; + private Channel channel = null; + private PathResolver pResolver; + + public FileServer(PathResolver pResolver){ + this.pResolver = pResolver; + } + + public void run(int port) { + // Configure the server. + bootstrap = new ServerBootstrap(); + try { + bootstrap.group(new OioEventLoopGroup(), new OioEventLoopGroup()) + .channel(OioServerSocketChannel.class) + .option(ChannelOption.SO_BACKLOG, 100) + .option(ChannelOption.SO_RCVBUF, 1500) + .childHandler(new FileServerChannelInitializer(pResolver)); + // Start the server. + channel = bootstrap.bind(port).sync().channel(); + channel.closeFuture().sync(); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } finally{ + bootstrap.shutdown(); + } + } + + public void stop(){ + if (channel!=null){ + channel.close(); + } + if (bootstrap != null){ + bootstrap.shutdown(); + bootstrap = null; + } + } +} + + diff --git a/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java b/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java new file mode 100644 index 0000000000..9d0618ff1c --- /dev/null +++ b/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java @@ -0,0 +1,33 @@ +package spark.network.netty; + +import java.io.File; +import io.netty.buffer.BufType; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.socket.SocketChannel; +import io.netty.handler.codec.string.StringDecoder; +import io.netty.handler.codec.string.StringEncoder; +import io.netty.handler.codec.DelimiterBasedFrameDecoder; +import io.netty.handler.codec.Delimiters; +import io.netty.util.CharsetUtil; +import io.netty.handler.logging.LoggingHandler; +import io.netty.handler.logging.LogLevel; + +public class FileServerChannelInitializer extends + ChannelInitializer { + + PathResolver pResolver; + + public FileServerChannelInitializer(PathResolver pResolver) { + this.pResolver = pResolver; + } + + @Override + public void initChannel(SocketChannel channel) { + channel.pipeline() + .addLast("framer", new DelimiterBasedFrameDecoder( + 8192, Delimiters.lineDelimiter())) + .addLast("strDecoder", new StringDecoder()) + .addLast("handler", new FileServerHandler(pResolver)); + + } +} diff --git a/core/src/main/java/spark/network/netty/FileServerHandler.java b/core/src/main/java/spark/network/netty/FileServerHandler.java new file mode 100644 index 0000000000..e1083e87a2 --- /dev/null +++ b/core/src/main/java/spark/network/netty/FileServerHandler.java @@ -0,0 +1,68 @@ +package spark.network.netty; + +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundMessageHandlerAdapter; +import io.netty.channel.DefaultFileRegion; +import io.netty.handler.stream.ChunkedFile; +import java.io.File; +import java.io.FileInputStream; + +public class FileServerHandler extends + ChannelInboundMessageHandlerAdapter { + + PathResolver pResolver; + + public FileServerHandler(PathResolver pResolver){ + this.pResolver = pResolver; + } + + @Override + public void messageReceived(ChannelHandlerContext ctx, String blockId) { + String path = pResolver.getAbsolutePath(blockId); + // if getFilePath returns null, close the channel + if (path == null) { + //ctx.close(); + return; + } + File file = new File(path); + if (file.exists()) { + if (!file.isFile()) { + //logger.info("Not a file : " + file.getAbsolutePath()); + ctx.write(new FileHeader(0, blockId).buffer()); + ctx.flush(); + return; + } + long length = file.length(); + if (length > Integer.MAX_VALUE || length <= 0 ) { + //logger.info("too large file : " + file.getAbsolutePath() + " of size "+ length); + ctx.write(new FileHeader(0, blockId).buffer()); + ctx.flush(); + return; + } + int len = new Long(length).intValue(); + //logger.info("Sending block "+blockId+" filelen = "+len); + //logger.info("header = "+ (new FileHeader(len, blockId)).buffer()); + ctx.write((new FileHeader(len, blockId)).buffer()); + try { + ctx.sendFile(new DefaultFileRegion(new FileInputStream(file) + .getChannel(), 0, file.length())); + } catch (Exception e) { + // TODO Auto-generated catch block + //logger.warning("Exception when sending file : " + //+ file.getAbsolutePath()); + e.printStackTrace(); + } + } else { + //logger.warning("File not found: " + file.getAbsolutePath()); + ctx.write(new FileHeader(0, blockId).buffer()); + } + ctx.flush(); + } + + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + cause.printStackTrace(); + ctx.close(); + } +} diff --git a/core/src/main/java/spark/network/netty/PathResolver.java b/core/src/main/java/spark/network/netty/PathResolver.java new file mode 100755 index 0000000000..5d5eda006e --- /dev/null +++ b/core/src/main/java/spark/network/netty/PathResolver.java @@ -0,0 +1,12 @@ +package spark.network.netty; + +public interface PathResolver { + /** + * Get the absolute path of the file + * + * @param fileId + * @return the absolute path of file + */ + public String getAbsolutePath(String fileId); + +} diff --git a/core/src/main/scala/spark/network/netty/FileHeader.scala b/core/src/main/scala/spark/network/netty/FileHeader.scala new file mode 100644 index 0000000000..aed4254234 --- /dev/null +++ b/core/src/main/scala/spark/network/netty/FileHeader.scala @@ -0,0 +1,57 @@ +package spark.network.netty + +import io.netty.buffer._ + +import spark.Logging + +private[spark] class FileHeader ( + val fileLen: Int, + val blockId: String) extends Logging { + + lazy val buffer = { + val buf = Unpooled.buffer() + buf.capacity(FileHeader.HEADER_SIZE) + buf.writeInt(fileLen) + buf.writeInt(blockId.length) + blockId.foreach((x: Char) => buf.writeByte(x)) + //padding the rest of header + if (FileHeader.HEADER_SIZE - buf.readableBytes > 0 ) { + buf.writeZero(FileHeader.HEADER_SIZE - buf.readableBytes) + } else { + throw new Exception("too long header " + buf.readableBytes) + logInfo("too long header") + } + buf + } + +} + +private[spark] object FileHeader { + + val HEADER_SIZE = 40 + + def getFileLenOffset = 0 + def getFileLenSize = Integer.SIZE/8 + + def create(buf: ByteBuf): FileHeader = { + val length = buf.readInt + val idLength = buf.readInt + val idBuilder = new StringBuilder(idLength) + for (i <- 1 to idLength) { + idBuilder += buf.readByte().asInstanceOf[Char] + } + val blockId = idBuilder.toString() + new FileHeader(length, blockId) + } + + + def main (args:Array[String]){ + + val header = new FileHeader(25,"block_0"); + val buf = header.buffer; + val newheader = FileHeader.create(buf); + System.out.println("id="+newheader.blockId+",size="+newheader.fileLen) + + } +} + diff --git a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala new file mode 100644 index 0000000000..d8d35bfeec --- /dev/null +++ b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala @@ -0,0 +1,88 @@ +package spark.network.netty + +import io.netty.buffer.ByteBuf +import io.netty.channel.ChannelHandlerContext +import io.netty.channel.ChannelInboundByteHandlerAdapter +import io.netty.util.CharsetUtil + +import java.util.concurrent.atomic.AtomicInteger +import java.util.logging.Logger +import spark.Logging +import spark.network.ConnectionManagerId +import java.util.concurrent.Executors + +private[spark] class ShuffleCopier extends Logging { + + def getBlock(cmId: ConnectionManagerId, + blockId: String, + resultCollectCallback: (String, Long, ByteBuf) => Unit) = { + + val handler = new ShuffleClientHandler(resultCollectCallback) + val fc = new FileClient(handler) + fc.init() + fc.connect(cmId.host, cmId.port) + fc.sendRequest(blockId) + fc.waitForClose() + fc.close() + } + + def getBlocks(cmId: ConnectionManagerId, + blocks: Seq[(String, Long)], + resultCollectCallback: (String, Long, ByteBuf) => Unit) = { + + blocks.map { + case(blockId,size) => { + getBlock(cmId,blockId,resultCollectCallback) + } + } + } +} + +private[spark] class ShuffleClientHandler(val resultCollectCallBack: (String, Long, ByteBuf) => Unit ) extends FileClientHandler with Logging { + + def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) { + logDebug("Received Block: " + header.blockId + " (" + header.fileLen + "B)"); + resultCollectCallBack(header.blockId, header.fileLen.toLong, in.readBytes(header.fileLen)) + } +} + +private[spark] object ShuffleCopier extends Logging { + + def echoResultCollectCallBack(blockId: String, size: Long, content: ByteBuf) = { + logInfo("File: " + blockId + " content is : \" " + + content.toString(CharsetUtil.UTF_8) + "\"") + } + + def runGetBlock(host:String, port:Int, file:String){ + val handler = new ShuffleClientHandler(echoResultCollectCallBack) + val fc = new FileClient(handler) + fc.init(); + fc.connect(host, port) + fc.sendRequest(file) + fc.waitForClose(); + fc.close() + } + + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println("Usage: ShuffleCopier ") + System.exit(1) + } + val host = args(0) + val port = args(1).toInt + val file = args(2) + val threads = if (args.length>3) args(3).toInt else 10 + + val copiers = Executors.newFixedThreadPool(80) + for (i <- Range(0,threads)){ + val runnable = new Runnable() { + def run() { + runGetBlock(host,port,file) + } + } + copiers.execute(runnable) + } + copiers.shutdown + } + +} diff --git a/core/src/main/scala/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/spark/network/netty/ShuffleSender.scala new file mode 100644 index 0000000000..c1986812e9 --- /dev/null +++ b/core/src/main/scala/spark/network/netty/ShuffleSender.scala @@ -0,0 +1,50 @@ +package spark.network.netty + +import spark.Logging +import java.io.File + + +private[spark] class ShuffleSender(val port: Int, val pResolver:PathResolver) extends Logging { + val server = new FileServer(pResolver) + + Runtime.getRuntime().addShutdownHook( + new Thread() { + override def run() { + server.stop() + } + } + ) + + def start() { + server.run(port) + } +} + +private[spark] object ShuffleSender { + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println("Usage: ShuffleSender ") + System.exit(1) + } + val port = args(0).toInt + val subDirsPerLocalDir = args(1).toInt + val localDirs = args.drop(2) map {new File(_)} + val pResovler = new PathResolver { + def getAbsolutePath(blockId:String):String = { + if (!blockId.startsWith("shuffle_")) { + throw new Exception("Block " + blockId + " is not a shuffle block") + } + // Figure out which local directory it hashes to, and which subdirectory in that + val hash = math.abs(blockId.hashCode) + val dirId = hash % localDirs.length + val subDirId = (hash / localDirs.length) % subDirsPerLocalDir + val subDir = new File(localDirs(dirId), "%02x".format(subDirId)) + val file = new File(subDir, blockId) + return file.getAbsolutePath + } + } + val sender = new ShuffleSender(port, pResovler) + + sender.start() + } +} diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 210061e972..b8b68d4283 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -23,6 +23,8 @@ import spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStam import sun.nio.ch.DirectBuffer +import spark.network.netty.ShuffleCopier +import io.netty.buffer.ByteBuf private[spark] case class BlockException(blockId: String, message: String, ex: Exception = null) @@ -467,6 +469,21 @@ class BlockManager( getLocal(blockId).orElse(getRemote(blockId)) } + /** + * A request to fetch one or more blocks, complete with their sizes + */ + class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) { + val size = blocks.map(_._2).sum + } + + /** + * A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize + * the block (since we want all deserializaton to happen in the calling thread); can also + * represent a fetch failure if size == -1. + */ + class FetchResult(val blockId: String, val size: Long, val deserialize: () => Iterator[Any]) { + def failed: Boolean = size == -1 + } /** * Get multiple blocks from local and remote block manager using their BlockManagerIds. Returns * an Iterator of (block ID, value) pairs so that clients may handle blocks in a pipelined @@ -475,7 +492,12 @@ class BlockManager( */ def getMultiple(blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])]) : BlockFetcherIterator = { - return new BlockFetcherIterator(this, blocksByAddress) + + if(System.getProperty("spark.shuffle.use.netty", "false").toBoolean){ + return new NettyBlockFetcherIterator(this, blocksByAddress) + } else { + return new BlockFetcherIterator(this, blocksByAddress) + } } def put(blockId: String, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean) @@ -908,7 +930,7 @@ class BlockFetcherIterator( if (blocksByAddress == null) { throw new IllegalArgumentException("BlocksByAddress is null") } - val totalBlocks = blocksByAddress.map(_._2.size).sum + var totalBlocks = blocksByAddress.map(_._2.size).sum logDebug("Getting " + totalBlocks + " blocks") var startTime = System.currentTimeMillis val localBlockIds = new ArrayBuffer[String]() @@ -974,68 +996,83 @@ class BlockFetcherIterator( } } - // Split local and remote blocks. Remote blocks are further split into FetchRequests of size - // at most maxBytesInFlight in order to limit the amount of data in flight. - val remoteRequests = new ArrayBuffer[FetchRequest] - for ((address, blockInfos) <- blocksByAddress) { - if (address == blockManagerId) { - localBlockIds ++= blockInfos.map(_._1) - } else { - remoteBlockIds ++= blockInfos.map(_._1) - // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them - // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 - // nodes, rather than blocking on reading output from one node. - val minRequestSize = math.max(maxBytesInFlight / 5, 1L) - logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) - val iterator = blockInfos.iterator - var curRequestSize = 0L - var curBlocks = new ArrayBuffer[(String, Long)] - while (iterator.hasNext) { - val (blockId, size) = iterator.next() - curBlocks += ((blockId, size)) - curRequestSize += size - if (curRequestSize >= minRequestSize) { - // Add this FetchRequest + def splitLocalRemoteBlocks():ArrayBuffer[FetchRequest] = { + // Split local and remote blocks. Remote blocks are further split into FetchRequests of size + // at most maxBytesInFlight in order to limit the amount of data in flight. + val remoteRequests = new ArrayBuffer[FetchRequest] + for ((address, blockInfos) <- blocksByAddress) { + if (address == blockManagerId) { + localBlockIds ++= blockInfos.map(_._1) + } else { + remoteBlockIds ++= blockInfos.map(_._1) + // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + val minRequestSize = math.max(maxBytesInFlight / 5, 1L) + logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) + val iterator = blockInfos.iterator + var curRequestSize = 0L + var curBlocks = new ArrayBuffer[(String, Long)] + while (iterator.hasNext) { + val (blockId, size) = iterator.next() + curBlocks += ((blockId, size)) + curRequestSize += size + if (curRequestSize >= minRequestSize) { + // Add this FetchRequest + remoteRequests += new FetchRequest(address, curBlocks) + curRequestSize = 0 + curBlocks = new ArrayBuffer[(String, Long)] + } + } + // Add in the final request + if (!curBlocks.isEmpty) { remoteRequests += new FetchRequest(address, curBlocks) - curRequestSize = 0 - curBlocks = new ArrayBuffer[(String, Long)] } } - // Add in the final request - if (!curBlocks.isEmpty) { - remoteRequests += new FetchRequest(address, curBlocks) - } } + remoteRequests } - // Add the remote requests into our queue in a random order - fetchRequests ++= Utils.randomize(remoteRequests) - // Send out initial requests for blocks, up to our maxBytesInFlight - while (!fetchRequests.isEmpty && - (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - sendRequest(fetchRequests.dequeue()) + def getLocalBlocks(){ + // Get the local blocks while remote blocks are being fetched. Note that it's okay to do + // these all at once because they will just memory-map some files, so they won't consume + // any memory that might exceed our maxBytesInFlight + for (id <- localBlockIds) { + getLocal(id) match { + case Some(iter) => { + results.put(new FetchResult(id, 0, () => iter)) // Pass 0 as size since it's not in flight + logDebug("Got local block " + id) + } + case None => { + throw new BlockException(id, "Could not get block " + id + " from local machine") + } + } + } } - val numGets = remoteBlockIds.size - fetchRequests.size - logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime)) - - // Get the local blocks while remote blocks are being fetched. Note that it's okay to do - // these all at once because they will just memory-map some files, so they won't consume - // any memory that might exceed our maxBytesInFlight - startTime = System.currentTimeMillis - for (id <- localBlockIds) { - getLocal(id) match { - case Some(iter) => { - results.put(new FetchResult(id, 0, () => iter)) // Pass 0 as size since it's not in flight - logDebug("Got local block " + id) - } - case None => { - throw new BlockException(id, "Could not get block " + id + " from local machine") - } + def initialize(){ + // Split local and remote blocks. + val remoteRequests = splitLocalRemoteBlocks() + // Add the remote requests into our queue in a random order + fetchRequests ++= Utils.randomize(remoteRequests) + + // Send out initial requests for blocks, up to our maxBytesInFlight + while (!fetchRequests.isEmpty && + (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { + sendRequest(fetchRequests.dequeue()) } + + val numGets = remoteBlockIds.size - fetchRequests.size + logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime)) + + // Get Local Blocks + startTime = System.currentTimeMillis + getLocalBlocks() + logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") + } - logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") + initialize() //an iterator that will read fetched blocks off the queue as they arrive. var resultsGotten = 0 @@ -1066,3 +1103,132 @@ class BlockFetcherIterator( def remoteBytesRead = _remoteBytesRead } + +class NettyBlockFetcherIterator( + blockManager: BlockManager, + blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])] +) extends BlockFetcherIterator(blockManager,blocksByAddress) { + + import blockManager._ + + val fetchRequestsSync = new LinkedBlockingQueue[FetchRequest] + + def putResult(blockId:String, blockSize:Long, blockData:ByteBuffer, + results : LinkedBlockingQueue[FetchResult]){ + results.put(new FetchResult( + blockId, blockSize, () => dataDeserialize(blockId, blockData) )) + } + + def startCopiers (numCopiers: Int): List [ _ <: Thread]= { + (for ( i <- Range(0,numCopiers) ) yield { + val copier = new Thread { + override def run(){ + try { + while(!isInterrupted && !fetchRequestsSync.isEmpty) { + sendRequest(fetchRequestsSync.take()) + } + } catch { + case x: InterruptedException => logInfo("Copier Interrupted") + case _ => throw new SparkException("Exception Throw in Shuffle Copier") + } + } + } + copier.start + copier + }).toList + } + + //keep this to interrupt the threads when necessary + def stopCopiers(copiers : List[_ <: Thread]) { + for (copier <- copiers) { + copier.interrupt() + } + } + + override def sendRequest(req: FetchRequest) { + logDebug("Sending request for %d blocks (%s) from %s".format( + req.blocks.size, Utils.memoryBytesToString(req.size), req.address.ip)) + val cmId = new ConnectionManagerId(req.address.ip, System.getProperty("spark.shuffle.sender.port", "6653").toInt) + val cpier = new ShuffleCopier + cpier.getBlocks(cmId,req.blocks,(blockId:String,blockSize:Long,blockData:ByteBuf) => putResult(blockId,blockSize,blockData.nioBuffer,results)) + logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.ip ) + } + + override def splitLocalRemoteBlocks() : ArrayBuffer[FetchRequest] = { + // Split local and remote blocks. Remote blocks are further split into FetchRequests of size + // at most maxBytesInFlight in order to limit the amount of data in flight. + val originalTotalBlocks = totalBlocks; + val remoteRequests = new ArrayBuffer[FetchRequest] + for ((address, blockInfos) <- blocksByAddress) { + if (address == blockManagerId) { + localBlockIds ++= blockInfos.map(_._1) + } else { + remoteBlockIds ++= blockInfos.map(_._1) + // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + val minRequestSize = math.max(maxBytesInFlight / 5, 1L) + logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) + val iterator = blockInfos.iterator + var curRequestSize = 0L + var curBlocks = new ArrayBuffer[(String, Long)] + while (iterator.hasNext) { + val (blockId, size) = iterator.next() + if (size > 0) { + curBlocks += ((blockId, size)) + curRequestSize += size + } else if (size == 0){ + //here we changes the totalBlocks + totalBlocks -= 1 + } else { + throw new SparkException("Negative block size "+blockId) + } + if (curRequestSize >= minRequestSize) { + // Add this FetchRequest + remoteRequests += new FetchRequest(address, curBlocks) + curRequestSize = 0 + curBlocks = new ArrayBuffer[(String, Long)] + } + } + // Add in the final request + if (!curBlocks.isEmpty) { + remoteRequests += new FetchRequest(address, curBlocks) + } + } + } + logInfo("Getting " + totalBlocks + " non 0-byte blocks out of " + originalTotalBlocks + " blocks") + remoteRequests + } + + var copiers : List[_ <: Thread] = null + + override def initialize(){ + // Split Local Remote Blocks and adjust totalBlocks to include only the non 0-byte blocks + val remoteRequests = splitLocalRemoteBlocks() + // Add the remote requests into our queue in a random order + for (request <- Utils.randomize(remoteRequests)) { + fetchRequestsSync.put(request) + } + + copiers = startCopiers(System.getProperty("spark.shuffle.copier.threads", "6").toInt) + logInfo("Started " + fetchRequestsSync.size + " remote gets in " + Utils.getUsedTimeMs(startTime)) + + // Get Local Blocks + startTime = System.currentTimeMillis + getLocalBlocks() + logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") + } + + override def next(): (String, Option[Iterator[Any]]) = { + resultsGotten += 1 + val result = results.take() + // if all the results has been retrieved + // shutdown the copiers + if (resultsGotten == totalBlocks) { + if( copiers != null ) + stopCopiers(copiers) + } + (result.blockId, if (result.failed) None else Some(result.deserialize())) + } + } + diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index ddbf8821ad..d702bb23e0 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -13,24 +13,35 @@ import scala.collection.mutable.ArrayBuffer import spark.executor.ExecutorExitCode import spark.Utils +import spark.Logging +import spark.network.netty.ShuffleSender +import spark.network.netty.PathResolver /** * Stores BlockManager blocks on disk. */ private class DiskStore(blockManager: BlockManager, rootDirs: String) - extends BlockStore(blockManager) { + extends BlockStore(blockManager) with Logging { val MAX_DIR_CREATION_ATTEMPTS: Int = 10 val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt + var shuffleSender : Thread = null + val thisInstance = this // Create one local directory for each path mentioned in spark.local.dir; then, inside this // directory, create multiple subdirectories that we will hash files into, in order to avoid // having really large inodes at the top level. val localDirs = createLocalDirs() val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) + val useNetty = System.getProperty("spark.shuffle.use.netty", "false").toBoolean + addShutdownHook() + if(useNetty){ + startShuffleBlockSender() + } + override def getSize(blockId: String): Long = { getFile(blockId).length() } @@ -180,10 +191,48 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) logDebug("Shutdown hook called") try { localDirs.foreach(localDir => Utils.deleteRecursively(localDir)) + if (useNetty && shuffleSender != null) + shuffleSender.stop } catch { case t: Throwable => logError("Exception while deleting local spark dirs", t) } } }) } + + private def startShuffleBlockSender (){ + try { + val port = System.getProperty("spark.shuffle.sender.port", "6653").toInt + + val pResolver = new PathResolver { + def getAbsolutePath(blockId:String):String = { + if (!blockId.startsWith("shuffle_")) { + return null + } + thisInstance.getFile(blockId).getAbsolutePath() + } + } + shuffleSender = new Thread { + override def run() = { + val sender = new ShuffleSender(port,pResolver) + logInfo("created ShuffleSender binding to port : "+ port) + sender.start + } + } + shuffleSender.setDaemon(true) + shuffleSender.start + + } catch { + case interrupted: InterruptedException => + logInfo("Runner thread for ShuffleBlockSender interrupted") + + case e: Exception => { + logError("Error running ShuffleBlockSender ", e) + if (shuffleSender != null) { + shuffleSender.stop + shuffleSender = null + } + } + } + } } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5f378b2398..e3645653ee 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -141,7 +141,8 @@ object SparkBuild extends Build { "cc.spray" % "spray-can" % "1.0-M2.1", "cc.spray" % "spray-server" % "1.0-M2.1", "cc.spray" %% "spray-json" % "1.1.1", - "org.apache.mesos" % "mesos" % "0.9.0-incubating" + "org.apache.mesos" % "mesos" % "0.9.0-incubating", + "io.netty" % "netty-all" % "4.0.0.Beta2" ) ++ (if (HADOOP_MAJOR_VERSION == "2") Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION) else None).toSeq, unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / ("src/hadoop" + HADOOP_MAJOR_VERSION + "/scala") } ) ++ assemblySettings ++ extraAssemblySettings ++ Twirl.settings diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala index d8b987ec86..bd0b0e74c1 100644 --- a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala +++ b/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala @@ -5,7 +5,7 @@ import spark.util.{RateLimitedOutputStream, IntParam} import java.net.ServerSocket import spark.{Logging, KryoSerializer} import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream -import io.Source +import scala.io.Source import java.io.IOException /** -- cgit v1.2.3 From 2f883c515fe4577f0105e62dd9f395d7de42bd68 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Tue, 9 Apr 2013 13:02:50 +0800 Subject: Contiue to update codes for scala code style 1.refactor braces for "class" "if" "while" "for" "match" 2.make code lines less than 100 3.refactor class parameter and extends defination --- core/src/main/scala/spark/SparkContext.scala | 12 +- .../scala/spark/scheduler/DAGSchedulerEvent.scala | 3 +- core/src/main/scala/spark/scheduler/TaskSet.scala | 9 +- .../spark/scheduler/cluster/ClusterScheduler.scala | 6 +- .../cluster/FIFOTaskSetQueuesManager.scala | 10 +- .../cluster/FairTaskSetQueuesManager.scala | 139 +++++++++------------ .../main/scala/spark/scheduler/cluster/Pool.scala | 46 ++++--- .../spark/scheduler/cluster/Schedulable.scala | 3 +- .../scheduler/cluster/SchedulingAlgorithm.scala | 30 ++--- .../spark/scheduler/cluster/SchedulingMode.scala | 5 +- .../spark/scheduler/cluster/TaskSetManager.scala | 6 +- 11 files changed, 122 insertions(+), 147 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index ed5f686379..7c96ae637b 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -146,7 +146,8 @@ class SparkContext( case SPARK_REGEX(sparkUrl) => val scheduler = new ClusterScheduler(this) val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName) - val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")).newInstance().asInstanceOf[TaskSetQueuesManager] + val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")). + newInstance().asInstanceOf[TaskSetQueuesManager] scheduler.initialize(backend, taskSetQueuesManager) scheduler @@ -166,7 +167,8 @@ class SparkContext( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) val sparkUrl = localCluster.start() val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName) - val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")).newInstance().asInstanceOf[TaskSetQueuesManager] + val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")). + newInstance().asInstanceOf[TaskSetQueuesManager] scheduler.initialize(backend, taskSetQueuesManager) backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => { localCluster.stop() @@ -186,7 +188,8 @@ class SparkContext( } else { new MesosSchedulerBackend(scheduler, this, masterWithoutProtocol, appName) } - val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")).newInstance().asInstanceOf[TaskSetQueuesManager] + val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")). + newInstance().asInstanceOf[TaskSetQueuesManager] scheduler.initialize(backend, taskSetQueuesManager) scheduler } @@ -602,7 +605,8 @@ class SparkContext( val callSite = Utils.getSparkCallSite logInfo("Starting job: " + callSite) val start = System.nanoTime - val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal, resultHandler,localProperties.value) + val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal, resultHandler + ,localProperties.value) logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s") rdd.doCheckpoint() result diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index 6f4e5cd83e..11fec568c6 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -23,7 +23,8 @@ private[spark] case class JobSubmitted( partitions: Array[Int], allowLocal: Boolean, callSite: String, - listener: JobListener, properties: Properties = null) + listener: JobListener, + properties: Properties = null) extends DAGSchedulerEvent private[spark] case class CompletionEvent( diff --git a/core/src/main/scala/spark/scheduler/TaskSet.scala b/core/src/main/scala/spark/scheduler/TaskSet.scala index 2498e8a5aa..e4b5fcaedb 100644 --- a/core/src/main/scala/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/spark/scheduler/TaskSet.scala @@ -6,8 +6,13 @@ import java.util.Properties * A set of tasks submitted together to the low-level TaskScheduler, usually representing * missing partitions of a particular stage. */ -private[spark] class TaskSet(val tasks: Array[Task[_]], val stageId: Int, val attempt: Int, val priority: Int, val properties: Properties) { - val id: String = stageId + "." + attempt +private[spark] class TaskSet( + val tasks: Array[Task[_]], + val stageId: Int, + val attempt: Int, + val priority: Int, + val properties: Properties) { + val id: String = stageId + "." + attempt override def toString: String = "TaskSet " + id } diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index be0d480aa0..2ddac0ff30 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -155,13 +155,11 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // Build a list of tasks to assign to each slave val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) val availableCpus = offers.map(o => o.cores).toArray - for (i <- 0 until offers.size) - { + for (i <- 0 until offers.size){ var launchedTask = true val execId = offers(i).executorId val host = offers(i).hostname - while (availableCpus(i) > 0 && launchedTask) - { + while (availableCpus(i) > 0 && launchedTask){ launchedTask = false taskSetQueuesManager.receiveOffer(execId,host,availableCpus(i)) match { case Some(task) => diff --git a/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala index 5949ee773f..62d3130341 100644 --- a/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala @@ -28,13 +28,11 @@ private[spark] class FIFOTaskSetQueuesManager extends TaskSetQueuesManager with activeTaskSetsQueue.foreach(_.executorLost(executorId, host)) } - override def receiveOffer(execId:String, host:String,avaiableCpus:Double):Option[TaskDescription] = - { - for(manager <- activeTaskSetsQueue.sortWith(tasksetSchedulingAlgorithm.comparator)) - { + override def receiveOffer(execId:String, host:String,avaiableCpus:Double):Option[TaskDescription] = { + + for (manager <- activeTaskSetsQueue.sortWith(tasksetSchedulingAlgorithm.comparator)) { val task = manager.slaveOffer(execId,host,avaiableCpus) - if (task != None) - { + if (task != None) { return task } } diff --git a/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala index 0609600f35..89b74fbb47 100644 --- a/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala @@ -38,71 +38,17 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with loadPoolProperties() - def loadPoolProperties() { - //first check if the file exists - val file = new File(schedulerAllocFile) - if(file.exists()) - { - val xml = XML.loadFile(file) - for (poolNode <- (xml \\ POOL_POOLS_PROPERTY)) { - - val poolName = (poolNode \ POOL_POOL_NAME_PROPERTY).text - var schedulingMode = POOL_DEFAULT_SCHEDULING_MODE - var minShares = POOL_DEFAULT_MINIMUM_SHARES - var weight = POOL_DEFAULT_WEIGHT - - val xmlSchedulingMode = (poolNode \ POOL_SCHEDULING_MODE_PROPERTY).text - if( xmlSchedulingMode != "") - { - try - { - schedulingMode = SchedulingMode.withName(xmlSchedulingMode) - } - catch{ - case e:Exception => logInfo("Error xml schedulingMode, using default schedulingMode") - } - } - - val xmlMinShares = (poolNode \ POOL_MINIMUM_SHARES_PROPERTY).text - if(xmlMinShares != "") - { - minShares = xmlMinShares.toInt - } - - val xmlWeight = (poolNode \ POOL_WEIGHT_PROPERTY).text - if(xmlWeight != "") - { - weight = xmlWeight.toInt - } - - val pool = new Pool(poolName,schedulingMode,minShares,weight) - pools += pool - poolNameToPool(poolName) = pool - logInfo("Create new pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format(poolName,schedulingMode,minShares,weight)) - } - } - - if(!poolNameToPool.contains(POOL_DEFAULT_POOL_NAME)) - { - val pool = new Pool(POOL_DEFAULT_POOL_NAME, POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT) - pools += pool - poolNameToPool(POOL_DEFAULT_POOL_NAME) = pool - logInfo("Create default pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format(POOL_DEFAULT_POOL_NAME,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) - } - } - override def addTaskSetManager(manager: TaskSetManager) { var poolName = POOL_DEFAULT_POOL_NAME - if(manager.taskSet.properties != null) - { + if (manager.taskSet.properties != null) { poolName = manager.taskSet.properties.getProperty(POOL_FAIR_SCHEDULER_PROPERTIES,POOL_DEFAULT_POOL_NAME) - if(!poolNameToPool.contains(poolName)) - { + if (!poolNameToPool.contains(poolName)) { //we will create a new pool that user has configured in app instead of being defined in xml file val pool = new Pool(poolName,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT) pools += pool poolNameToPool(poolName) = pool - logInfo("Create pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format(poolName,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) + logInfo("Create pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format( + poolName,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) } } poolNameToPool(poolName).addTaskSetManager(manager) @@ -110,10 +56,8 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with } override def removeTaskSetManager(manager: TaskSetManager) { - var poolName = POOL_DEFAULT_POOL_NAME - if(manager.taskSet.properties != null) - { + if (manager.taskSet.properties != null) { poolName = manager.taskSet.properties.getProperty(POOL_FAIR_SCHEDULER_PROPERTIES,POOL_DEFAULT_POOL_NAME) } logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id,poolName)) @@ -124,8 +68,7 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with override def taskFinished(manager: TaskSetManager) { var poolName = POOL_DEFAULT_POOL_NAME - if(manager.taskSet.properties != null) - { + if (manager.taskSet.properties != null) { poolName = manager.taskSet.properties.getProperty(POOL_FAIR_SCHEDULER_PROPERTIES,POOL_DEFAULT_POOL_NAME) } val pool = poolNameToPool(poolName) @@ -139,19 +82,15 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with } } - override def receiveOffer(execId: String,host:String,avaiableCpus:Double):Option[TaskDescription] = - { - + override def receiveOffer(execId: String,host:String,avaiableCpus:Double):Option[TaskDescription] = { val sortedPools = pools.sortWith(poolScheduleAlgorithm.comparator) - for(pool <- sortedPools) - { - logDebug("poolName:%s,tasksetNum:%d,minShares:%d,runningTasks:%d".format(pool.poolName,pool.activeTaskSetsQueue.length,pool.minShare,pool.runningTasks)) + for (pool <- sortedPools) { + logDebug("poolName:%s,tasksetNum:%d,minShares:%d,runningTasks:%d".format( + pool.poolName,pool.activeTaskSetsQueue.length,pool.minShare,pool.runningTasks)) } - for (pool <- sortedPools) - { + for (pool <- sortedPools) { val task = pool.receiveOffer(execId,host,avaiableCpus) - if(task != None) - { + if(task != None) { pool.runningTasks += 1 return task } @@ -159,14 +98,60 @@ private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with return None } - override def checkSpeculatableTasks(): Boolean = - { + override def checkSpeculatableTasks(): Boolean = { var shouldRevive = false - for (pool <- pools) - { + for (pool <- pools) { shouldRevive |= pool.checkSpeculatableTasks() } return shouldRevive } + def loadPoolProperties() { + //first check if the file exists + val file = new File(schedulerAllocFile) + if (file.exists()) { + val xml = XML.loadFile(file) + for (poolNode <- (xml \\ POOL_POOLS_PROPERTY)) { + + val poolName = (poolNode \ POOL_POOL_NAME_PROPERTY).text + var schedulingMode = POOL_DEFAULT_SCHEDULING_MODE + var minShares = POOL_DEFAULT_MINIMUM_SHARES + var weight = POOL_DEFAULT_WEIGHT + + val xmlSchedulingMode = (poolNode \ POOL_SCHEDULING_MODE_PROPERTY).text + if (xmlSchedulingMode != "") { + try{ + schedulingMode = SchedulingMode.withName(xmlSchedulingMode) + } + catch{ + case e:Exception => logInfo("Error xml schedulingMode, using default schedulingMode") + } + } + + val xmlMinShares = (poolNode \ POOL_MINIMUM_SHARES_PROPERTY).text + if (xmlMinShares != "") { + minShares = xmlMinShares.toInt + } + + val xmlWeight = (poolNode \ POOL_WEIGHT_PROPERTY).text + if (xmlWeight != "") { + weight = xmlWeight.toInt + } + + val pool = new Pool(poolName,schedulingMode,minShares,weight) + pools += pool + poolNameToPool(poolName) = pool + logInfo("Create new pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format( + poolName,schedulingMode,minShares,weight)) + } + } + + if (!poolNameToPool.contains(POOL_DEFAULT_POOL_NAME)) { + val pool = new Pool(POOL_DEFAULT_POOL_NAME, POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT) + pools += pool + poolNameToPool(POOL_DEFAULT_POOL_NAME) = pool + logInfo("Create default pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format( + POOL_DEFAULT_POOL_NAME,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) + } + } } diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/spark/scheduler/cluster/Pool.scala index 8fdca5d2b4..e0917ca1ca 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Pool.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Pool.scala @@ -7,8 +7,13 @@ import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * An Schedulable entity that represent collection of TaskSetManager */ -private[spark] class Pool(val poolName: String,val schedulingMode: SchedulingMode, initMinShare:Int, initWeight:Int) extends Schedulable with Logging -{ +private[spark] class Pool( + val poolName: String, + val schedulingMode: SchedulingMode, + initMinShare:Int, + initWeight:Int) + extends Schedulable + with Logging { var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] @@ -19,10 +24,8 @@ private[spark] class Pool(val poolName: String,val schedulingMode: SchedulingMod val priority = 0 val stageId = 0 - var taskSetSchedulingAlgorithm: SchedulingAlgorithm = - { - schedulingMode match - { + var taskSetSchedulingAlgorithm: SchedulingAlgorithm = { + schedulingMode match { case SchedulingMode.FAIR => val schedule = new FairSchedulingAlgorithm() schedule @@ -32,43 +35,36 @@ private[spark] class Pool(val poolName: String,val schedulingMode: SchedulingMod } } - def addTaskSetManager(manager:TaskSetManager) - { + def addTaskSetManager(manager:TaskSetManager) { activeTaskSetsQueue += manager } - def removeTaskSetManager(manager:TaskSetManager) - { + def removeTaskSetManager(manager:TaskSetManager) { activeTaskSetsQueue -= manager } - def removeExecutor(executorId: String, host: String) - { + def removeExecutor(executorId: String, host: String) { activeTaskSetsQueue.foreach(_.executorLost(executorId,host)) } - def checkSpeculatableTasks(): Boolean = - { + def checkSpeculatableTasks(): Boolean = { var shouldRevive = false - for(ts <- activeTaskSetsQueue) - { + for (ts <- activeTaskSetsQueue) { shouldRevive |= ts.checkSpeculatableTasks() } return shouldRevive } - def receiveOffer(execId:String,host:String,availableCpus:Double):Option[TaskDescription] = - { + def receiveOffer(execId:String,host:String,availableCpus:Double):Option[TaskDescription] = { val sortedActiveTasksSetQueue = activeTaskSetsQueue.sortWith(taskSetSchedulingAlgorithm.comparator) - for(manager <- sortedActiveTasksSetQueue) - { - logDebug("poolname:%s,taskSetId:%s,taskNum:%d,minShares:%d,weight:%d,runningTasks:%d".format(poolName,manager.taskSet.id,manager.numTasks,manager.minShare,manager.weight,manager.runningTasks)) + for (manager <- sortedActiveTasksSetQueue) { + logDebug("poolname:%s,taskSetId:%s,taskNum:%d,minShares:%d,weight:%d,runningTasks:%d".format( + poolName,manager.taskSet.id,manager.numTasks,manager.minShare,manager.weight,manager.runningTasks)) } - for(manager <- sortedActiveTasksSetQueue) - { + + for (manager <- sortedActiveTasksSetQueue) { val task = manager.slaveOffer(execId,host,availableCpus) - if (task != None) - { + if (task != None) { manager.runningTasks += 1 return task } diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala index 6f4f104f42..8dfc369c03 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala @@ -1,7 +1,8 @@ package spark.scheduler.cluster /** - * An interface for schedulable entities, there are two type Schedulable entities(Pools and TaskSetManagers) + * An interface for schedulable entities. + * there are two type of Schedulable entities(Pools and TaskSetManagers) */ private[spark] trait Schedulable { def weight:Int diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala index 2f8123587f..ac2237a7ef 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala @@ -9,34 +9,25 @@ private[spark] trait SchedulingAlgorithm { def comparator(s1: Schedulable,s2: Schedulable): Boolean } -private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm -{ - override def comparator(s1: Schedulable, s2: Schedulable): Boolean = - { +private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm { + override def comparator(s1: Schedulable, s2: Schedulable): Boolean = { val priority1 = s1.priority val priority2 = s2.priority var res = Math.signum(priority1 - priority2) - if (res == 0) - { + if (res == 0) { val stageId1 = s1.stageId val stageId2 = s2.stageId res = Math.signum(stageId1 - stageId2) } if (res < 0) - { return true - } else - { return false - } } } -private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm -{ - def comparator(s1: Schedulable, s2:Schedulable): Boolean = - { +private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { + def comparator(s1: Schedulable, s2:Schedulable): Boolean = { val minShare1 = s1.minShare val minShare2 = s2.minShare val runningTasks1 = s1.runningTasks @@ -49,22 +40,15 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm val taskToWeightRatio2 = runningTasks2.toDouble / s2.weight.toDouble var res:Boolean = true - if(s1Needy && !s2Needy) - { + if (s1Needy && !s2Needy) res = true - } else if(!s1Needy && s2Needy) - { res = false - } else if (s1Needy && s2Needy) - { res = minShareRatio1 <= minShareRatio2 - } else - { res = taskToWeightRatio1 <= taskToWeightRatio2 - } + return res } } diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala index 480af2c1a3..6e0c6793e0 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala @@ -1,8 +1,7 @@ package spark.scheduler.cluster -object SchedulingMode extends Enumeration("FAIR","FIFO") -{ - type SchedulingMode = Value +object SchedulingMode extends Enumeration("FAIR","FIFO"){ + type SchedulingMode = Value val FAIR,FIFO = Value } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index ddc4fa6642..7ec2f69da5 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -17,7 +17,11 @@ import java.nio.ByteBuffer /** * Schedules the tasks within a single TaskSet in the ClusterScheduler. */ -private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSet) extends Schedulable with Logging { +private[spark] class TaskSetManager( + sched: ClusterScheduler, + val taskSet: TaskSet) + extends Schedulable + with Logging { // Maximum time to wait to run a task in a preferred location (in ms) val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong -- cgit v1.2.3 From b42d68c8ce9f63513969297b65f4b5a2b06e6078 Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 15 Apr 2013 12:54:55 -0600 Subject: fixing Spark Streaming count() so that 0 will be emitted when there is nothing to count --- streaming/src/main/scala/spark/streaming/DStream.scala | 5 ++++- streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala | 4 ++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index e1be5ef51c..e3a9247924 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -441,7 +441,10 @@ abstract class DStream[T: ClassManifest] ( * Return a new DStream in which each RDD has a single element generated by counting each RDD * of this DStream. */ - def count(): DStream[Long] = this.map(_ => 1L).reduce(_ + _) + def count(): DStream[Long] = { + val zero = new ConstantInputDStream(context, context.sparkContext.makeRDD(Seq((null, 0L)), 1)) + this.map(_ => (null, 1L)).union(zero).reduceByKey(_ + _).map(_._2) + } /** * Return a new DStream in which each RDD contains the counts of each distinct value in diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala index 8fce91853c..168e1b7a55 100644 --- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala @@ -90,9 +90,9 @@ class BasicOperationsSuite extends TestSuiteBase { test("count") { testOperation( - Seq(1 to 1, 1 to 2, 1 to 3, 1 to 4), + Seq(Seq(), 1 to 1, 1 to 2, 1 to 3, 1 to 4), (s: DStream[Int]) => s.count(), - Seq(Seq(1L), Seq(2L), Seq(3L), Seq(4L)) + Seq(Seq(0L), Seq(1L), Seq(2L), Seq(3L), Seq(4L)) ) } -- cgit v1.2.3 From b493f55a4fe43c83061a361eef029edbac50c006 Mon Sep 17 00:00:00 2001 From: shane-huang Date: Tue, 16 Apr 2013 10:00:33 +0800 Subject: fix a bug in netty Block Fetcher Signed-off-by: shane-huang --- .../main/java/spark/network/netty/FileServer.java | 1 - .../main/scala/spark/storage/BlockManager.scala | 69 +++++++++++----------- core/src/main/scala/spark/storage/DiskStore.scala | 4 +- 3 files changed, 37 insertions(+), 37 deletions(-) diff --git a/core/src/main/java/spark/network/netty/FileServer.java b/core/src/main/java/spark/network/netty/FileServer.java index 729e45f0a1..38af305096 100644 --- a/core/src/main/java/spark/network/netty/FileServer.java +++ b/core/src/main/java/spark/network/netty/FileServer.java @@ -51,7 +51,6 @@ public class FileServer { } if (bootstrap != null){ bootstrap.shutdown(); - bootstrap = null; } } } diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index b8b68d4283..5a00180922 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -469,21 +469,6 @@ class BlockManager( getLocal(blockId).orElse(getRemote(blockId)) } - /** - * A request to fetch one or more blocks, complete with their sizes - */ - class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) { - val size = blocks.map(_._2).sum - } - - /** - * A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize - * the block (since we want all deserializaton to happen in the calling thread); can also - * represent a fetch failure if size == -1. - */ - class FetchResult(val blockId: String, val size: Long, val deserialize: () => Iterator[Any]) { - def failed: Boolean = size == -1 - } /** * Get multiple blocks from local and remote block manager using their BlockManagerIds. Returns * an Iterator of (block ID, value) pairs so that clients may handle blocks in a pipelined @@ -494,9 +479,9 @@ class BlockManager( : BlockFetcherIterator = { if(System.getProperty("spark.shuffle.use.netty", "false").toBoolean){ - return new NettyBlockFetcherIterator(this, blocksByAddress) + return BlockFetcherIterator("netty",this, blocksByAddress) } else { - return new BlockFetcherIterator(this, blocksByAddress) + return BlockFetcherIterator("", this, blocksByAddress) } } @@ -916,10 +901,29 @@ object BlockManager extends Logging { } } -class BlockFetcherIterator( + +trait BlockFetcherIterator extends Iterator[(String, Option[Iterator[Any]])] with Logging with BlockFetchTracker { + def initialize +} + +object BlockFetcherIterator { + + // A request to fetch one or more blocks, complete with their sizes + class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) { + val size = blocks.map(_._2).sum + } + + // A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize + // the block (since we want all deserializaton to happen in the calling thread); can also + // represent a fetch failure if size == -1. + class FetchResult(val blockId: String, val size: Long, val deserialize: () => Iterator[Any]) { + def failed: Boolean = size == -1 + } + +class BasicBlockFetcherIterator( private val blockManager: BlockManager, val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])] -) extends Iterator[(String, Option[Iterator[Any]])] with Logging with BlockFetchTracker { +) extends BlockFetcherIterator { import blockManager._ @@ -936,21 +940,9 @@ class BlockFetcherIterator( val localBlockIds = new ArrayBuffer[String]() val remoteBlockIds = new HashSet[String]() - // A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize - // the block (since we want all deserializaton to happen in the calling thread); can also - // represent a fetch failure if size == -1. - class FetchResult(val blockId: String, val size: Long, val deserialize: () => Iterator[Any]) { - def failed: Boolean = size == -1 - } - // A queue to hold our results. val results = new LinkedBlockingQueue[FetchResult] - // A request to fetch one or more blocks, complete with their sizes - class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) { - val size = blocks.map(_._2).sum - } - // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that // the number of bytes in flight is limited to maxBytesInFlight val fetchRequests = new Queue[FetchRequest] @@ -1072,7 +1064,6 @@ class BlockFetcherIterator( } - initialize() //an iterator that will read fetched blocks off the queue as they arrive. var resultsGotten = 0 @@ -1107,7 +1098,7 @@ class BlockFetcherIterator( class NettyBlockFetcherIterator( blockManager: BlockManager, blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])] -) extends BlockFetcherIterator(blockManager,blocksByAddress) { +) extends BasicBlockFetcherIterator(blockManager,blocksByAddress) { import blockManager._ @@ -1129,7 +1120,7 @@ class NettyBlockFetcherIterator( } } catch { case x: InterruptedException => logInfo("Copier Interrupted") - case _ => throw new SparkException("Exception Throw in Shuffle Copier") + //case _ => throw new SparkException("Exception Throw in Shuffle Copier") } } } @@ -1232,3 +1223,13 @@ class NettyBlockFetcherIterator( } } + def apply(t: String, + blockManager: BlockManager, + blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])]): BlockFetcherIterator = { + val iter = if (t == "netty") { new NettyBlockFetcherIterator(blockManager,blocksByAddress) } + else { new BasicBlockFetcherIterator(blockManager,blocksByAddress) } + iter.initialize + iter + } + +} diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index d702bb23e0..cc5bf29a32 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -39,7 +39,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) addShutdownHook() if(useNetty){ - startShuffleBlockSender() + startShuffleBlockSender() } override def getSize(blockId: String): Long = { @@ -229,7 +229,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) case e: Exception => { logError("Error running ShuffleBlockSender ", e) if (shuffleSender != null) { - shuffleSender.stop + shuffleSender.stop shuffleSender = null } } -- cgit v1.2.3 From ab0f834dbb509d323577572691293b74368a9d86 Mon Sep 17 00:00:00 2001 From: seanm Date: Tue, 16 Apr 2013 11:57:05 -0600 Subject: adding spark.streaming.blockInterval property --- docs/configuration.md | 7 +++++++ .../main/scala/spark/streaming/dstream/NetworkInputDStream.scala | 2 +- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 04eb6daaa5..55f1962b18 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -253,6 +253,13 @@ Apart from these, the following properties are also available, and may be useful applications). Note that any RDD that persists in memory for more than this duration will be cleared as well. + + spark.streaming.blockInterval + 200 + + Duration (milliseconds) of how long to batch new objects coming from network receivers. + + diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala index 7385474963..26805e9621 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala @@ -198,7 +198,7 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log case class Block(id: String, iterator: Iterator[T], metadata: Any = null) val clock = new SystemClock() - val blockInterval = 200L + val blockInterval = System.getProperty("spark.streaming.blockInterval", "200").toLong val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer) val blockStorageLevel = storageLevel val blocksForPushing = new ArrayBlockingQueue[Block](1000) -- cgit v1.2.3 From 7e56e99573b4cf161293e648aeb159375c9c0fcb Mon Sep 17 00:00:00 2001 From: seanm Date: Sun, 24 Mar 2013 13:40:19 -0600 Subject: Surfacing decoders on KafkaInputDStream --- .../spark/streaming/examples/KafkaWordCount.scala | 2 +- .../scala/spark/streaming/StreamingContext.scala | 11 ++++---- .../streaming/api/java/JavaStreamingContext.scala | 33 ++++++++++++++++------ .../streaming/dstream/KafkaInputDStream.scala | 17 ++++++----- .../test/java/spark/streaming/JavaAPISuite.java | 6 ++-- 5 files changed, 44 insertions(+), 25 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala index 9b135a5c54..e0c3555f21 100644 --- a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala @@ -37,7 +37,7 @@ object KafkaWordCount { ssc.checkpoint("checkpoint") val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap - val lines = ssc.kafkaStream[String](zkQuorum, group, topicpMap) + val lines = ssc.kafkaStream(zkQuorum, group, topicpMap) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1l)).reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2) wordCounts.print() diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index bb7f216ca7..2c6326943d 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.Path import java.util.UUID import twitter4j.Status + /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic * information (such as, cluster URL and job name) to internally create a SparkContext, it provides @@ -207,14 +208,14 @@ class StreamingContext private ( * @param storageLevel Storage level to use for storing the received objects * (default: StorageLevel.MEMORY_AND_DISK_SER_2) */ - def kafkaStream[T: ClassManifest]( + def kafkaStream( zkQuorum: String, groupId: String, topics: Map[String, Int], storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2 - ): DStream[T] = { + ): DStream[String] = { val kafkaParams = Map[String, String]("zk.connect" -> zkQuorum, "groupid" -> groupId, "zk.connectiontimeout.ms" -> "10000"); - kafkaStream[T](kafkaParams, topics, storageLevel) + kafkaStream[String, kafka.serializer.StringDecoder](kafkaParams, topics, storageLevel) } /** @@ -224,12 +225,12 @@ class StreamingContext private ( * in its own thread. * @param storageLevel Storage level to use for storing the received objects */ - def kafkaStream[T: ClassManifest]( + def kafkaStream[T: ClassManifest, D <: kafka.serializer.Decoder[_]: Manifest]( kafkaParams: Map[String, String], topics: Map[String, Int], storageLevel: StorageLevel ): DStream[T] = { - val inputStream = new KafkaInputDStream[T](this, kafkaParams, topics, storageLevel) + val inputStream = new KafkaInputDStream[T, D](this, kafkaParams, topics, storageLevel) registerInputStream(inputStream) inputStream } diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index 7a8864614c..13427873ff 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -68,33 +68,50 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. */ - def kafkaStream[T]( + def kafkaStream( zkQuorum: String, groupId: String, topics: JMap[String, JInt]) - : JavaDStream[T] = { - implicit val cmt: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] - ssc.kafkaStream[T](zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) + : JavaDStream[String] = { + implicit val cmt: ClassManifest[String] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[String]] + ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), StorageLevel.MEMORY_ONLY_SER_2) } /** * Create an input stream that pulls messages form a Kafka Broker. - * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * @param storageLevel RDD storage level. Defaults to memory-only + * in its own thread. + */ + def kafkaStream( + zkQuorum: String, + groupId: String, + topics: JMap[String, JInt], + storageLevel: StorageLevel) + : JavaDStream[String] = { + implicit val cmt: ClassManifest[String] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[String]] + ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) + } + + /** + * Create an input stream that pulls messages form a Kafka Broker. + * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. * @param storageLevel RDD storage level. Defaults to memory-only */ - def kafkaStream[T]( + def kafkaStream[T, D <: kafka.serializer.Decoder[_]: Manifest]( kafkaParams: JMap[String, String], topics: JMap[String, JInt], storageLevel: StorageLevel) : JavaDStream[T] = { implicit val cmt: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] - ssc.kafkaStream[T]( + ssc.kafkaStream[T, D]( kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index 17a5be3420..7bd53fb6dd 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -9,7 +9,7 @@ import java.util.concurrent.Executors import kafka.consumer._ import kafka.message.{Message, MessageSet, MessageAndMetadata} -import kafka.serializer.StringDecoder +import kafka.serializer.Decoder import kafka.utils.{Utils, ZKGroupTopicDirs} import kafka.utils.ZkUtils._ import kafka.utils.ZKStringSerializer @@ -28,7 +28,7 @@ import scala.collection.JavaConversions._ * @param storageLevel RDD storage level. */ private[streaming] -class KafkaInputDStream[T: ClassManifest]( +class KafkaInputDStream[T: ClassManifest, D <: Decoder[_]: Manifest]( @transient ssc_ : StreamingContext, kafkaParams: Map[String, String], topics: Map[String, Int], @@ -37,15 +37,17 @@ class KafkaInputDStream[T: ClassManifest]( def getReceiver(): NetworkReceiver[T] = { - new KafkaReceiver(kafkaParams, topics, storageLevel) + new KafkaReceiver[T, D](kafkaParams, topics, storageLevel) .asInstanceOf[NetworkReceiver[T]] } } private[streaming] -class KafkaReceiver(kafkaParams: Map[String, String], +class KafkaReceiver[T: ClassManifest, D <: Decoder[_]: Manifest]( + kafkaParams: Map[String, String], topics: Map[String, Int], - storageLevel: StorageLevel) extends NetworkReceiver[Any] { + storageLevel: StorageLevel + ) extends NetworkReceiver[Any] { // Handles pushing data into the BlockManager lazy protected val blockGenerator = new BlockGenerator(storageLevel) @@ -82,7 +84,8 @@ class KafkaReceiver(kafkaParams: Map[String, String], } // Create Threads for each Topic/Message Stream we are listening - val topicMessageStreams = consumerConnector.createMessageStreams(topics, new StringDecoder()) + val decoder = manifest[D].erasure.newInstance.asInstanceOf[Decoder[T]] + val topicMessageStreams = consumerConnector.createMessageStreams(topics, decoder) // Start the messages handler for each partition topicMessageStreams.values.foreach { streams => @@ -91,7 +94,7 @@ class KafkaReceiver(kafkaParams: Map[String, String], } // Handles Kafka Messages - private class MessageHandler(stream: KafkaStream[String]) extends Runnable { + private class MessageHandler[T: ClassManifest](stream: KafkaStream[T]) extends Runnable { def run() { logInfo("Starting MessageHandler.") for (msgAndMetadata <- stream) { diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 3bed500f73..61e4c0a207 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -23,7 +23,6 @@ import spark.streaming.api.java.JavaPairDStream; import spark.streaming.api.java.JavaStreamingContext; import spark.streaming.JavaTestUtils; import spark.streaming.JavaCheckpointTestUtils; -import spark.streaming.dstream.KafkaPartitionKey; import spark.streaming.InputStreamsSuite; import java.io.*; @@ -1203,10 +1202,9 @@ public class JavaAPISuite implements Serializable { @Test public void testKafkaStream() { HashMap topics = Maps.newHashMap(); - HashMap offsets = Maps.newHashMap(); JavaDStream test1 = ssc.kafkaStream("localhost:12345", "group", topics); - JavaDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics, offsets); - JavaDStream test3 = ssc.kafkaStream("localhost:12345", "group", topics, offsets, + JavaDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics); + JavaDStream test3 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK()); } -- cgit v1.2.3 From e0603d7e8bfa991dfd5dc43b303c23a47aa70bca Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Thu, 18 Apr 2013 13:13:54 +0800 Subject: refactor the Schedulable interface and add unit test for SchedulingAlgorithm --- core/src/main/scala/spark/SparkContext.scala | 12 +- .../spark/scheduler/cluster/ClusterScheduler.scala | 77 ++++---- .../cluster/FIFOTaskSetQueuesManager.scala | 49 ----- .../cluster/FairTaskSetQueuesManager.scala | 157 ---------------- .../main/scala/spark/scheduler/cluster/Pool.scala | 96 ++++++---- .../spark/scheduler/cluster/Schedulable.scala | 23 ++- .../scheduler/cluster/SchedulableBuilder.scala | 115 ++++++++++++ .../scheduler/cluster/SchedulingAlgorithm.scala | 33 ++-- .../spark/scheduler/cluster/TaskSetManager.scala | 58 ++++-- core/src/test/resources/fairscheduler.xml | 14 ++ .../spark/scheduler/ClusterSchedulerSuite.scala | 207 +++++++++++++++++++++ 11 files changed, 525 insertions(+), 316 deletions(-) delete mode 100644 core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala delete mode 100644 core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala create mode 100644 core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala create mode 100644 core/src/test/resources/fairscheduler.xml create mode 100644 core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 7c96ae637b..5d9a0357ad 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -146,9 +146,7 @@ class SparkContext( case SPARK_REGEX(sparkUrl) => val scheduler = new ClusterScheduler(this) val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName) - val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")). - newInstance().asInstanceOf[TaskSetQueuesManager] - scheduler.initialize(backend, taskSetQueuesManager) + scheduler.initialize(backend) scheduler case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => @@ -167,9 +165,7 @@ class SparkContext( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) val sparkUrl = localCluster.start() val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName) - val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")). - newInstance().asInstanceOf[TaskSetQueuesManager] - scheduler.initialize(backend, taskSetQueuesManager) + scheduler.initialize(backend) backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => { localCluster.stop() } @@ -188,9 +184,7 @@ class SparkContext( } else { new MesosSchedulerBackend(scheduler, this, masterWithoutProtocol, appName) } - val taskSetQueuesManager = Class.forName(System.getProperty("spark.cluster.taskscheduler")). - newInstance().asInstanceOf[TaskSetQueuesManager] - scheduler.initialize(backend, taskSetQueuesManager) + scheduler.initialize(backend) scheduler } } diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 2ddac0ff30..1a300c9e8c 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -61,17 +61,31 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val mapOutputTracker = SparkEnv.get.mapOutputTracker - var taskSetQueuesManager: TaskSetQueuesManager = null + var schedulableBuilder: SchedulableBuilder = null + var rootPool: Pool = null override def setListener(listener: TaskSchedulerListener) { this.listener = listener } - def initialize(context: SchedulerBackend, taskSetQueuesManager: TaskSetQueuesManager) { + def initialize(context: SchedulerBackend) { backend = context - this.taskSetQueuesManager = taskSetQueuesManager + //default scheduler is FIFO + val schedulingMode = System.getProperty("spark.cluster.schedulingmode", "FIFO") + //temporarily set rootPool name to empty + rootPool = new Pool("", SchedulingMode.withName(schedulingMode), 0, 0) + schedulableBuilder = { + schedulingMode match { + case "FIFO" => + new FIFOSchedulableBuilder(rootPool) + case "FAIR" => + new FairSchedulableBuilder(rootPool) + } + } + schedulableBuilder.buildPools() } + def newTaskId(): Long = nextTaskId.getAndIncrement() override def start() { @@ -101,7 +115,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) this.synchronized { val manager = new TaskSetManager(this, taskSet) activeTaskSets(taskSet.id) = manager - taskSetQueuesManager.addTaskSetManager(manager) + schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) taskSetTaskIds(taskSet.id) = new HashSet[Long]() if (hasReceivedTask == false) { @@ -124,26 +138,21 @@ private[spark] class ClusterScheduler(val sc: SparkContext) def taskSetFinished(manager: TaskSetManager) { this.synchronized { activeTaskSets -= manager.taskSet.id - taskSetQueuesManager.removeTaskSetManager(manager) + manager.parent.removeSchedulable(manager) + logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id, manager.parent.name)) taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id) taskIdToExecutorId --= taskSetTaskIds(manager.taskSet.id) taskSetTaskIds.remove(manager.taskSet.id) } } - def taskFinished(manager: TaskSetManager) { - this.synchronized { - taskSetQueuesManager.taskFinished(manager) - } - } - /** * Called by cluster manager to offer resources on slaves. We respond by asking our active task * sets for tasks in order of priority. We fill each node with tasks in a round-robin manner so * that tasks are balanced across the cluster. */ def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = { - synchronized { + synchronized { SparkEnv.set(sc.env) // Mark each slave as alive and remember its hostname for (o <- offers) { @@ -155,27 +164,27 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // Build a list of tasks to assign to each slave val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) val availableCpus = offers.map(o => o.cores).toArray - for (i <- 0 until offers.size){ - var launchedTask = true - val execId = offers(i).executorId - val host = offers(i).hostname - while (availableCpus(i) > 0 && launchedTask){ - launchedTask = false - taskSetQueuesManager.receiveOffer(execId,host,availableCpus(i)) match { - case Some(task) => - tasks(i) += task - val tid = task.taskId - taskIdToTaskSetId(tid) = task.taskSetId - taskSetTaskIds(task.taskSetId) += tid - taskIdToExecutorId(tid) = execId - activeExecutorIds += execId - executorsByHost(host) += execId - availableCpus(i) -= 1 - launchedTask = true - - case None => {} - } + for (i <- 0 until offers.size) { + var launchedTask = true + val execId = offers(i).executorId + val host = offers(i).hostname + while (availableCpus(i) > 0 && launchedTask) { + launchedTask = false + rootPool.receiveOffer(execId,host,availableCpus(i)) match { + case Some(task) => + tasks(i) += task + val tid = task.taskId + taskIdToTaskSetId(tid) = task.taskSetId + taskSetTaskIds(task.taskSetId) += tid + taskIdToExecutorId(tid) = execId + activeExecutorIds += execId + executorsByHost(host) += execId + availableCpus(i) -= 1 + launchedTask = true + + case None => {} } + } } if (tasks.size > 0) { hasLaunchedTask = true @@ -271,7 +280,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) def checkSpeculatableTasks() { var shouldRevive = false synchronized { - shouldRevive = taskSetQueuesManager.checkSpeculatableTasks() + shouldRevive = rootPool.checkSpeculatableTasks() } if (shouldRevive) { backend.reviveOffers() @@ -314,6 +323,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext) executorsByHost -= host } executorIdToHost -= executorId - taskSetQueuesManager.removeExecutor(executorId, host) + rootPool.executorLost(executorId, host) } } diff --git a/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala deleted file mode 100644 index 62d3130341..0000000000 --- a/core/src/main/scala/spark/scheduler/cluster/FIFOTaskSetQueuesManager.scala +++ /dev/null @@ -1,49 +0,0 @@ -package spark.scheduler.cluster - -import scala.collection.mutable.ArrayBuffer - -import spark.Logging - -/** - * A FIFO Implementation of the TaskSetQueuesManager - */ -private[spark] class FIFOTaskSetQueuesManager extends TaskSetQueuesManager with Logging { - - var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] - val tasksetSchedulingAlgorithm = new FIFOSchedulingAlgorithm() - - override def addTaskSetManager(manager: TaskSetManager) { - activeTaskSetsQueue += manager - } - - override def removeTaskSetManager(manager: TaskSetManager) { - activeTaskSetsQueue -= manager - } - - override def taskFinished(manager: TaskSetManager) { - //do nothing - } - - override def removeExecutor(executorId: String, host: String) { - activeTaskSetsQueue.foreach(_.executorLost(executorId, host)) - } - - override def receiveOffer(execId:String, host:String,avaiableCpus:Double):Option[TaskDescription] = { - - for (manager <- activeTaskSetsQueue.sortWith(tasksetSchedulingAlgorithm.comparator)) { - val task = manager.slaveOffer(execId,host,avaiableCpus) - if (task != None) { - return task - } - } - return None - } - - override def checkSpeculatableTasks(): Boolean = { - var shouldRevive = false - for (ts <- activeTaskSetsQueue) { - shouldRevive |= ts.checkSpeculatableTasks() - } - return shouldRevive - } -} diff --git a/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala deleted file mode 100644 index 89b74fbb47..0000000000 --- a/core/src/main/scala/spark/scheduler/cluster/FairTaskSetQueuesManager.scala +++ /dev/null @@ -1,157 +0,0 @@ -package spark.scheduler.cluster - -import java.io.{File, FileInputStream, FileOutputStream} - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet -import scala.util.control.Breaks._ -import scala.xml._ - -import spark.Logging -import spark.scheduler.cluster.SchedulingMode.SchedulingMode - -/** - * A Fair Implementation of the TaskSetQueuesManager - * - * Currently we support minShare,weight for fair scheduler between pools - * Within a pool, it supports FIFO or FS - * Also, currently we could allocate pools dynamically - */ -private[spark] class FairTaskSetQueuesManager extends TaskSetQueuesManager with Logging { - - val schedulerAllocFile = System.getProperty("spark.fairscheduler.allocation.file","unspecified") - val poolNameToPool= new HashMap[String, Pool] - var pools = new ArrayBuffer[Pool] - val poolScheduleAlgorithm = new FairSchedulingAlgorithm() - val POOL_FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.cluster.fair.pool" - val POOL_DEFAULT_POOL_NAME = "default" - val POOL_MINIMUM_SHARES_PROPERTY = "minShares" - val POOL_SCHEDULING_MODE_PROPERTY = "schedulingMode" - val POOL_WEIGHT_PROPERTY = "weight" - val POOL_POOL_NAME_PROPERTY = "@name" - val POOL_POOLS_PROPERTY = "pool" - val POOL_DEFAULT_SCHEDULING_MODE = SchedulingMode.FIFO - val POOL_DEFAULT_MINIMUM_SHARES = 2 - val POOL_DEFAULT_WEIGHT = 1 - - loadPoolProperties() - - override def addTaskSetManager(manager: TaskSetManager) { - var poolName = POOL_DEFAULT_POOL_NAME - if (manager.taskSet.properties != null) { - poolName = manager.taskSet.properties.getProperty(POOL_FAIR_SCHEDULER_PROPERTIES,POOL_DEFAULT_POOL_NAME) - if (!poolNameToPool.contains(poolName)) { - //we will create a new pool that user has configured in app instead of being defined in xml file - val pool = new Pool(poolName,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT) - pools += pool - poolNameToPool(poolName) = pool - logInfo("Create pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format( - poolName,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) - } - } - poolNameToPool(poolName).addTaskSetManager(manager) - logInfo("Added task set " + manager.taskSet.id + " tasks to pool "+poolName) - } - - override def removeTaskSetManager(manager: TaskSetManager) { - var poolName = POOL_DEFAULT_POOL_NAME - if (manager.taskSet.properties != null) { - poolName = manager.taskSet.properties.getProperty(POOL_FAIR_SCHEDULER_PROPERTIES,POOL_DEFAULT_POOL_NAME) - } - logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id,poolName)) - val pool = poolNameToPool(poolName) - pool.removeTaskSetManager(manager) - pool.runningTasks -= manager.runningTasks - } - - override def taskFinished(manager: TaskSetManager) { - var poolName = POOL_DEFAULT_POOL_NAME - if (manager.taskSet.properties != null) { - poolName = manager.taskSet.properties.getProperty(POOL_FAIR_SCHEDULER_PROPERTIES,POOL_DEFAULT_POOL_NAME) - } - val pool = poolNameToPool(poolName) - pool.runningTasks -= 1 - manager.runningTasks -=1 - } - - override def removeExecutor(executorId: String, host: String) { - for (pool <- pools) { - pool.removeExecutor(executorId,host) - } - } - - override def receiveOffer(execId: String,host:String,avaiableCpus:Double):Option[TaskDescription] = { - val sortedPools = pools.sortWith(poolScheduleAlgorithm.comparator) - for (pool <- sortedPools) { - logDebug("poolName:%s,tasksetNum:%d,minShares:%d,runningTasks:%d".format( - pool.poolName,pool.activeTaskSetsQueue.length,pool.minShare,pool.runningTasks)) - } - for (pool <- sortedPools) { - val task = pool.receiveOffer(execId,host,avaiableCpus) - if(task != None) { - pool.runningTasks += 1 - return task - } - } - return None - } - - override def checkSpeculatableTasks(): Boolean = { - var shouldRevive = false - for (pool <- pools) { - shouldRevive |= pool.checkSpeculatableTasks() - } - return shouldRevive - } - - def loadPoolProperties() { - //first check if the file exists - val file = new File(schedulerAllocFile) - if (file.exists()) { - val xml = XML.loadFile(file) - for (poolNode <- (xml \\ POOL_POOLS_PROPERTY)) { - - val poolName = (poolNode \ POOL_POOL_NAME_PROPERTY).text - var schedulingMode = POOL_DEFAULT_SCHEDULING_MODE - var minShares = POOL_DEFAULT_MINIMUM_SHARES - var weight = POOL_DEFAULT_WEIGHT - - val xmlSchedulingMode = (poolNode \ POOL_SCHEDULING_MODE_PROPERTY).text - if (xmlSchedulingMode != "") { - try{ - schedulingMode = SchedulingMode.withName(xmlSchedulingMode) - } - catch{ - case e:Exception => logInfo("Error xml schedulingMode, using default schedulingMode") - } - } - - val xmlMinShares = (poolNode \ POOL_MINIMUM_SHARES_PROPERTY).text - if (xmlMinShares != "") { - minShares = xmlMinShares.toInt - } - - val xmlWeight = (poolNode \ POOL_WEIGHT_PROPERTY).text - if (xmlWeight != "") { - weight = xmlWeight.toInt - } - - val pool = new Pool(poolName,schedulingMode,minShares,weight) - pools += pool - poolNameToPool(poolName) = pool - logInfo("Create new pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format( - poolName,schedulingMode,minShares,weight)) - } - } - - if (!poolNameToPool.contains(POOL_DEFAULT_POOL_NAME)) { - val pool = new Pool(POOL_DEFAULT_POOL_NAME, POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT) - pools += pool - poolNameToPool(POOL_DEFAULT_POOL_NAME) = pool - logInfo("Create default pool with name:%s,schedulingMode:%s,minShares:%d,weight:%d".format( - POOL_DEFAULT_POOL_NAME,POOL_DEFAULT_SCHEDULING_MODE,POOL_DEFAULT_MINIMUM_SHARES,POOL_DEFAULT_WEIGHT)) - } - } - } diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/spark/scheduler/cluster/Pool.scala index e0917ca1ca..d5482f71ad 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Pool.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Pool.scala @@ -1,74 +1,106 @@ package spark.scheduler.cluster import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap import spark.Logging import spark.scheduler.cluster.SchedulingMode.SchedulingMode + /** - * An Schedulable entity that represent collection of TaskSetManager + * An Schedulable entity that represent collection of Pools or TaskSetManagers */ + private[spark] class Pool( val poolName: String, val schedulingMode: SchedulingMode, - initMinShare:Int, - initWeight:Int) + initMinShare: Int, + initWeight: Int) extends Schedulable with Logging { - var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] + var schedulableQueue = new ArrayBuffer[Schedulable] + var schedulableNameToSchedulable = new HashMap[String, Schedulable] var weight = initWeight var minShare = initMinShare var runningTasks = 0 - val priority = 0 - val stageId = 0 + var priority = 0 + var stageId = 0 + var name = poolName + var parent:Schedulable = null var taskSetSchedulingAlgorithm: SchedulingAlgorithm = { - schedulingMode match { + schedulingMode match { case SchedulingMode.FAIR => - val schedule = new FairSchedulingAlgorithm() - schedule + new FairSchedulingAlgorithm() case SchedulingMode.FIFO => - val schedule = new FIFOSchedulingAlgorithm() - schedule + new FIFOSchedulingAlgorithm() } } - def addTaskSetManager(manager:TaskSetManager) { - activeTaskSetsQueue += manager + override def addSchedulable(schedulable: Schedulable) { + schedulableQueue += schedulable + schedulableNameToSchedulable(schedulable.name) = schedulable + schedulable.parent= this } - def removeTaskSetManager(manager:TaskSetManager) { - activeTaskSetsQueue -= manager + override def removeSchedulable(schedulable: Schedulable) { + schedulableQueue -= schedulable + schedulableNameToSchedulable -= schedulable.name } - def removeExecutor(executorId: String, host: String) { - activeTaskSetsQueue.foreach(_.executorLost(executorId,host)) + override def getSchedulableByName(schedulableName: String): Schedulable = { + if (schedulableNameToSchedulable.contains(schedulableName)) { + return schedulableNameToSchedulable(schedulableName) + } + for (schedulable <- schedulableQueue) { + var sched = schedulable.getSchedulableByName(schedulableName) + if (sched != null) { + return sched + } + } + return null } - def checkSpeculatableTasks(): Boolean = { + override def executorLost(executorId: String, host: String) { + schedulableQueue.foreach(_.executorLost(executorId, host)) + } + + override def checkSpeculatableTasks(): Boolean = { var shouldRevive = false - for (ts <- activeTaskSetsQueue) { - shouldRevive |= ts.checkSpeculatableTasks() + for (schedulable <- schedulableQueue) { + shouldRevive |= schedulable.checkSpeculatableTasks() } return shouldRevive } - def receiveOffer(execId:String,host:String,availableCpus:Double):Option[TaskDescription] = { - val sortedActiveTasksSetQueue = activeTaskSetsQueue.sortWith(taskSetSchedulingAlgorithm.comparator) - for (manager <- sortedActiveTasksSetQueue) { - logDebug("poolname:%s,taskSetId:%s,taskNum:%d,minShares:%d,weight:%d,runningTasks:%d".format( - poolName,manager.taskSet.id,manager.numTasks,manager.minShare,manager.weight,manager.runningTasks)) + override def receiveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { + val sortedSchedulableQueue = schedulableQueue.sortWith(taskSetSchedulingAlgorithm.comparator) + for (manager <- sortedSchedulableQueue) { + logInfo("parentName:%s,schedulableName:%s,minShares:%d,weight:%d,runningTasks:%d".format( + manager.parent.name, manager.name, manager.minShare, manager.weight, manager.runningTasks)) } - - for (manager <- sortedActiveTasksSetQueue) { - val task = manager.slaveOffer(execId,host,availableCpus) - if (task != None) { - manager.runningTasks += 1 - return task - } + for (manager <- sortedSchedulableQueue) { + val task = manager.receiveOffer(execId, host, availableCpus) + if (task != None) { + return task + } } return None } + + override def increaseRunningTasks(taskNum: Int) { + runningTasks += taskNum + if (parent != null) { + parent.increaseRunningTasks(taskNum) + } + } + + override def decreaseRunningTasks(taskNum: Int) { + runningTasks -= taskNum + if (parent != null) { + parent.decreaseRunningTasks(taskNum) + } + } } diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala index 8dfc369c03..54e8ae95f9 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala @@ -1,13 +1,26 @@ package spark.scheduler.cluster +import scala.collection.mutable.ArrayBuffer + /** * An interface for schedulable entities. * there are two type of Schedulable entities(Pools and TaskSetManagers) */ private[spark] trait Schedulable { - def weight:Int - def minShare:Int - def runningTasks:Int - def priority:Int - def stageId:Int + var parent: Schedulable + def weight: Int + def minShare: Int + def runningTasks: Int + def priority: Int + def stageId: Int + def name: String + + def increaseRunningTasks(taskNum: Int): Unit + def decreaseRunningTasks(taskNum: Int): Unit + def addSchedulable(schedulable: Schedulable): Unit + def removeSchedulable(schedulable: Schedulable): Unit + def getSchedulableByName(name: String): Schedulable + def executorLost(executorId: String, host: String): Unit + def receiveOffer(execId: String, host: String, avaiableCpus: Double): Option[TaskDescription] + def checkSpeculatableTasks(): Boolean } diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala new file mode 100644 index 0000000000..47a426a45b --- /dev/null +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala @@ -0,0 +1,115 @@ +package spark.scheduler.cluster + +import java.io.{File, FileInputStream, FileOutputStream} + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet +import scala.util.control.Breaks._ +import scala.xml._ + +import spark.Logging +import spark.scheduler.cluster.SchedulingMode.SchedulingMode + +import java.util.Properties + +/** + * An interface to build Schedulable tree + * buildPools: build the tree nodes(pools) + * addTaskSetManager: build the leaf nodes(TaskSetManagers) + */ +private[spark] trait SchedulableBuilder { + def buildPools() + def addTaskSetManager(manager: Schedulable, properties: Properties) +} + +private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) extends SchedulableBuilder with Logging { + + override def buildPools() { + //nothing + } + + override def addTaskSetManager(manager: Schedulable, properties: Properties) { + rootPool.addSchedulable(manager) + } +} + +private[spark] class FairSchedulableBuilder(val rootPool: Pool) extends SchedulableBuilder with Logging { + + val schedulerAllocFile = System.getProperty("spark.fairscheduler.allocation.file","unspecified") + val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.cluster.fair.pool" + val DEFAULT_POOL_NAME = "default" + val MINIMUM_SHARES_PROPERTY = "minShare" + val SCHEDULING_MODE_PROPERTY = "schedulingMode" + val WEIGHT_PROPERTY = "weight" + val POOL_NAME_PROPERTY = "@name" + val POOLS_PROPERTY = "pool" + val DEFAULT_SCHEDULING_MODE = SchedulingMode.FIFO + val DEFAULT_MINIMUM_SHARE = 2 + val DEFAULT_WEIGHT = 1 + + override def buildPools() { + val file = new File(schedulerAllocFile) + if (file.exists()) { + val xml = XML.loadFile(file) + for (poolNode <- (xml \\ POOLS_PROPERTY)) { + + val poolName = (poolNode \ POOL_NAME_PROPERTY).text + var schedulingMode = DEFAULT_SCHEDULING_MODE + var minShare = DEFAULT_MINIMUM_SHARE + var weight = DEFAULT_WEIGHT + + val xmlSchedulingMode = (poolNode \ SCHEDULING_MODE_PROPERTY).text + if (xmlSchedulingMode != "") { + try { + schedulingMode = SchedulingMode.withName(xmlSchedulingMode) + } catch { + case e: Exception => logInfo("Error xml schedulingMode, using default schedulingMode") + } + } + + val xmlMinShare = (poolNode \ MINIMUM_SHARES_PROPERTY).text + if (xmlMinShare != "") { + minShare = xmlMinShare.toInt + } + + val xmlWeight = (poolNode \ WEIGHT_PROPERTY).text + if (xmlWeight != "") { + weight = xmlWeight.toInt + } + + val pool = new Pool(poolName, schedulingMode, minShare, weight) + rootPool.addSchedulable(pool) + logInfo("Create new pool with name:%s,schedulingMode:%s,minShare:%d,weight:%d".format( + poolName, schedulingMode, minShare, weight)) + } + } + + //finally create "default" pool + if (rootPool.getSchedulableByName(DEFAULT_POOL_NAME) == null) { + val pool = new Pool(DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT) + rootPool.addSchedulable(pool) + logInfo("Create default pool with name:%s,schedulingMode:%s,minShare:%d,weight:%d".format( + DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT)) + } +} + + override def addTaskSetManager(manager: Schedulable, properties: Properties) { + var poolName = DEFAULT_POOL_NAME + var parentPool = rootPool.getSchedulableByName(poolName) + if (properties != null) { + poolName = properties.getProperty(FAIR_SCHEDULER_PROPERTIES, DEFAULT_POOL_NAME) + parentPool = rootPool.getSchedulableByName(poolName) + if (parentPool == null) { + //we will create a new pool that user has configured in app instead of being defined in xml file + parentPool = new Pool(poolName,DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT) + rootPool.addSchedulable(parentPool) + logInfo("Create pool with name:%s,schedulingMode:%s,minShare:%d,weight:%d".format( + poolName, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT)) + } + } + parentPool.addSchedulable(manager) + logInfo("Added task set " + manager.name + " tasks to pool "+poolName) + } +} diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala index ac2237a7ef..a5d6285c99 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala @@ -2,11 +2,11 @@ package spark.scheduler.cluster /** * An interface for sort algorithm - * FIFO: FIFO algorithm for TaskSetManagers - * FS: FS algorithm for Pools, and FIFO or FS for TaskSetManagers + * FIFO: FIFO algorithm between TaskSetManagers + * FS: FS algorithm between Pools, and FIFO or FS within Pools */ private[spark] trait SchedulingAlgorithm { - def comparator(s1: Schedulable,s2: Schedulable): Boolean + def comparator(s1: Schedulable, s2: Schedulable): Boolean } private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm { @@ -15,40 +15,41 @@ private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm { val priority2 = s2.priority var res = Math.signum(priority1 - priority2) if (res == 0) { - val stageId1 = s1.stageId - val stageId2 = s2.stageId - res = Math.signum(stageId1 - stageId2) + val stageId1 = s1.stageId + val stageId2 = s2.stageId + res = Math.signum(stageId1 - stageId2) } - if (res < 0) + if (res < 0) { return true - else + } else { return false + } } } private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { - def comparator(s1: Schedulable, s2:Schedulable): Boolean = { + override def comparator(s1: Schedulable, s2: Schedulable): Boolean = { val minShare1 = s1.minShare val minShare2 = s2.minShare val runningTasks1 = s1.runningTasks val runningTasks2 = s2.runningTasks val s1Needy = runningTasks1 < minShare1 val s2Needy = runningTasks2 < minShare2 - val minShareRatio1 = runningTasks1.toDouble / Math.max(minShare1,1.0).toDouble - val minShareRatio2 = runningTasks2.toDouble / Math.max(minShare2,1.0).toDouble + val minShareRatio1 = runningTasks1.toDouble / Math.max(minShare1, 1.0).toDouble + val minShareRatio2 = runningTasks2.toDouble / Math.max(minShare2, 1.0).toDouble val taskToWeightRatio1 = runningTasks1.toDouble / s1.weight.toDouble val taskToWeightRatio2 = runningTasks2.toDouble / s2.weight.toDouble var res:Boolean = true - if (s1Needy && !s2Needy) + if (s1Needy && !s2Needy) { res = true - else if(!s1Needy && s2Needy) + } else if (!s1Needy && s2Needy) { res = false - else if (s1Needy && s2Needy) + } else if (s1Needy && s2Needy) { res = minShareRatio1 <= minShareRatio2 - else + } else { res = taskToWeightRatio1 <= taskToWeightRatio2 - + } return res } } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 7ec2f69da5..baaaa41a37 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -32,8 +32,6 @@ private[spark] class TaskSetManager( // Maximum times a task is allowed to fail before failing the job val MAX_TASK_FAILURES = 4 - val TASKSET_MINIMUM_SHARES = 1 - val TASKSET_WEIGHT = 1 // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble val SPECULATION_MULTIPLIER = System.getProperty("spark.speculation.multiplier", "1.5").toDouble @@ -41,12 +39,6 @@ private[spark] class TaskSetManager( // Serializer for closures and tasks. val ser = SparkEnv.get.closureSerializer.newInstance() - var weight = TASKSET_WEIGHT - var minShare = TASKSET_MINIMUM_SHARES - var runningTasks = 0 - val priority = taskSet.priority - val stageId = taskSet.stageId - val tasks = taskSet.tasks val numTasks = tasks.length val copiesRunning = new Array[Int](numTasks) @@ -55,6 +47,14 @@ private[spark] class TaskSetManager( val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) var tasksFinished = 0 + var weight = 1 + var minShare = 0 + var runningTasks = 0 + var priority = taskSet.priority + var stageId = taskSet.stageId + var name = "TaskSet_"+taskSet.stageId.toString + var parent:Schedulable = null + // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis @@ -198,7 +198,7 @@ private[spark] class TaskSetManager( } // Respond to an offer of a single slave from the scheduler by finding a task - def slaveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { + override def receiveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { val time = System.currentTimeMillis val localOnly = (time - lastPreferredLaunchTime < LOCALITY_WAIT) @@ -230,10 +230,11 @@ private[spark] class TaskSetManager( val serializedTask = Task.serializeWithDependencies( task, sched.sc.addedFiles, sched.sc.addedJars, ser) val timeTaken = System.currentTimeMillis - startTime + increaseRunningTasks(1) logInfo("Serialized task %s:%d as %d bytes in %d ms".format( taskSet.id, index, serializedTask.limit, timeTaken)) val taskName = "task %s:%d".format(taskSet.id, index) - return Some(new TaskDescription(taskId,taskSet.id,execId, taskName, serializedTask)) + return Some(new TaskDescription(taskId, taskSet.id, execId, taskName, serializedTask)) } case _ => } @@ -264,7 +265,7 @@ private[spark] class TaskSetManager( } val index = info.index info.markSuccessful() - sched.taskFinished(this) + decreaseRunningTasks(1) if (!finished(index)) { tasksFinished += 1 logInfo("Finished TID %s in %d ms (progress: %d/%d)".format( @@ -293,7 +294,7 @@ private[spark] class TaskSetManager( } val index = info.index info.markFailed() - sched.taskFinished(this) + decreaseRunningTasks(1) if (!finished(index)) { logInfo("Lost TID %s (task %s:%d)".format(tid, taskSet.id, index)) copiesRunning(index) -= 1 @@ -308,6 +309,7 @@ private[spark] class TaskSetManager( finished(index) = true tasksFinished += 1 sched.taskSetFinished(this) + decreaseRunningTasks(runningTasks) return case ef: ExceptionFailure => @@ -365,10 +367,38 @@ private[spark] class TaskSetManager( causeOfFailure = message // TODO: Kill running tasks if we were not terminated due to a Mesos error sched.listener.taskSetFailed(taskSet, message) + decreaseRunningTasks(runningTasks) sched.taskSetFinished(this) } - def executorLost(execId: String, hostname: String) { + override def increaseRunningTasks(taskNum: Int) { + runningTasks += taskNum + if (parent != null) { + parent.increaseRunningTasks(taskNum) + } + } + + override def decreaseRunningTasks(taskNum: Int) { + runningTasks -= taskNum + if (parent != null) { + parent.decreaseRunningTasks(taskNum) + } + } + + //TODO: for now we just find Pool not TaskSetManager, we can extend this function in future if needed + override def getSchedulableByName(name: String): Schedulable = { + return null + } + + override def addSchedulable(schedulable:Schedulable) { + //nothing + } + + override def removeSchedulable(schedulable:Schedulable) { + //nothing + } + + override def executorLost(execId: String, hostname: String) { logInfo("Re-queueing tasks for " + execId + " from TaskSet " + taskSet.id) val newHostsAlive = sched.hostsAlive // If some task has preferred locations only on hostname, and there are no more executors there, @@ -409,7 +439,7 @@ private[spark] class TaskSetManager( * TODO: To make this scale to large jobs, we need to maintain a list of running tasks, so that * we don't scan the whole task set. It might also help to make this sorted by launch time. */ - def checkSpeculatableTasks(): Boolean = { + override def checkSpeculatableTasks(): Boolean = { // Can't speculate if we only have one task, or if all tasks have finished. if (numTasks == 1 || tasksFinished == numTasks) { return false diff --git a/core/src/test/resources/fairscheduler.xml b/core/src/test/resources/fairscheduler.xml new file mode 100644 index 0000000000..5a688b0ebb --- /dev/null +++ b/core/src/test/resources/fairscheduler.xml @@ -0,0 +1,14 @@ + + + 2 + 1 + FIFO + + + 3 + 1 + FIFO + + + + diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala new file mode 100644 index 0000000000..2eda48196b --- /dev/null +++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala @@ -0,0 +1,207 @@ +package spark.scheduler + +import org.scalatest.FunSuite +import org.scalatest.BeforeAndAfter + +import spark._ +import spark.scheduler._ +import spark.scheduler.cluster._ + +import java.util.Properties + +class DummyTaskSetManager( + initPriority: Int, + initStageId: Int, + initNumTasks: Int) + extends Schedulable { + + var parent: Schedulable = null + var weight = 1 + var minShare = 2 + var runningTasks = 0 + var priority = initPriority + var stageId = initStageId + var name = "TaskSet_"+stageId + var numTasks = initNumTasks + var tasksFinished = 0 + + def increaseRunningTasks(taskNum: Int) { + runningTasks += taskNum + if (parent != null) { + parent.increaseRunningTasks(taskNum) + } + } + + def decreaseRunningTasks(taskNum: Int) { + runningTasks -= taskNum + if (parent != null) { + parent.decreaseRunningTasks(taskNum) + } + } + + def addSchedulable(schedulable: Schedulable) { + } + + def removeSchedulable(schedulable: Schedulable) { + } + + def getSchedulableByName(name: String): Schedulable = { + return null + } + + def executorLost(executorId: String, host: String): Unit = { + } + + def receiveOffer(execId: String, host: String, avaiableCpus: Double): Option[TaskDescription] = { + if (tasksFinished + runningTasks < numTasks) { + increaseRunningTasks(1) + return Some(new TaskDescription(0, stageId.toString, execId, "task 0:0", null)) + } + return None + } + + def checkSpeculatableTasks(): Boolean = { + return true + } + + def taskFinished() { + decreaseRunningTasks(1) + tasksFinished +=1 + if (tasksFinished == numTasks) { + parent.removeSchedulable(this) + } + } + + def abort() { + decreaseRunningTasks(runningTasks) + parent.removeSchedulable(this) + } +} + +class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { + + def receiveOffer(rootPool: Pool) : Option[TaskDescription] = { + rootPool.receiveOffer("execId_1", "hostname_1", 1) + } + + def checkTaskSetId(rootPool: Pool, expectedTaskSetId: Int) { + receiveOffer(rootPool) match { + case Some(task) => + assert(task.taskSetId.toInt === expectedTaskSetId) + case _ => + } + } + + test("FIFO Scheduler Test") { + val rootPool = new Pool("", SchedulingMode.FIFO, 0, 0) + val schedulableBuilder = new FIFOSchedulableBuilder(rootPool) + schedulableBuilder.buildPools() + + val taskSetManager0 = new DummyTaskSetManager(0, 0, 2) + val taskSetManager1 = new DummyTaskSetManager(0, 1, 2) + val taskSetManager2 = new DummyTaskSetManager(0, 2, 2) + schedulableBuilder.addTaskSetManager(taskSetManager0, null) + schedulableBuilder.addTaskSetManager(taskSetManager1, null) + schedulableBuilder.addTaskSetManager(taskSetManager2, null) + + checkTaskSetId(rootPool, 0) + receiveOffer(rootPool) + checkTaskSetId(rootPool, 1) + receiveOffer(rootPool) + taskSetManager1.abort() + checkTaskSetId(rootPool, 2) + } + + test("Fair Scheduler Test") { + val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() + System.setProperty("spark.fairscheduler.allocation.file", xmlPath) + val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) + val schedulableBuilder = new FairSchedulableBuilder(rootPool) + schedulableBuilder.buildPools() + + assert(rootPool.getSchedulableByName("default") != null) + assert(rootPool.getSchedulableByName("1") != null) + assert(rootPool.getSchedulableByName("2") != null) + assert(rootPool.getSchedulableByName("3") != null) + assert(rootPool.getSchedulableByName("1").minShare === 2) + assert(rootPool.getSchedulableByName("1").weight === 1) + assert(rootPool.getSchedulableByName("2").minShare === 3) + assert(rootPool.getSchedulableByName("2").weight === 1) + assert(rootPool.getSchedulableByName("3").minShare === 2) + assert(rootPool.getSchedulableByName("3").weight === 1) + + val properties1 = new Properties() + properties1.setProperty("spark.scheduler.cluster.fair.pool","1") + val properties2 = new Properties() + properties2.setProperty("spark.scheduler.cluster.fair.pool","2") + + val taskSetManager10 = new DummyTaskSetManager(1, 0, 1) + val taskSetManager11 = new DummyTaskSetManager(1, 1, 1) + val taskSetManager12 = new DummyTaskSetManager(1, 2, 2) + schedulableBuilder.addTaskSetManager(taskSetManager10, properties1) + schedulableBuilder.addTaskSetManager(taskSetManager11, properties1) + schedulableBuilder.addTaskSetManager(taskSetManager12, properties1) + + val taskSetManager23 = new DummyTaskSetManager(2, 3, 2) + val taskSetManager24 = new DummyTaskSetManager(2, 4, 2) + schedulableBuilder.addTaskSetManager(taskSetManager23, properties2) + schedulableBuilder.addTaskSetManager(taskSetManager24, properties2) + + checkTaskSetId(rootPool, 0) + checkTaskSetId(rootPool, 3) + checkTaskSetId(rootPool, 3) + checkTaskSetId(rootPool, 1) + checkTaskSetId(rootPool, 4) + checkTaskSetId(rootPool, 2) + checkTaskSetId(rootPool, 2) + checkTaskSetId(rootPool, 4) + + taskSetManager12.taskFinished() + assert(rootPool.getSchedulableByName("1").runningTasks === 3) + taskSetManager24.abort() + assert(rootPool.getSchedulableByName("2").runningTasks === 2) + } + + test("Nested Pool Test") { + val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) + val pool0 = new Pool("0", SchedulingMode.FAIR, 3, 1) + val pool1 = new Pool("1", SchedulingMode.FAIR, 4, 1) + rootPool.addSchedulable(pool0) + rootPool.addSchedulable(pool1) + + val pool00 = new Pool("00", SchedulingMode.FAIR, 2, 2) + val pool01 = new Pool("01", SchedulingMode.FAIR, 1, 1) + pool0.addSchedulable(pool00) + pool0.addSchedulable(pool01) + + val pool10 = new Pool("10", SchedulingMode.FAIR, 2, 2) + val pool11 = new Pool("11", SchedulingMode.FAIR, 2, 1) + pool1.addSchedulable(pool10) + pool1.addSchedulable(pool11) + + val taskSetManager000 = new DummyTaskSetManager(0, 0, 5) + val taskSetManager001 = new DummyTaskSetManager(0, 1, 5) + pool00.addSchedulable(taskSetManager000) + pool00.addSchedulable(taskSetManager001) + + val taskSetManager010 = new DummyTaskSetManager(1, 2, 5) + val taskSetManager011 = new DummyTaskSetManager(1, 3, 5) + pool01.addSchedulable(taskSetManager010) + pool01.addSchedulable(taskSetManager011) + + val taskSetManager100 = new DummyTaskSetManager(2, 4, 5) + val taskSetManager101 = new DummyTaskSetManager(2, 5, 5) + pool10.addSchedulable(taskSetManager100) + pool10.addSchedulable(taskSetManager101) + + val taskSetManager110 = new DummyTaskSetManager(3, 6, 5) + val taskSetManager111 = new DummyTaskSetManager(3, 7, 5) + pool11.addSchedulable(taskSetManager110) + pool11.addSchedulable(taskSetManager111) + + checkTaskSetId(rootPool, 0) + checkTaskSetId(rootPool, 4) + checkTaskSetId(rootPool, 6) + checkTaskSetId(rootPool, 2) + } +} -- cgit v1.2.3 From 8436bd5d4a96480ac1871330a28d9d712e64959d Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Fri, 19 Apr 2013 02:17:22 +0800 Subject: remove TaskSetQueueManager and update code style --- core/src/main/scala/spark/SparkContext.scala | 10 ++-------- .../main/scala/spark/scheduler/DAGSchedulerEvent.scala | 1 - .../spark/scheduler/cluster/SchedulableBuilder.scala | 14 +++++++------- .../spark/scheduler/cluster/TaskSetQueuesManager.scala | 16 ---------------- 4 files changed, 9 insertions(+), 32 deletions(-) delete mode 100644 core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 5d9a0357ad..eef25ef588 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -39,7 +39,7 @@ import spark.partial.PartialResult import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD} import spark.scheduler._ import spark.scheduler.local.LocalScheduler -import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler, TaskSetQueuesManager} +import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler} import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import spark.storage.BlockManagerUI import spark.util.{MetadataCleaner, TimeStampedHashMap} @@ -75,11 +75,6 @@ class SparkContext( System.setProperty("spark.driver.port", "0") } - //Set the default task scheduler - if (System.getProperty("spark.cluster.taskscheduler") == null) { - System.setProperty("spark.cluster.taskscheduler", "spark.scheduler.cluster.FIFOTaskSetQueuesManager") - } - private val isLocal = (master == "local" || master.startsWith("local[")) // Create the Spark execution environment (cache, map output tracker, etc) @@ -599,8 +594,7 @@ class SparkContext( val callSite = Utils.getSparkCallSite logInfo("Starting job: " + callSite) val start = System.nanoTime - val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal, resultHandler - ,localProperties.value) + val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal, resultHandler, localProperties.value) logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s") rdd.doCheckpoint() result diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index 11fec568c6..303c211e2a 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -1,6 +1,5 @@ package spark.scheduler - import java.util.Properties import spark.scheduler.cluster.TaskInfo diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala index 47a426a45b..18cc15c2a5 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala @@ -86,14 +86,14 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool) extends Schedula } } - //finally create "default" pool - if (rootPool.getSchedulableByName(DEFAULT_POOL_NAME) == null) { - val pool = new Pool(DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT) - rootPool.addSchedulable(pool) - logInfo("Create default pool with name:%s,schedulingMode:%s,minShare:%d,weight:%d".format( - DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT)) + //finally create "default" pool + if (rootPool.getSchedulableByName(DEFAULT_POOL_NAME) == null) { + val pool = new Pool(DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT) + rootPool.addSchedulable(pool) + logInfo("Create default pool with name:%s,schedulingMode:%s,minShare:%d,weight:%d".format( + DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT)) + } } -} override def addTaskSetManager(manager: Schedulable, properties: Properties) { var poolName = DEFAULT_POOL_NAME diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala deleted file mode 100644 index 86971d47e6..0000000000 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetQueuesManager.scala +++ /dev/null @@ -1,16 +0,0 @@ -package spark.scheduler.cluster - -import scala.collection.mutable.ArrayBuffer - -/** - * An interface for managing TaskSet queue/s that allows plugging different policy for - * offering tasks to resources - */ -private[spark] trait TaskSetQueuesManager { - def addTaskSetManager(manager: TaskSetManager): Unit - def removeTaskSetManager(manager: TaskSetManager): Unit - def taskFinished(manager: TaskSetManager): Unit - def removeExecutor(executorId: String, host: String): Unit - def receiveOffer(execId: String, host:String, avaiableCpus:Double):Option[TaskDescription] - def checkSpeculatableTasks(): Boolean -} -- cgit v1.2.3 From b2a3f24dde7a69587a5fea50d3e1e4e8f02a2dc3 Mon Sep 17 00:00:00 2001 From: koeninger Date: Sun, 21 Apr 2013 00:29:37 -0500 Subject: first attempt at an RDD to pull data from JDBC sources --- core/src/main/scala/spark/rdd/JdbcRDD.scala | 79 +++++++++++++++++++++++++++++ 1 file changed, 79 insertions(+) create mode 100644 core/src/main/scala/spark/rdd/JdbcRDD.scala diff --git a/core/src/main/scala/spark/rdd/JdbcRDD.scala b/core/src/main/scala/spark/rdd/JdbcRDD.scala new file mode 100644 index 0000000000..c8a5d76012 --- /dev/null +++ b/core/src/main/scala/spark/rdd/JdbcRDD.scala @@ -0,0 +1,79 @@ +package spark.rdd + +import java.sql.{Connection, ResultSet} + +import spark.{Logging, Partition, RDD, SparkContext, TaskContext} +import spark.util.NextIterator + +/** + An RDD that executes an SQL query on a JDBC connection and reads results. + @param getConnection a function that returns an open Connection. + The RDD takes care of closing the connection. + @param sql the text of the query. + The query must contain two ? placeholders for parameters used to partition the results. + E.g. "select title, author from books where ? <= id and id <= ?" + @param lowerBound the minimum value of the first placeholder + @param upperBound the maximum value of the second placeholder + The lower and upper bounds are inclusive. + @param numPartitions the amount of parallelism. + Given a lowerBound of 1, an upperBound of 20, and a numPartitions of 2, + the query would be executed twice, once with (1, 10) and once with (11, 20) + @param mapRow a function from a ResultSet to a single row of the desired result type(s). + This should only call getInt, getString, etc; the RDD takes care of calling next. + The default maps a ResultSet to an array of Object. +*/ +class JdbcRDD[T: ClassManifest]( + sc: SparkContext, + getConnection: () => Connection, + sql: String, + lowerBound: Long, + upperBound: Long, + numPartitions: Int, + mapRow: (ResultSet) => T = JdbcRDD.resultSetToObjectArray) + extends RDD[T](sc, Nil) with Logging { + + override def getPartitions: Array[Partition] = + ParallelCollectionRDD.slice(lowerBound to upperBound, numPartitions). + filter(! _.isEmpty). + zipWithIndex. + map(x => new JdbcPartition(x._2, x._1.head, x._1.last)). + toArray + + override def compute(thePart: Partition, context: TaskContext) = new NextIterator[T] { + val part = thePart.asInstanceOf[JdbcPartition] + val conn = getConnection() + context.addOnCompleteCallback{ () => closeIfNeeded() } + val stmt = conn.prepareStatement(sql) + stmt.setLong(1, part.lower) + stmt.setLong(2, part.upper) + val rs = stmt.executeQuery() + + override def getNext: T = { + if (rs.next()) { + mapRow(rs) + } else { + finished = true + null.asInstanceOf[T] + } + } + + override def close() { + try { + logInfo("closing connection") + conn.close() + } catch { + case e: Exception => logWarning("Exception closing connection", e) + } + } + } + +} + +private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { + override def index = idx +} + +object JdbcRDD { + val resultSetToObjectArray = (rs: ResultSet) => + Array.tabulate[Object](rs.getMetaData.getColumnCount)(i => rs.getObject(i + 1)) +} -- cgit v1.2.3 From 0dc1e2d60f89f07f54e0985d37cdcd32ad388f6a Mon Sep 17 00:00:00 2001 From: unknown Date: Mon, 22 Apr 2013 09:22:45 -0600 Subject: Examaple of cumulative counting using updateStateByKey --- ...etworkWordCumulativeCountUpdateStateByKey.scala | 63 ++++++++++++++++++++++ 1 file changed, 63 insertions(+) create mode 100644 examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala b/examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala new file mode 100644 index 0000000000..db62246387 --- /dev/null +++ b/examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala @@ -0,0 +1,63 @@ +package spark.streaming.examples + +import spark.streaming._ +import spark.streaming.StreamingContext._ + +/** + * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + * Usage: NetworkWordCumulativeCountUpdateStateByKey + * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. + * and describe the TCP server that Spark Streaming would connect to receive data. + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ ./run spark.streaming.examples.NetworkWordCumulativeCountUpdateStateByKey local[2] localhost 9999` + */ +object NetworkWordCumulativeCountUpdateStateByKey { + private def className[A](a: A)(implicit m: Manifest[A]) = m.toString + + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println("Usage: NetworkWordCountUpdateStateByKey \n" + + "In local mode, should be 'local[n]' with n > 1") + System.exit(1) + } + + val updateFunc = (values: Seq[Int], state: Option[Int]) => { + val currentCount = values.foldLeft(0)(_ + _) + //println("currentCount: " + currentCount) + + val previousCount = state.getOrElse(0) + //println("previousCount: " + previousCount) + + val cumulative = Some(currentCount + previousCount) + //println("Cumulative: " + cumulative) + + cumulative + } + + // Create the context with a 10 second batch size + val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", Seconds(10), + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + ssc.checkpoint(".") + + // Create a NetworkInputDStream on target ip:port and count the + // words in input stream of \n delimited test (eg. generated by 'nc') + val lines = ssc.socketTextStream(args(1), args(2).toInt) + val words = lines.flatMap(_.split(" ")) + val wordDstream = words.map(x => (x, 1)) + + // Update the cumulative count using updateStateByKey + // This will give a Dstream made of state (which is the cumulative count of the words) + val stateDstream = wordDstream.updateStateByKey[Int](updateFunc) + + stateDstream.foreach(rdd => { + rdd.foreach(rddVal => { + println("Current Count: " + rddVal) + }) + }) + + ssc.start() + } +} -- cgit v1.2.3 From dfac0aa5c2e5f46955b008b1e8d9ee5d8069efa5 Mon Sep 17 00:00:00 2001 From: koeninger Date: Mon, 22 Apr 2013 21:12:52 -0500 Subject: prevent mysql driver from pulling entire resultset into memory. explicitly close resultset and statement. --- core/src/main/scala/spark/rdd/JdbcRDD.scala | 25 ++++++++++++++++++++----- 1 file changed, 20 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/rdd/JdbcRDD.scala b/core/src/main/scala/spark/rdd/JdbcRDD.scala index c8a5d76012..4c3054465c 100644 --- a/core/src/main/scala/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/spark/rdd/JdbcRDD.scala @@ -15,7 +15,7 @@ import spark.util.NextIterator @param lowerBound the minimum value of the first placeholder @param upperBound the maximum value of the second placeholder The lower and upper bounds are inclusive. - @param numPartitions the amount of parallelism. + @param numPartitions the number of partitions. Given a lowerBound of 1, an upperBound of 20, and a numPartitions of 2, the query would be executed twice, once with (1, 10) and once with (11, 20) @param mapRow a function from a ResultSet to a single row of the desired result type(s). @@ -40,10 +40,15 @@ class JdbcRDD[T: ClassManifest]( toArray override def compute(thePart: Partition, context: TaskContext) = new NextIterator[T] { + context.addOnCompleteCallback{ () => closeIfNeeded() } val part = thePart.asInstanceOf[JdbcPartition] val conn = getConnection() - context.addOnCompleteCallback{ () => closeIfNeeded() } - val stmt = conn.prepareStatement(sql) + val stmt = conn.prepareStatement(sql, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY) + // force mysql driver to stream rather than pull entire resultset into memory + if (conn.getMetaData.getURL.matches("jdbc:mysql:.*")) { + stmt.setFetchSize(Integer.MIN_VALUE) + logInfo("statement fetch size set to: " + stmt.getFetchSize + " to force MySQL streaming ") + } stmt.setLong(1, part.lower) stmt.setLong(2, part.upper) val rs = stmt.executeQuery() @@ -59,8 +64,18 @@ class JdbcRDD[T: ClassManifest]( override def close() { try { - logInfo("closing connection") - conn.close() + if (null != rs && ! rs.isClosed()) rs.close() + } catch { + case e: Exception => logWarning("Exception closing resultset", e) + } + try { + if (null != stmt && ! stmt.isClosed()) stmt.close() + } catch { + case e: Exception => logWarning("Exception closing statement", e) + } + try { + if (null != conn && ! stmt.isClosed()) conn.close() + logInfo("closed connection") } catch { case e: Exception => logWarning("Exception closing connection", e) } -- cgit v1.2.3 From 31ce6c66d6f29302d0f0f2c70e494fad0ba71e4d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Apr 2013 17:48:59 -0700 Subject: Added a BlockObjectWriter interface in block manager so ShuffleMapTask doesn't need to build up an array buffer for each shuffle bucket. --- core/src/main/scala/spark/SparkEnv.scala | 18 ++++++--- .../scala/spark/scheduler/ShuffleMapTask.scala | 30 +++++++++------ .../main/scala/spark/storage/BlockException.scala | 5 +++ .../main/scala/spark/storage/BlockManager.scala | 36 ++++++++++++++---- .../scala/spark/storage/BlockObjectWriter.scala | 27 ++++++++++++++ core/src/main/scala/spark/storage/DiskStore.scala | 43 +++++++++++++++++++--- .../main/scala/spark/storage/ThreadingTest.scala | 2 +- 7 files changed, 129 insertions(+), 32 deletions(-) create mode 100644 core/src/main/scala/spark/storage/BlockException.scala create mode 100644 core/src/main/scala/spark/storage/BlockObjectWriter.scala diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 7157fd2688..c10bedb8f6 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -22,6 +22,7 @@ class SparkEnv ( val actorSystem: ActorSystem, val serializer: Serializer, val closureSerializer: Serializer, + val shuffleSerializer: Serializer, val cacheManager: CacheManager, val mapOutputTracker: MapOutputTracker, val shuffleFetcher: ShuffleFetcher, @@ -82,7 +83,7 @@ object SparkEnv extends Logging { } val serializer = instantiateClass[Serializer]("spark.serializer", "spark.JavaSerializer") - + def registerOrLookup(name: String, newActor: => Actor): ActorRef = { if (isDriver) { logInfo("Registering " + name) @@ -96,18 +97,22 @@ object SparkEnv extends Logging { } } + val closureSerializer = instantiateClass[Serializer]( + "spark.closure.serializer", "spark.JavaSerializer") + + val shuffleSerializer = instantiateClass[Serializer]( + "spark.shuffle.serializer", "spark.JavaSerializer") + val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", new spark.storage.BlockManagerMasterActor(isLocal))) - val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer) + val blockManager = new BlockManager( + executorId, actorSystem, blockManagerMaster, serializer, shuffleSerializer) val connectionManager = blockManager.connectionManager val broadcastManager = new BroadcastManager(isDriver) - val closureSerializer = instantiateClass[Serializer]( - "spark.closure.serializer", "spark.JavaSerializer") - val cacheManager = new CacheManager(blockManager) // Have to assign trackerActor after initialization as MapOutputTrackerActor @@ -144,6 +149,7 @@ object SparkEnv extends Logging { actorSystem, serializer, closureSerializer, + shuffleSerializer, cacheManager, mapOutputTracker, shuffleFetcher, @@ -153,5 +159,5 @@ object SparkEnv extends Logging { httpFileServer, sparkFilesDir) } - + } diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 36d087a4d0..97b668cd58 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -122,27 +122,33 @@ private[spark] class ShuffleMapTask( val taskContext = new TaskContext(stageId, partition, attemptId) metrics = Some(taskContext.taskMetrics) try { - // Partition the map output. - val buckets = Array.fill(numOutputSplits)(new ArrayBuffer[(Any, Any)]) + // Obtain all the block writers for shuffle blocks. + val blockManager = SparkEnv.get.blockManager + val buckets = Array.tabulate[BlockObjectWriter](numOutputSplits) { bucketId => + val blockId = "shuffle_" + dep.shuffleId + "_" + partition + "_" + bucketId + blockManager.getBlockWriter(blockId) + } + + // Write the map output to its associated buckets. for (elem <- rdd.iterator(split, taskContext)) { val pair = elem.asInstanceOf[(Any, Any)] val bucketId = dep.partitioner.getPartition(pair._1) - buckets(bucketId) += pair + buckets(bucketId).write(pair) } + // Close the bucket writers and get the sizes of each block. val compressedSizes = new Array[Byte](numOutputSplits) - - var totalBytes = 0l - - val blockManager = SparkEnv.get.blockManager - for (i <- 0 until numOutputSplits) { - val blockId = "shuffle_" + dep.shuffleId + "_" + partition + "_" + i - // Get a Scala iterator from Java map - val iter: Iterator[(Any, Any)] = buckets(i).iterator - val size = blockManager.put(blockId, iter, StorageLevel.DISK_ONLY, false) + var i = 0 + var totalBytes = 0L + while (i < numOutputSplits) { + buckets(i).close() + val size = buckets(i).size() totalBytes += size compressedSizes(i) = MapOutputTracker.compressSize(size) + i += 1 } + + // Update shuffle metrics. val shuffleMetrics = new ShuffleWriteMetrics shuffleMetrics.shuffleBytesWritten = totalBytes metrics.get.shuffleWriteMetrics = Some(shuffleMetrics) diff --git a/core/src/main/scala/spark/storage/BlockException.scala b/core/src/main/scala/spark/storage/BlockException.scala new file mode 100644 index 0000000000..f275d476df --- /dev/null +++ b/core/src/main/scala/spark/storage/BlockException.scala @@ -0,0 +1,5 @@ +package spark.storage + +private[spark] +case class BlockException(blockId: String, message: String) extends Exception(message) + diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 210061e972..2f97bad916 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -24,16 +24,13 @@ import spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStam import sun.nio.ch.DirectBuffer -private[spark] -case class BlockException(blockId: String, message: String, ex: Exception = null) -extends Exception(message) - private[spark] class BlockManager( executorId: String, actorSystem: ActorSystem, val master: BlockManagerMaster, val serializer: Serializer, + val shuffleSerializer: Serializer, maxMemory: Long) extends Logging { @@ -78,7 +75,7 @@ class BlockManager( private val blockInfo = new TimeStampedHashMap[String, BlockInfo] private[storage] val memoryStore: BlockStore = new MemoryStore(this, maxMemory) - private[storage] val diskStore: BlockStore = + private[storage] val diskStore: DiskStore = new DiskStore(this, System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir"))) val connectionManager = new ConnectionManager(0) @@ -126,8 +123,17 @@ class BlockManager( * Construct a BlockManager with a memory limit set based on system properties. */ def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, - serializer: Serializer) = { - this(execId, actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties) + serializer: Serializer, shuffleSerializer: Serializer) = { + this(execId, actorSystem, master, serializer, shuffleSerializer, + BlockManager.getMaxMemoryFromSystemProperties) + } + + /** + * Construct a BlockManager with a memory limit set based on system properties. + */ + def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, + serializer: Serializer, maxMemory: Long) = { + this(execId, actorSystem, master, serializer, serializer, maxMemory) } /** @@ -485,6 +491,21 @@ class BlockManager( put(blockId, elements, level, tellMaster) } + /** + * A short circuited method to get a block writer that can write data directly to disk. + * This is currently used for writing shuffle files out. + */ + def getBlockWriter(blockId: String): BlockObjectWriter = { + val writer = diskStore.getBlockWriter(blockId) + writer.registerCloseEventHandler(() => { + // TODO(rxin): This doesn't handle error cases. + val myInfo = new BlockInfo(StorageLevel.DISK_ONLY, false) + blockInfo.put(blockId, myInfo) + myInfo.markReady(writer.size()) + }) + writer + } + /** * Put a new block of values to the block manager. Returns its (estimated) size in bytes. */ @@ -574,7 +595,6 @@ class BlockManager( } logDebug("Put block " + blockId + " locally took " + Utils.getUsedTimeMs(startTimeMs)) - // Replicate block if required if (level.replication > 1) { val remoteStartTime = System.currentTimeMillis diff --git a/core/src/main/scala/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/spark/storage/BlockObjectWriter.scala new file mode 100644 index 0000000000..657a7e9143 --- /dev/null +++ b/core/src/main/scala/spark/storage/BlockObjectWriter.scala @@ -0,0 +1,27 @@ +package spark.storage + +import java.nio.ByteBuffer + + +abstract class BlockObjectWriter(val blockId: String) { + + // TODO(rxin): What if there is an exception when the block is being written out? + + var closeEventHandler: () => Unit = _ + + def registerCloseEventHandler(handler: () => Unit) { + closeEventHandler = handler + } + + def write(value: Any) + + def writeAll(value: Iterator[Any]) { + value.foreach(write) + } + + def close() { + closeEventHandler() + } + + def size(): Long +} diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index ddbf8821ad..493936fdbe 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -1,7 +1,7 @@ package spark.storage import java.nio.ByteBuffer -import java.io.{File, FileOutputStream, RandomAccessFile} +import java.io.{File, FileOutputStream, OutputStream, RandomAccessFile} import java.nio.channels.FileChannel.MapMode import java.util.{Random, Date} import java.text.SimpleDateFormat @@ -10,9 +10,9 @@ import it.unimi.dsi.fastutil.io.FastBufferedOutputStream import scala.collection.mutable.ArrayBuffer +import spark.Utils import spark.executor.ExecutorExitCode -import spark.Utils /** * Stores BlockManager blocks on disk. @@ -20,6 +20,33 @@ import spark.Utils private class DiskStore(blockManager: BlockManager, rootDirs: String) extends BlockStore(blockManager) { + class DiskBlockObjectWriter(blockId: String) extends BlockObjectWriter(blockId) { + + private val f: File = createFile(blockId /*, allowAppendExisting */) + private val bs: OutputStream = blockManager.wrapForCompression(blockId, + new FastBufferedOutputStream(new FileOutputStream(f))) + private val objOut = blockManager.shuffleSerializer.newInstance().serializeStream(bs) + + private var _size: Long = -1L + + override def write(value: Any) { + objOut.writeObject(value) + } + + override def close() { + objOut.close() + bs.close() + super.close() + } + + override def size(): Long = { + if (_size < 0) { + _size = f.length() + } + _size + } + } + val MAX_DIR_CREATION_ATTEMPTS: Int = 10 val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt @@ -31,6 +58,10 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) addShutdownHook() + def getBlockWriter(blockId: String): BlockObjectWriter = { + new DiskBlockObjectWriter(blockId) + } + override def getSize(blockId: String): Long = { getFile(blockId).length() } @@ -65,8 +96,10 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) objOut.writeAll(values.iterator) objOut.close() val length = file.length() + + val timeTaken = System.currentTimeMillis - startTime logDebug("Block %s stored as %s file on disk in %d ms".format( - blockId, Utils.memoryBytesToString(length), (System.currentTimeMillis - startTime))) + blockId, Utils.memoryBytesToString(length), timeTaken)) if (returnValues) { // Return a byte buffer for the contents of the file @@ -106,9 +139,9 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) getFile(blockId).exists() } - private def createFile(blockId: String): File = { + private def createFile(blockId: String, allowAppendExisting: Boolean = false): File = { val file = getFile(blockId) - if (file.exists()) { + if (!allowAppendExisting && file.exists()) { throw new Exception("File for block " + blockId + " already exists on disk: " + file) } file diff --git a/core/src/main/scala/spark/storage/ThreadingTest.scala b/core/src/main/scala/spark/storage/ThreadingTest.scala index 5c406e68cb..3875e7459e 100644 --- a/core/src/main/scala/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/spark/storage/ThreadingTest.scala @@ -78,7 +78,7 @@ private[spark] object ThreadingTest { val blockManagerMaster = new BlockManagerMaster( actorSystem.actorOf(Props(new BlockManagerMasterActor(true)))) val blockManager = new BlockManager( - "", actorSystem, blockManagerMaster, serializer, 1024 * 1024) + "", actorSystem, blockManagerMaster, serializer, serializer, 1024 * 1024) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) producers.foreach(_.start) -- cgit v1.2.3 From aa618ed2a2df209da3f93a025928366959c37d04 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 24 Apr 2013 14:52:49 -0700 Subject: Allow changing the serializer on a per shuffle basis. --- .../scala/spark/BlockStoreShuffleFetcher.scala | 13 ++++-- core/src/main/scala/spark/Dependency.scala | 4 +- core/src/main/scala/spark/PairRDDFunctions.scala | 11 ++--- core/src/main/scala/spark/ShuffleFetcher.scala | 7 ++- core/src/main/scala/spark/SparkEnv.scala | 20 ++++----- core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 13 +++--- core/src/main/scala/spark/rdd/ShuffledRDD.scala | 11 +++-- core/src/main/scala/spark/rdd/SubtractedRDD.scala | 18 +++++--- .../scala/spark/scheduler/ShuffleMapTask.scala | 8 ++-- .../main/scala/spark/serializer/Serializer.scala | 50 +++++++++++++++++++++- .../main/scala/spark/storage/BlockManager.scala | 42 +++++++++--------- .../scala/spark/storage/BlockManagerWorker.scala | 18 ++++---- core/src/main/scala/spark/storage/DiskStore.scala | 18 ++++---- .../main/scala/spark/storage/ThreadingTest.scala | 2 +- 14 files changed, 153 insertions(+), 82 deletions(-) diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index c27ed36406..2156efbd45 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -1,14 +1,19 @@ package spark -import executor.{ShuffleReadMetrics, TaskMetrics} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap +import spark.executor.{ShuffleReadMetrics, TaskMetrics} +import spark.serializer.Serializer import spark.storage.{DelegateBlockFetchTracker, BlockManagerId} -import util.{CompletionIterator, TimedIterator} +import spark.util.{CompletionIterator, TimedIterator} + private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { - override def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics) = { + + override def fetch[K, V]( + shuffleId: Int, reduceId: Int, metrics: TaskMetrics, serializer: Serializer) = { + logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId)) val blockManager = SparkEnv.get.blockManager @@ -48,7 +53,7 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin } } - val blockFetcherItr = blockManager.getMultiple(blocksByAddress) + val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer) val itr = new TimedIterator(blockFetcherItr.flatMap(unpackBlock)) with DelegateBlockFetchTracker itr.setDelegate(blockFetcherItr) CompletionIterator[(K,V), Iterator[(K,V)]](itr, { diff --git a/core/src/main/scala/spark/Dependency.scala b/core/src/main/scala/spark/Dependency.scala index 5eea907322..2af44aa383 100644 --- a/core/src/main/scala/spark/Dependency.scala +++ b/core/src/main/scala/spark/Dependency.scala @@ -25,10 +25,12 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { * @param shuffleId the shuffle id * @param rdd the parent RDD * @param partitioner partitioner used to partition the shuffle output + * @param serializerClass class name of the serializer to use */ class ShuffleDependency[K, V]( @transient rdd: RDD[(K, V)], - val partitioner: Partitioner) + val partitioner: Partitioner, + val serializerClass: String = null) extends Dependency(rdd) { val shuffleId: Int = rdd.context.newShuffleId() diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 07efba9e8d..1b9b9d21d8 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -52,7 +52,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, partitioner: Partitioner, - mapSideCombine: Boolean = true): RDD[(K, C)] = { + mapSideCombine: Boolean = true, + serializerClass: String = null): RDD[(K, C)] = { if (getKeyClass().isArray) { if (mapSideCombine) { throw new SparkException("Cannot use map-side combining with array keys.") @@ -67,13 +68,13 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( self.mapPartitions(aggregator.combineValuesByKey(_), true) } else if (mapSideCombine) { val mapSideCombined = self.mapPartitions(aggregator.combineValuesByKey(_), true) - val partitioned = new ShuffledRDD[K, C](mapSideCombined, partitioner) + val partitioned = new ShuffledRDD[K, C](mapSideCombined, partitioner, serializerClass) partitioned.mapPartitions(aggregator.combineCombinersByKey(_), true) } else { // Don't apply map-side combiner. // A sanity check to make sure mergeCombiners is not defined. assert(mergeCombiners == null) - val values = new ShuffledRDD[K, V](self, partitioner) + val values = new ShuffledRDD[K, V](self, partitioner, serializerClass) values.mapPartitions(aggregator.combineValuesByKey(_), true) } } @@ -469,7 +470,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( /** * Return an RDD with the pairs from `this` whose keys are not in `other`. - * + * * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting * RDD will be <= us. */ @@ -644,7 +645,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * Return an RDD with the keys of each tuple. */ def keys: RDD[K] = self.map(_._1) - + /** * Return an RDD with the values of each tuple. */ diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala index 442e9f0269..49addc0c10 100644 --- a/core/src/main/scala/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/spark/ShuffleFetcher.scala @@ -1,13 +1,16 @@ package spark -import executor.TaskMetrics +import spark.executor.TaskMetrics +import spark.serializer.Serializer + private[spark] abstract class ShuffleFetcher { /** * Fetch the shuffle outputs for a given ShuffleDependency. * @return An iterator over the elements of the fetched shuffle outputs. */ - def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics) : Iterator[(K,V)] + def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics, + serializer: Serializer = Serializer.default): Iterator[(K,V)] /** Stop the fetcher */ def stop() {} diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index c10bedb8f6..8a751fbd6e 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -3,13 +3,14 @@ package spark import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem} import akka.remote.RemoteActorRefProvider -import serializer.Serializer import spark.broadcast.BroadcastManager import spark.storage.BlockManager import spark.storage.BlockManagerMaster import spark.network.ConnectionManager +import spark.serializer.Serializer import spark.util.AkkaUtils + /** * Holds all the runtime environment objects for a running Spark instance (either master or worker), * including the serializer, Akka actor system, block manager, map output tracker, etc. Currently @@ -22,7 +23,6 @@ class SparkEnv ( val actorSystem: ActorSystem, val serializer: Serializer, val closureSerializer: Serializer, - val shuffleSerializer: Serializer, val cacheManager: CacheManager, val mapOutputTracker: MapOutputTracker, val shuffleFetcher: ShuffleFetcher, @@ -82,7 +82,11 @@ object SparkEnv extends Logging { Class.forName(name, true, classLoader).newInstance().asInstanceOf[T] } - val serializer = instantiateClass[Serializer]("spark.serializer", "spark.JavaSerializer") + val serializer = Serializer.setDefault( + System.getProperty("spark.serializer", "spark.JavaSerializer")) + + val closureSerializer = Serializer.get( + System.getProperty("spark.closure.serializer", "spark.JavaSerializer")) def registerOrLookup(name: String, newActor: => Actor): ActorRef = { if (isDriver) { @@ -97,17 +101,10 @@ object SparkEnv extends Logging { } } - val closureSerializer = instantiateClass[Serializer]( - "spark.closure.serializer", "spark.JavaSerializer") - - val shuffleSerializer = instantiateClass[Serializer]( - "spark.shuffle.serializer", "spark.JavaSerializer") - val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", new spark.storage.BlockManagerMasterActor(isLocal))) - val blockManager = new BlockManager( - executorId, actorSystem, blockManagerMaster, serializer, shuffleSerializer) + val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer) val connectionManager = blockManager.connectionManager @@ -149,7 +146,6 @@ object SparkEnv extends Logging { actorSystem, serializer, closureSerializer, - shuffleSerializer, cacheManager, mapOutputTracker, shuffleFetcher, diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index a6235491ca..9e996e9958 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -8,6 +8,7 @@ import scala.collection.mutable.ArrayBuffer import spark.{Aggregator, Logging, Partition, Partitioner, RDD, SparkEnv, TaskContext} import spark.{Dependency, OneToOneDependency, ShuffleDependency} +import spark.serializer.Serializer private[spark] sealed trait CoGroupSplitDep extends Serializable @@ -54,7 +55,8 @@ private[spark] class CoGroupAggregator class CoGroupedRDD[K]( @transient var rdds: Seq[RDD[(K, _)]], part: Partitioner, - val mapSideCombine: Boolean = true) + val mapSideCombine: Boolean = true, + val serializerClass: String = null) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { private val aggr = new CoGroupAggregator @@ -68,9 +70,9 @@ class CoGroupedRDD[K]( logInfo("Adding shuffle dependency with " + rdd) if (mapSideCombine) { val mapSideCombinedRDD = rdd.mapPartitions(aggr.combineValuesByKey(_), true) - new ShuffleDependency[Any, ArrayBuffer[Any]](mapSideCombinedRDD, part) + new ShuffleDependency[Any, ArrayBuffer[Any]](mapSideCombinedRDD, part, serializerClass) } else { - new ShuffleDependency[Any, Any](rdd.asInstanceOf[RDD[(Any, Any)]], part) + new ShuffleDependency[Any, Any](rdd.asInstanceOf[RDD[(Any, Any)]], part, serializerClass) } } } @@ -112,6 +114,7 @@ class CoGroupedRDD[K]( } } + val ser = Serializer.get(serializerClass) for ((dep, depNum) <- split.deps.zipWithIndex) dep match { case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { // Read them from the parent @@ -124,12 +127,12 @@ class CoGroupedRDD[K]( val fetcher = SparkEnv.get.shuffleFetcher if (mapSideCombine) { // With map side combine on, for each key, the shuffle fetcher returns a list of values. - fetcher.fetch[K, Seq[Any]](shuffleId, split.index, context.taskMetrics).foreach { + fetcher.fetch[K, Seq[Any]](shuffleId, split.index, context.taskMetrics, ser).foreach { case (key, values) => getSeq(key)(depNum) ++= values } } else { // With map side combine off, for each key the shuffle fetcher returns a single value. - fetcher.fetch[K, Any](shuffleId, split.index, context.taskMetrics).foreach { + fetcher.fetch[K, Any](shuffleId, split.index, context.taskMetrics, ser).foreach { case (key, value) => getSeq(key)(depNum) += value } } diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala index 4e33b7dd5c..8175e23eff 100644 --- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala @@ -2,6 +2,8 @@ package spark.rdd import spark.{Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext} import spark.SparkContext._ +import spark.serializer.Serializer + private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { override val index = idx @@ -12,13 +14,15 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { * The resulting RDD from a shuffle (e.g. repartitioning of data). * @param prev the parent RDD. * @param part the partitioner used to partition the RDD + * @param serializerClass class name of the serializer to use. * @tparam K the key class. * @tparam V the value class. */ class ShuffledRDD[K, V]( @transient prev: RDD[(K, V)], - part: Partitioner) - extends RDD[(K, V)](prev.context, List(new ShuffleDependency(prev, part))) { + part: Partitioner, + serializerClass: String = null) + extends RDD[(K, V)](prev.context, List(new ShuffleDependency(prev, part, serializerClass))) { override val partitioner = Some(part) @@ -28,6 +32,7 @@ class ShuffledRDD[K, V]( override def compute(split: Partition, context: TaskContext): Iterator[(K, V)] = { val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId - SparkEnv.get.shuffleFetcher.fetch[K, V](shuffledId, split.index, context.taskMetrics) + SparkEnv.get.shuffleFetcher.fetch[K, V]( + shuffledId, split.index, context.taskMetrics, Serializer.get(serializerClass)) } } diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala index 481e03b349..f60c35c38e 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -11,6 +11,7 @@ import spark.Partition import spark.SparkEnv import spark.ShuffleDependency import spark.OneToOneDependency +import spark.serializer.Serializer /** * An optimized version of cogroup for set difference/subtraction. @@ -31,7 +32,9 @@ import spark.OneToOneDependency private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassManifest]( @transient var rdd1: RDD[(K, V)], @transient var rdd2: RDD[(K, W)], - part: Partitioner) extends RDD[(K, V)](rdd1.context, Nil) { + part: Partitioner, + val serializerClass: String = null) + extends RDD[(K, V)](rdd1.context, Nil) { override def getDependencies: Seq[Dependency[_]] = { Seq(rdd1, rdd2).map { rdd => @@ -40,7 +43,7 @@ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassM new OneToOneDependency(rdd) } else { logInfo("Adding shuffle dependency with " + rdd) - new ShuffleDependency(rdd.asInstanceOf[RDD[(K, Any)]], part) + new ShuffleDependency(rdd.asInstanceOf[RDD[(K, Any)]], part, serializerClass) } } } @@ -65,6 +68,7 @@ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassM override def compute(p: Partition, context: TaskContext): Iterator[(K, V)] = { val partition = p.asInstanceOf[CoGroupPartition] + val serializer = Serializer.get(serializerClass) val map = new JHashMap[K, ArrayBuffer[V]] def getSeq(k: K): ArrayBuffer[V] = { val seq = map.get(k) @@ -77,12 +81,16 @@ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassM } } def integrate(dep: CoGroupSplitDep, op: ((K, V)) => Unit) = dep match { - case NarrowCoGroupSplitDep(rdd, _, itsSplit) => + case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { for (t <- rdd.iterator(itsSplit, context)) op(t.asInstanceOf[(K, V)]) - case ShuffleCoGroupSplitDep(shuffleId) => - for (t <- SparkEnv.get.shuffleFetcher.fetch(shuffleId, partition.index, context.taskMetrics)) + } + case ShuffleCoGroupSplitDep(shuffleId) => { + val iter = SparkEnv.get.shuffleFetcher.fetch(shuffleId, partition.index, + context.taskMetrics, serializer) + for (t <- iter) op(t.asInstanceOf[(K, V)]) + } } // the first dep is rdd1; add all values to the map integrate(partition.deps(0), t => getSeq(t._1) += t._2) diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 97b668cd58..d9b26c9db9 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -13,9 +13,11 @@ import com.ning.compress.lzf.LZFInputStream import com.ning.compress.lzf.LZFOutputStream import spark._ -import executor.ShuffleWriteMetrics +import spark.executor.ShuffleWriteMetrics +import spark.serializer.Serializer import spark.storage._ -import util.{TimeStampedHashMap, MetadataCleaner} +import spark.util.{TimeStampedHashMap, MetadataCleaner} + private[spark] object ShuffleMapTask { @@ -126,7 +128,7 @@ private[spark] class ShuffleMapTask( val blockManager = SparkEnv.get.blockManager val buckets = Array.tabulate[BlockObjectWriter](numOutputSplits) { bucketId => val blockId = "shuffle_" + dep.shuffleId + "_" + partition + "_" + bucketId - blockManager.getBlockWriter(blockId) + blockManager.getBlockWriter(blockId, Serializer.get(dep.serializerClass)) } // Write the map output to its associated buckets. diff --git a/core/src/main/scala/spark/serializer/Serializer.scala b/core/src/main/scala/spark/serializer/Serializer.scala index aca86ab6f0..77b1a1a434 100644 --- a/core/src/main/scala/spark/serializer/Serializer.scala +++ b/core/src/main/scala/spark/serializer/Serializer.scala @@ -1,10 +1,14 @@ package spark.serializer -import java.nio.ByteBuffer import java.io.{EOFException, InputStream, OutputStream} +import java.nio.ByteBuffer +import java.util.concurrent.ConcurrentHashMap + import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream + import spark.util.ByteBufferInputStream + /** * A serializer. Because some serialization libraries are not thread safe, this class is used to * create [[spark.serializer.SerializerInstance]] objects that do the actual serialization and are @@ -14,6 +18,48 @@ trait Serializer { def newInstance(): SerializerInstance } + +/** + * A singleton object that can be used to fetch serializer objects based on the serializer + * class name. If a previous instance of the serializer object has been created, the get + * method returns that instead of creating a new one. + */ +object Serializer { + + private val serializers = new ConcurrentHashMap[String, Serializer] + private var _default: Serializer = _ + + def default = _default + + def setDefault(clsName: String): Serializer = { + _default = get(clsName) + _default + } + + def get(clsName: String): Serializer = { + if (clsName == null) { + default + } else { + var serializer = serializers.get(clsName) + if (serializer != null) { + // If the serializer has been created previously, reuse that. + serializer + } else this.synchronized { + // Otherwise, create a new one. But make sure no other thread has attempted + // to create another new one at the same time. + serializer = serializers.get(clsName) + if (serializer == null) { + val clsLoader = Thread.currentThread.getContextClassLoader + serializer = Class.forName(clsName, true, clsLoader).newInstance().asInstanceOf[Serializer] + serializers.put(clsName, serializer) + } + serializer + } + } + } +} + + /** * An instance of a serializer, for use by one thread at a time. */ @@ -45,6 +91,7 @@ trait SerializerInstance { } } + /** * A stream for writing serialized objects. */ @@ -61,6 +108,7 @@ trait SerializationStream { } } + /** * A stream for reading serialized objects. */ diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 2f97bad916..9f7985e2e8 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -29,8 +29,7 @@ class BlockManager( executorId: String, actorSystem: ActorSystem, val master: BlockManagerMaster, - val serializer: Serializer, - val shuffleSerializer: Serializer, + val defaultSerializer: Serializer, maxMemory: Long) extends Logging { @@ -123,17 +122,8 @@ class BlockManager( * Construct a BlockManager with a memory limit set based on system properties. */ def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, - serializer: Serializer, shuffleSerializer: Serializer) = { - this(execId, actorSystem, master, serializer, shuffleSerializer, - BlockManager.getMaxMemoryFromSystemProperties) - } - - /** - * Construct a BlockManager with a memory limit set based on system properties. - */ - def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, - serializer: Serializer, maxMemory: Long) = { - this(execId, actorSystem, master, serializer, serializer, maxMemory) + serializer: Serializer) = { + this(execId, actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties) } /** @@ -479,9 +469,10 @@ class BlockManager( * fashion as they're received. Expects a size in bytes to be provided for each block fetched, * so that we can control the maxMegabytesInFlight for the fetch. */ - def getMultiple(blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])]) + def getMultiple( + blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], serializer: Serializer) : BlockFetcherIterator = { - return new BlockFetcherIterator(this, blocksByAddress) + return new BlockFetcherIterator(this, blocksByAddress, serializer) } def put(blockId: String, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean) @@ -495,8 +486,8 @@ class BlockManager( * A short circuited method to get a block writer that can write data directly to disk. * This is currently used for writing shuffle files out. */ - def getBlockWriter(blockId: String): BlockObjectWriter = { - val writer = diskStore.getBlockWriter(blockId) + def getBlockWriter(blockId: String, serializer: Serializer): BlockObjectWriter = { + val writer = diskStore.getBlockWriter(blockId, serializer) writer.registerCloseEventHandler(() => { // TODO(rxin): This doesn't handle error cases. val myInfo = new BlockInfo(StorageLevel.DISK_ONLY, false) @@ -850,7 +841,10 @@ class BlockManager( if (shouldCompress(blockId)) new LZFInputStream(s) else s } - def dataSerialize(blockId: String, values: Iterator[Any]): ByteBuffer = { + def dataSerialize( + blockId: String, + values: Iterator[Any], + serializer: Serializer = defaultSerializer): ByteBuffer = { val byteStream = new FastByteArrayOutputStream(4096) val ser = serializer.newInstance() ser.serializeStream(wrapForCompression(blockId, byteStream)).writeAll(values).close() @@ -862,7 +856,10 @@ class BlockManager( * Deserializes a ByteBuffer into an iterator of values and disposes of it when the end of * the iterator is reached. */ - def dataDeserialize(blockId: String, bytes: ByteBuffer): Iterator[Any] = { + def dataDeserialize( + blockId: String, + bytes: ByteBuffer, + serializer: Serializer = defaultSerializer): Iterator[Any] = { bytes.rewind() val stream = wrapForCompression(blockId, new ByteBufferInputStream(bytes, true)) serializer.newInstance().deserializeStream(stream).asIterator @@ -916,7 +913,8 @@ object BlockManager extends Logging { class BlockFetcherIterator( private val blockManager: BlockManager, - val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])] + val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], + serializer: Serializer ) extends Iterator[(String, Option[Iterator[Any]])] with Logging with BlockFetchTracker { import blockManager._ @@ -979,8 +977,8 @@ class BlockFetcherIterator( "Unexpected message " + blockMessage.getType + " received from " + cmId) } val blockId = blockMessage.getId - results.put(new FetchResult( - blockId, sizeMap(blockId), () => dataDeserialize(blockId, blockMessage.getData))) + results.put(new FetchResult(blockId, sizeMap(blockId), + () => dataDeserialize(blockId, blockMessage.getData, serializer))) _remoteBytesRead += req.size logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) } diff --git a/core/src/main/scala/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/spark/storage/BlockManagerWorker.scala index d2985559c1..15225f93a6 100644 --- a/core/src/main/scala/spark/storage/BlockManagerWorker.scala +++ b/core/src/main/scala/spark/storage/BlockManagerWorker.scala @@ -19,7 +19,7 @@ import spark.network._ */ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends Logging { initLogging() - + blockManager.connectionManager.onReceiveMessage(onBlockMessageReceive) def onBlockMessageReceive(msg: Message, id: ConnectionManagerId): Option[Message] = { @@ -51,7 +51,7 @@ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends logDebug("Received [" + pB + "]") putBlock(pB.id, pB.data, pB.level) return None - } + } case BlockMessage.TYPE_GET_BLOCK => { val gB = new GetBlock(blockMessage.getId) logDebug("Received [" + gB + "]") @@ -90,28 +90,26 @@ private[spark] object BlockManagerWorker extends Logging { private var blockManagerWorker: BlockManagerWorker = null private val DATA_TRANSFER_TIME_OUT_MS: Long = 500 private val REQUEST_RETRY_INTERVAL_MS: Long = 1000 - + initLogging() - + def startBlockManagerWorker(manager: BlockManager) { blockManagerWorker = new BlockManagerWorker(manager) } - + def syncPutBlock(msg: PutBlock, toConnManagerId: ConnectionManagerId): Boolean = { val blockManager = blockManagerWorker.blockManager - val connectionManager = blockManager.connectionManager - val serializer = blockManager.serializer + val connectionManager = blockManager.connectionManager val blockMessage = BlockMessage.fromPutBlock(msg) val blockMessageArray = new BlockMessageArray(blockMessage) val resultMessage = connectionManager.sendMessageReliablySync( toConnManagerId, blockMessageArray.toBufferMessage) return (resultMessage != None) } - + def syncGetBlock(msg: GetBlock, toConnManagerId: ConnectionManagerId): ByteBuffer = { val blockManager = blockManagerWorker.blockManager - val connectionManager = blockManager.connectionManager - val serializer = blockManager.serializer + val connectionManager = blockManager.connectionManager val blockMessage = BlockMessage.fromGetBlock(msg) val blockMessageArray = new BlockMessageArray(blockMessage) val responseMessage = connectionManager.sendMessageReliablySync( diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 493936fdbe..70ad887c3b 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -1,17 +1,18 @@ package spark.storage -import java.nio.ByteBuffer import java.io.{File, FileOutputStream, OutputStream, RandomAccessFile} +import java.nio.ByteBuffer import java.nio.channels.FileChannel.MapMode import java.util.{Random, Date} import java.text.SimpleDateFormat -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream - import scala.collection.mutable.ArrayBuffer +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream + import spark.Utils import spark.executor.ExecutorExitCode +import spark.serializer.Serializer /** @@ -20,12 +21,13 @@ import spark.executor.ExecutorExitCode private class DiskStore(blockManager: BlockManager, rootDirs: String) extends BlockStore(blockManager) { - class DiskBlockObjectWriter(blockId: String) extends BlockObjectWriter(blockId) { + class DiskBlockObjectWriter(blockId: String, serializer: Serializer) + extends BlockObjectWriter(blockId) { private val f: File = createFile(blockId /*, allowAppendExisting */) private val bs: OutputStream = blockManager.wrapForCompression(blockId, new FastBufferedOutputStream(new FileOutputStream(f))) - private val objOut = blockManager.shuffleSerializer.newInstance().serializeStream(bs) + private val objOut = serializer.newInstance().serializeStream(bs) private var _size: Long = -1L @@ -58,8 +60,8 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) addShutdownHook() - def getBlockWriter(blockId: String): BlockObjectWriter = { - new DiskBlockObjectWriter(blockId) + def getBlockWriter(blockId: String, serializer: Serializer): BlockObjectWriter = { + new DiskBlockObjectWriter(blockId, serializer) } override def getSize(blockId: String): Long = { @@ -92,7 +94,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) val file = createFile(blockId) val fileOut = blockManager.wrapForCompression(blockId, new FastBufferedOutputStream(new FileOutputStream(file))) - val objOut = blockManager.serializer.newInstance().serializeStream(fileOut) + val objOut = blockManager.defaultSerializer.newInstance().serializeStream(fileOut) objOut.writeAll(values.iterator) objOut.close() val length = file.length() diff --git a/core/src/main/scala/spark/storage/ThreadingTest.scala b/core/src/main/scala/spark/storage/ThreadingTest.scala index 3875e7459e..5c406e68cb 100644 --- a/core/src/main/scala/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/spark/storage/ThreadingTest.scala @@ -78,7 +78,7 @@ private[spark] object ThreadingTest { val blockManagerMaster = new BlockManagerMaster( actorSystem.actorOf(Props(new BlockManagerMasterActor(true)))) val blockManager = new BlockManager( - "", actorSystem, blockManagerMaster, serializer, serializer, 1024 * 1024) + "", actorSystem, blockManagerMaster, serializer, 1024 * 1024) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) producers.foreach(_.start) -- cgit v1.2.3 From ba6ffa6a5f39765e1652735d1b16b54c2fc78674 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 24 Apr 2013 17:38:07 -0700 Subject: Allow the specification of a shuffle serializer in the read path (for local block reads). --- .../scala/spark/scheduler/ShuffleMapTask.scala | 2 +- .../main/scala/spark/storage/BlockManager.scala | 29 +++++++++++----------- core/src/main/scala/spark/storage/DiskStore.scala | 8 ++++++ 3 files changed, 24 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index d9b26c9db9..826f14d658 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -128,7 +128,7 @@ private[spark] class ShuffleMapTask( val blockManager = SparkEnv.get.blockManager val buckets = Array.tabulate[BlockObjectWriter](numOutputSplits) { bucketId => val blockId = "shuffle_" + dep.shuffleId + "_" + partition + "_" + bucketId - blockManager.getBlockWriter(blockId, Serializer.get(dep.serializerClass)) + blockManager.getDiskBlockWriter(blockId, Serializer.get(dep.serializerClass)) } // Write the map output to its associated buckets. diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 9f7985e2e8..fa02dd54b8 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -268,23 +268,24 @@ class BlockManager( return locations } + /** + * A short-circuited method to get blocks directly from disk. This is used for getting + * shuffle blocks. It is safe to do so without a lock on block info since disk store + * never deletes (recent) items. + */ + def getLocalFromDisk(blockId: String, serializer: Serializer): Option[Iterator[Any]] = { + diskStore.getValues(blockId, serializer) match { + case Some(iterator) => Some(iterator) + case None => + throw new Exception("Block " + blockId + " not found on disk, though it should be") + } + } + /** * Get block from local block manager. */ def getLocal(blockId: String): Option[Iterator[Any]] = { logDebug("Getting local block " + blockId) - - // As an optimization for map output fetches, if the block is for a shuffle, return it - // without acquiring a lock; the disk store never deletes (recent) items so this should work - if (blockId.startsWith("shuffle_")) { - return diskStore.getValues(blockId) match { - case Some(iterator) => - Some(iterator) - case None => - throw new Exception("Block " + blockId + " not found on disk, though it should be") - } - } - val info = blockInfo.get(blockId).orNull if (info != null) { info.synchronized { @@ -486,7 +487,7 @@ class BlockManager( * A short circuited method to get a block writer that can write data directly to disk. * This is currently used for writing shuffle files out. */ - def getBlockWriter(blockId: String, serializer: Serializer): BlockObjectWriter = { + def getDiskBlockWriter(blockId: String, serializer: Serializer): BlockObjectWriter = { val writer = diskStore.getBlockWriter(blockId, serializer) writer.registerCloseEventHandler(() => { // TODO(rxin): This doesn't handle error cases. @@ -1042,7 +1043,7 @@ class BlockFetcherIterator( // any memory that might exceed our maxBytesInFlight startTime = System.currentTimeMillis for (id <- localBlockIds) { - getLocal(id) match { + getLocalFromDisk(id, serializer) match { case Some(iter) => { results.put(new FetchResult(id, 0, () => iter)) // Pass 0 as size since it's not in flight logDebug("Got local block " + id) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 70ad887c3b..7f512b162a 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -127,6 +127,14 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) getBytes(blockId).map(bytes => blockManager.dataDeserialize(blockId, bytes)) } + /** + * A version of getValues that allows a custom serializer. This is used as part of the + * shuffle short-circuit code. + */ + def getValues(blockId: String, serializer: Serializer): Option[Iterator[Any]] = { + getBytes(blockId).map(bytes => blockManager.dataDeserialize(blockId, bytes, serializer)) + } + override def remove(blockId: String): Boolean = { val file = getFile(blockId) if (file.exists()) { -- cgit v1.2.3 From 7007201201981c6fb002e3008d97a6d6248f4dba Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Apr 2013 23:07:03 -0700 Subject: Added a shuffle block manager so it is easier in the future to consolidate shuffle output files. --- .../scala/spark/scheduler/ShuffleMapTask.scala | 50 +++++++++++------ .../main/scala/spark/storage/BlockManager.scala | 16 ++++-- .../scala/spark/storage/BlockObjectWriter.scala | 43 ++++++++++---- core/src/main/scala/spark/storage/DiskStore.scala | 65 +++++++++++++++------- .../scala/spark/storage/ShuffleBlockManager.scala | 52 +++++++++++++++++ 5 files changed, 175 insertions(+), 51 deletions(-) create mode 100644 core/src/main/scala/spark/storage/ShuffleBlockManager.scala diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 51ec89eb74..124d2d7e26 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -86,8 +86,14 @@ private[spark] class ShuffleMapTask( protected def this() = this(0, null, null, 0, null) - // data locality is on a per host basis, not hyper specific to container (host:port). Unique on set of hosts. - private val preferredLocs: Seq[String] = if (locs == null) Nil else locs.map(loc => Utils.parseHostPort(loc)._1).toSet.toSeq + // Data locality is on a per host basis, not hyper specific to container (host:port). + // Unique on set of hosts. + // TODO(rxin): The above statement seems problematic. Even if partitions are on the same host, + // the worker would still need to serialize / deserialize those data when they are in + // different jvm processes. Often that is very costly ... + @transient + private val preferredLocs: Seq[String] = + if (locs == null) Nil else locs.map(loc => Utils.parseHostPort(loc)._1).toSet.toSeq { // DEBUG code @@ -131,31 +137,32 @@ private[spark] class ShuffleMapTask( val taskContext = new TaskContext(stageId, partition, attemptId) metrics = Some(taskContext.taskMetrics) + + val blockManager = SparkEnv.get.blockManager + var shuffle: ShuffleBlockManager#Shuffle = null + var buckets: ShuffleWriterGroup = null + try { // Obtain all the block writers for shuffle blocks. - val blockManager = SparkEnv.get.blockManager - val buckets = Array.tabulate[BlockObjectWriter](numOutputSplits) { bucketId => - val blockId = "shuffle_" + dep.shuffleId + "_" + partition + "_" + bucketId - blockManager.getDiskBlockWriter(blockId, Serializer.get(dep.serializerClass)) - } + val ser = Serializer.get(dep.serializerClass) + shuffle = blockManager.shuffleBlockManager.forShuffle(dep.shuffleId, numOutputSplits, ser) + buckets = shuffle.acquireWriters(partition) // Write the map output to its associated buckets. for (elem <- rdd.iterator(split, taskContext)) { val pair = elem.asInstanceOf[(Any, Any)] val bucketId = dep.partitioner.getPartition(pair._1) - buckets(bucketId).write(pair) + buckets.writers(bucketId).write(pair) } - // Close the bucket writers and get the sizes of each block. - val compressedSizes = new Array[Byte](numOutputSplits) - var i = 0 + // Commit the writes. Get the size of each bucket block (total block size). var totalBytes = 0L - while (i < numOutputSplits) { - buckets(i).close() - val size = buckets(i).size() + val compressedSizes: Array[Byte] = buckets.writers.map { writer: BlockObjectWriter => + writer.commit() + writer.close() + val size = writer.size() totalBytes += size - compressedSizes(i) = MapOutputTracker.compressSize(size) - i += 1 + MapOutputTracker.compressSize(size) } // Update shuffle metrics. @@ -164,7 +171,18 @@ private[spark] class ShuffleMapTask( metrics.get.shuffleWriteMetrics = Some(shuffleMetrics) return new MapStatus(blockManager.blockManagerId, compressedSizes) + } catch { case e: Exception => + // If there is an exception from running the task, revert the partial writes + // and throw the exception upstream to Spark. + if (buckets != null) { + buckets.writers.foreach(_.revertPartialWrites()) + } + throw e } finally { + // Release the writers back to the shuffle block manager. + if (shuffle != null && buckets != null) { + shuffle.releaseWriters(buckets) + } // Execute the callbacks on task completion. taskContext.executeOnCompleteCallbacks() } diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 9190c96c71..b94d729923 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -88,6 +88,8 @@ class BlockManager( } } + val shuffleBlockManager = new ShuffleBlockManager(this) + private val blockInfo = new TimeStampedHashMap[String, BlockInfo] private[storage] val memoryStore: BlockStore = new MemoryStore(this, maxMemory) @@ -391,7 +393,7 @@ class BlockManager( // As an optimization for map output fetches, if the block is for a shuffle, return it // without acquiring a lock; the disk store never deletes (recent) items so this should work - if (blockId.startsWith("shuffle_")) { + if (ShuffleBlockManager.isShuffle(blockId)) { return diskStore.getBytes(blockId) match { case Some(bytes) => Some(bytes) @@ -508,12 +510,12 @@ class BlockManager( /** * A short circuited method to get a block writer that can write data directly to disk. - * This is currently used for writing shuffle files out. + * This is currently used for writing shuffle files out. Callers should handle error + * cases. */ def getDiskBlockWriter(blockId: String, serializer: Serializer): BlockObjectWriter = { val writer = diskStore.getBlockWriter(blockId, serializer) writer.registerCloseEventHandler(() => { - // TODO(rxin): This doesn't handle error cases. val myInfo = new BlockInfo(StorageLevel.DISK_ONLY, false) blockInfo.put(blockId, myInfo) myInfo.markReady(writer.size()) @@ -872,7 +874,7 @@ class BlockManager( } def shouldCompress(blockId: String): Boolean = { - if (blockId.startsWith("shuffle_")) { + if (ShuffleBlockManager.isShuffle(blockId)) { compressShuffle } else if (blockId.startsWith("broadcast_")) { compressBroadcast @@ -887,7 +889,11 @@ class BlockManager( * Wrap an output stream for compression if block compression is enabled for its block type */ def wrapForCompression(blockId: String, s: OutputStream): OutputStream = { - if (shouldCompress(blockId)) new LZFOutputStream(s) else s + if (shouldCompress(blockId)) { + (new LZFOutputStream(s)).setFinishBlockOnFlush(true) + } else { + s + } } /** diff --git a/core/src/main/scala/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/spark/storage/BlockObjectWriter.scala index 657a7e9143..42e2b07d5c 100644 --- a/core/src/main/scala/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/spark/storage/BlockObjectWriter.scala @@ -3,25 +3,48 @@ package spark.storage import java.nio.ByteBuffer +/** + * An interface for writing JVM objects to some underlying storage. This interface allows + * appending data to an existing block, and can guarantee atomicity in the case of faults + * as it allows the caller to revert partial writes. + * + * This interface does not support concurrent writes. + */ abstract class BlockObjectWriter(val blockId: String) { - // TODO(rxin): What if there is an exception when the block is being written out? - var closeEventHandler: () => Unit = _ - def registerCloseEventHandler(handler: () => Unit) { - closeEventHandler = handler + def open(): BlockObjectWriter + + def close() { + closeEventHandler() } - def write(value: Any) + def isOpen: Boolean - def writeAll(value: Iterator[Any]) { - value.foreach(write) + def registerCloseEventHandler(handler: () => Unit) { + closeEventHandler = handler } - def close() { - closeEventHandler() - } + /** + * Flush the partial writes and commit them as a single atomic block. Return the + * number of bytes written for this commit. + */ + def commit(): Long + + /** + * Reverts writes that haven't been flushed yet. Callers should invoke this function + * when there are runtime exceptions. + */ + def revertPartialWrites() + + /** + * Writes an object. + */ + def write(value: Any) + /** + * Size of the valid writes, in bytes. + */ def size(): Long } diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index b527a3c708..f23cd5475f 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -12,7 +12,7 @@ import it.unimi.dsi.fastutil.io.FastBufferedOutputStream import spark.Utils import spark.executor.ExecutorExitCode -import spark.serializer.Serializer +import spark.serializer.{Serializer, SerializationStream} /** @@ -21,35 +21,58 @@ import spark.serializer.Serializer private class DiskStore(blockManager: BlockManager, rootDirs: String) extends BlockStore(blockManager) { - private val mapMode = MapMode.READ_ONLY - private var mapOpenMode = "r" - class DiskBlockObjectWriter(blockId: String, serializer: Serializer) extends BlockObjectWriter(blockId) { private val f: File = createFile(blockId /*, allowAppendExisting */) - private val bs: OutputStream = blockManager.wrapForCompression(blockId, - new FastBufferedOutputStream(new FileOutputStream(f))) - private val objOut = serializer.newInstance().serializeStream(bs) - - private var _size: Long = -1L - override def write(value: Any) { - objOut.writeObject(value) + private var repositionableStream: FastBufferedOutputStream = null + private var bs: OutputStream = null + private var objOut: SerializationStream = null + private var validLength = 0L + + override def open(): DiskBlockObjectWriter = { + println("------------------------------------------------- opening " + f) + repositionableStream = new FastBufferedOutputStream(new FileOutputStream(f)) + bs = blockManager.wrapForCompression(blockId, repositionableStream) + objOut = serializer.newInstance().serializeStream(bs) + this } override def close() { objOut.close() bs.close() + objOut = null + bs = null + repositionableStream = null + // Invoke the close callback handler. super.close() } - override def size(): Long = { - if (_size < 0) { - _size = f.length() - } - _size + override def isOpen: Boolean = objOut != null + + // Flush the partial writes, and set valid length to be the length of the entire file. + // Return the number of bytes written for this commit. + override def commit(): Long = { + bs.flush() + repositionableStream.position() + } + + override def revertPartialWrites() { + // Flush the outstanding writes and delete the file. + objOut.close() + bs.close() + objOut = null + bs = null + repositionableStream = null + f.delete() + } + + override def write(value: Any) { + objOut.writeObject(value) } + + override def size(): Long = validLength } val MAX_DIR_CREATION_ATTEMPTS: Int = 10 @@ -90,9 +113,9 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) private def getFileBytes(file: File): ByteBuffer = { val length = file.length() - val channel = new RandomAccessFile(file, mapOpenMode).getChannel() + val channel = new RandomAccessFile(file, "r").getChannel() val buffer = try { - channel.map(mapMode, 0, length) + channel.map(MapMode.READ_ONLY, 0, length) } finally { channel.close() } @@ -230,12 +253,14 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) } private def addShutdownHook() { - localDirs.foreach(localDir => Utils.registerShutdownDeleteDir(localDir) ) + localDirs.foreach(localDir => Utils.registerShutdownDeleteDir(localDir)) Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { override def run() { logDebug("Shutdown hook called") try { - localDirs.foreach(localDir => if (! Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir)) + localDirs.foreach { localDir => + if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) + } } catch { case t: Throwable => logError("Exception while deleting local spark dirs", t) } diff --git a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala new file mode 100644 index 0000000000..2b1138e7a0 --- /dev/null +++ b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala @@ -0,0 +1,52 @@ +package spark.storage + +import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue} +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection.mutable.ArrayBuffer + +import spark.serializer.Serializer + + +private[spark] +class ShuffleWriterGroup(val id: Int, val writers: Array[BlockObjectWriter]) + + +private[spark] +class ShuffleBlockManager(blockManager: BlockManager) { + + val shuffles = new ConcurrentHashMap[Int, Shuffle] + + def forShuffle(shuffleId: Int, numBuckets: Int, serializer: Serializer): Shuffle = { + new Shuffle(shuffleId, numBuckets, serializer) + } + + class Shuffle(shuffleId: Int, numBuckets: Int, serializer: Serializer) { + + // Get a group of writers for a map task. + def acquireWriters(mapId: Int): ShuffleWriterGroup = { + val writers = Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => + val blockId = ShuffleBlockManager.blockId(shuffleId, bucketId, mapId) + blockManager.getDiskBlockWriter(blockId, serializer).open() + } + new ShuffleWriterGroup(mapId, writers) + } + + def releaseWriters(group: ShuffleWriterGroup) = { + // Nothing really to release here. + } + } +} + + +private[spark] +object ShuffleBlockManager { + + // Returns the block id for a given shuffle block. + def blockId(shuffleId: Int, bucketId: Int, groupId: Int): String = { + "shuffle_" + shuffleId + "_" + groupId + "_" + bucketId + } + + // Returns true if the block is a shuffle block. + def isShuffle(blockId: String): Boolean = blockId.startsWith("shuffle_") +} -- cgit v1.2.3 From 1055785a836ab2361239f0937a1a22fee953e029 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Apr 2013 23:33:56 -0700 Subject: Allow specifying the shuffle write file buffer size. The default buffer size is 8KB in FastBufferedOutputStream, which is too small and would cause a lot of disk seeks. --- core/src/main/scala/spark/storage/BlockManager.scala | 5 +++-- core/src/main/scala/spark/storage/DiskStore.scala | 10 +++++----- core/src/main/scala/spark/storage/ShuffleBlockManager.scala | 3 ++- 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index b94d729923..6e0ca9204d 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -513,8 +513,9 @@ class BlockManager( * This is currently used for writing shuffle files out. Callers should handle error * cases. */ - def getDiskBlockWriter(blockId: String, serializer: Serializer): BlockObjectWriter = { - val writer = diskStore.getBlockWriter(blockId, serializer) + def getDiskBlockWriter(blockId: String, serializer: Serializer, bufferSize: Int) + : BlockObjectWriter = { + val writer = diskStore.getBlockWriter(blockId, serializer, bufferSize) writer.registerCloseEventHandler(() => { val myInfo = new BlockInfo(StorageLevel.DISK_ONLY, false) blockInfo.put(blockId, myInfo) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index f23cd5475f..4cddcc86fc 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -21,7 +21,7 @@ import spark.serializer.{Serializer, SerializationStream} private class DiskStore(blockManager: BlockManager, rootDirs: String) extends BlockStore(blockManager) { - class DiskBlockObjectWriter(blockId: String, serializer: Serializer) + class DiskBlockObjectWriter(blockId: String, serializer: Serializer, bufferSize: Int) extends BlockObjectWriter(blockId) { private val f: File = createFile(blockId /*, allowAppendExisting */) @@ -32,7 +32,6 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) private var validLength = 0L override def open(): DiskBlockObjectWriter = { - println("------------------------------------------------- opening " + f) repositionableStream = new FastBufferedOutputStream(new FileOutputStream(f)) bs = blockManager.wrapForCompression(blockId, repositionableStream) objOut = serializer.newInstance().serializeStream(bs) @@ -55,7 +54,8 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) // Return the number of bytes written for this commit. override def commit(): Long = { bs.flush() - repositionableStream.position() + validLength = repositionableStream.position() + validLength } override def revertPartialWrites() { @@ -86,8 +86,8 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) addShutdownHook() - def getBlockWriter(blockId: String, serializer: Serializer): BlockObjectWriter = { - new DiskBlockObjectWriter(blockId, serializer) + def getBlockWriter(blockId: String, serializer: Serializer, bufferSize: Int): BlockObjectWriter = { + new DiskBlockObjectWriter(blockId, serializer, bufferSize) } override def getSize(blockId: String): Long = { diff --git a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala index 2b1138e7a0..2b22dad459 100644 --- a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala @@ -25,9 +25,10 @@ class ShuffleBlockManager(blockManager: BlockManager) { // Get a group of writers for a map task. def acquireWriters(mapId: Int): ShuffleWriterGroup = { + val bufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024 val writers = Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => val blockId = ShuffleBlockManager.blockId(shuffleId, bucketId, mapId) - blockManager.getDiskBlockWriter(blockId, serializer).open() + blockManager.getDiskBlockWriter(blockId, serializer, bufferSize).open() } new ShuffleWriterGroup(mapId, writers) } -- cgit v1.2.3 From e46d547ccd43c0fb3a79a30a7c43a78afba6f93f Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Tue, 30 Apr 2013 16:15:56 +0530 Subject: Fix issues reported by Reynold --- .../scala/spark/network/ConnectionManager.scala | 64 ++++++++++++++++++---- run | 7 ++- 2 files changed, 56 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 0c6bdb1559..a79fce8697 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -188,6 +188,38 @@ private[spark] class ConnectionManager(port: Int) extends Logging { } ) } + // MUST be called within selector loop - else deadlock. + private def triggerForceCloseByException(key: SelectionKey, e: Exception) { + try { + key.interestOps(0) + } catch { + // ignore exceptions + case e: Exception => logDebug("Ignoring exception", e) + } + + val conn = connectionsByKey.getOrElse(key, null) + if (conn == null) return + + // Pushing to connect threadpool + handleConnectExecutor.execute(new Runnable { + override def run() { + try { + conn.callOnExceptionCallback(e) + } catch { + // ignore exceptions + case e: Exception => logDebug("Ignoring exception", e) + } + try { + conn.close() + } catch { + // ignore exceptions + case e: Exception => logDebug("Ignoring exception", e) + } + } + }) + } + + def run() { try { while(!selectorThread.isInterrupted) { @@ -235,18 +267,26 @@ private[spark] class ConnectionManager(port: Int) extends Logging { while (selectedKeys.hasNext()) { val key = selectedKeys.next selectedKeys.remove() - if (key.isValid) { - if (key.isAcceptable) { - acceptConnection(key) - } else - if (key.isConnectable) { - triggerConnect(key) - } else - if (key.isReadable) { - triggerRead(key) - } else - if (key.isWritable) { - triggerWrite(key) + try { + if (key.isValid) { + if (key.isAcceptable) { + acceptConnection(key) + } else + if (key.isConnectable) { + triggerConnect(key) + } else + if (key.isReadable) { + triggerRead(key) + } else + if (key.isWritable) { + triggerWrite(key) + } + } + } catch { + // weird, but we saw this happening - even though key.isValid was true, key.isAcceptable would throw CancelledKeyException. + case e: CancelledKeyException => { + logInfo("key already cancelled ? " + key, e) + triggerForceCloseByException(key, e) } } } diff --git a/run b/run index 756f8703f2..0a58ac4a36 100755 --- a/run +++ b/run @@ -95,6 +95,7 @@ export JAVA_OPTS CORE_DIR="$FWDIR/core" REPL_DIR="$FWDIR/repl" +REPL_BIN_DIR="$FWDIR/repl-bin" EXAMPLES_DIR="$FWDIR/examples" BAGEL_DIR="$FWDIR/bagel" STREAMING_DIR="$FWDIR/streaming" @@ -125,8 +126,8 @@ if [ -e "$FWDIR/lib_managed" ]; then CLASSPATH+=":$FWDIR/lib_managed/bundles/*" fi CLASSPATH+=":$REPL_DIR/lib/*" -if [ -e repl-bin/target ]; then - for jar in `find "repl-bin/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do +if [ -e $REPL_BIN_DIR/target ]; then + for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do CLASSPATH+=":$jar" done fi @@ -134,7 +135,6 @@ CLASSPATH+=":$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do CLASSPATH+=":$jar" done -export CLASSPATH # Needed for spark-shell # Figure out the JAR file that our examples were packaged into. This includes a bit of a hack # to avoid the -sources and -doc packages that are built by publish-local. @@ -163,4 +163,5 @@ else EXTRA_ARGS="$JAVA_OPTS" fi +export CLASSPATH # Needed for spark-shell exec "$RUNNER" -cp "$CLASSPATH" $EXTRA_ARGS "$@" -- cgit v1.2.3 From 48854e1dbf1d02e1e19f59d0aee0e281d41b3b45 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Tue, 30 Apr 2013 23:59:33 +0530 Subject: If key is not valid, close connection --- .gitignore | 2 -- core/src/main/scala/spark/network/ConnectionManager.scala | 3 +++ 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/.gitignore b/.gitignore index 155e785b01..5bb2f33574 100644 --- a/.gitignore +++ b/.gitignore @@ -29,8 +29,6 @@ project/build/target/ project/plugins/target/ project/plugins/lib_managed/ project/plugins/src_managed/ -logs/ -log/ spark-tests.log streaming-tests.log dependency-reduced-pom.xml diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index a79fce8697..2d9b4be4b3 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -281,6 +281,9 @@ private[spark] class ConnectionManager(port: Int) extends Logging { if (key.isWritable) { triggerWrite(key) } + } else { + logInfo("Key not valid ? " + key) + throw new CancelledKeyException() } } catch { // weird, but we saw this happening - even though key.isValid was true, key.isAcceptable would throw CancelledKeyException. -- cgit v1.2.3 From 538614acfe95b0c064679122af3bc990b669e4e0 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 1 May 2013 00:05:32 +0530 Subject: Be more aggressive and defensive in select also --- .../scala/spark/network/ConnectionManager.scala | 83 ++++++++++++++-------- 1 file changed, 55 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 2d9b4be4b3..9b00fddd40 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -254,7 +254,32 @@ private[spark] class ConnectionManager(port: Int) extends Logging { } } - val selectedKeysCount = selector.select() + val selectedKeysCount = + try { + selector.select() + } catch { + case e: CancelledKeyException => { + // Some keys within the selectors list are invalid/closed. clear them. + val allKeys = selector.keys().iterator() + + while (allKeys.hasNext()) { + val key = allKeys.next() + try { + if (! key.isValid) { + logInfo("Key not valid ? " + key) + throw new CancelledKeyException() + } + } catch { + case e: CancelledKeyException => { + logInfo("key already cancelled ? " + key, e) + triggerForceCloseByException(key, e) + } + } + } + } + 0 + } + if (selectedKeysCount == 0) { logDebug("Selector selected " + selectedKeysCount + " of " + selector.keys.size + " keys") } @@ -262,34 +287,36 @@ private[spark] class ConnectionManager(port: Int) extends Logging { logInfo("Selector thread was interrupted!") return } - - val selectedKeys = selector.selectedKeys().iterator() - while (selectedKeys.hasNext()) { - val key = selectedKeys.next - selectedKeys.remove() - try { - if (key.isValid) { - if (key.isAcceptable) { - acceptConnection(key) - } else - if (key.isConnectable) { - triggerConnect(key) - } else - if (key.isReadable) { - triggerRead(key) - } else - if (key.isWritable) { - triggerWrite(key) + + if (0 != selectedKeysCount) { + val selectedKeys = selector.selectedKeys().iterator() + while (selectedKeys.hasNext()) { + val key = selectedKeys.next + selectedKeys.remove() + try { + if (key.isValid) { + if (key.isAcceptable) { + acceptConnection(key) + } else + if (key.isConnectable) { + triggerConnect(key) + } else + if (key.isReadable) { + triggerRead(key) + } else + if (key.isWritable) { + triggerWrite(key) + } + } else { + logInfo("Key not valid ? " + key) + throw new CancelledKeyException() + } + } catch { + // weird, but we saw this happening - even though key.isValid was true, key.isAcceptable would throw CancelledKeyException. + case e: CancelledKeyException => { + logInfo("key already cancelled ? " + key, e) + triggerForceCloseByException(key, e) } - } else { - logInfo("Key not valid ? " + key) - throw new CancelledKeyException() - } - } catch { - // weird, but we saw this happening - even though key.isValid was true, key.isAcceptable would throw CancelledKeyException. - case e: CancelledKeyException => { - logInfo("key already cancelled ? " + key, e) - triggerForceCloseByException(key, e) } } } -- cgit v1.2.3 From 0f45477be16254971763cbc07feac7460cffd0bd Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 1 May 2013 00:10:02 +0530 Subject: Change indentation --- .../scala/spark/network/ConnectionManager.scala | 40 +++++++++++----------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 9b00fddd40..925d076951 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -255,30 +255,30 @@ private[spark] class ConnectionManager(port: Int) extends Logging { } val selectedKeysCount = - try { - selector.select() - } catch { - case e: CancelledKeyException => { - // Some keys within the selectors list are invalid/closed. clear them. - val allKeys = selector.keys().iterator() - - while (allKeys.hasNext()) { - val key = allKeys.next() - try { - if (! key.isValid) { - logInfo("Key not valid ? " + key) - throw new CancelledKeyException() - } - } catch { - case e: CancelledKeyException => { - logInfo("key already cancelled ? " + key, e) - triggerForceCloseByException(key, e) + try { + selector.select() + } catch { + case e: CancelledKeyException => { + // Some keys within the selectors list are invalid/closed. clear them. + val allKeys = selector.keys().iterator() + + while (allKeys.hasNext()) { + val key = allKeys.next() + try { + if (! key.isValid) { + logInfo("Key not valid ? " + key) + throw new CancelledKeyException() + } + } catch { + case e: CancelledKeyException => { + logInfo("key already cancelled ? " + key, e) + triggerForceCloseByException(key, e) + } } } } + 0 } - 0 - } if (selectedKeysCount == 0) { logDebug("Selector selected " + selectedKeysCount + " of " + selector.keys.size + " keys") -- cgit v1.2.3 From 3b748ced2258246bd9b7c250363645cea27cf622 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 1 May 2013 00:30:30 +0530 Subject: Be more aggressive and defensive in all uses of SelectionKey in select loop --- .../scala/spark/network/ConnectionManager.scala | 47 ++++++++++++++-------- 1 file changed, 30 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 925d076951..03926a6038 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -232,24 +232,37 @@ private[spark] class ConnectionManager(port: Int) extends Logging { while(!keyInterestChangeRequests.isEmpty) { val (key, ops) = keyInterestChangeRequests.dequeue - val connection = connectionsByKey.getOrElse(key, null) - if (connection != null) { - val lastOps = key.interestOps() - key.interestOps(ops) - - // hot loop - prevent materialization of string if trace not enabled. - if (isTraceEnabled()) { - def intToOpStr(op: Int): String = { - val opStrs = ArrayBuffer[String]() - if ((op & SelectionKey.OP_READ) != 0) opStrs += "READ" - if ((op & SelectionKey.OP_WRITE) != 0) opStrs += "WRITE" - if ((op & SelectionKey.OP_CONNECT) != 0) opStrs += "CONNECT" - if ((op & SelectionKey.OP_ACCEPT) != 0) opStrs += "ACCEPT" - if (opStrs.size > 0) opStrs.reduceLeft(_ + " | " + _) else " " - } - logTrace("Changed key for connection to [" + connection.getRemoteConnectionManagerId() + - "] changed from [" + intToOpStr(lastOps) + "] to [" + intToOpStr(ops) + "]") + try { + if (key.isValid) { + val connection = connectionsByKey.getOrElse(key, null) + if (connection != null) { + val lastOps = key.interestOps() + key.interestOps(ops) + + // hot loop - prevent materialization of string if trace not enabled. + if (isTraceEnabled()) { + def intToOpStr(op: Int): String = { + val opStrs = ArrayBuffer[String]() + if ((op & SelectionKey.OP_READ) != 0) opStrs += "READ" + if ((op & SelectionKey.OP_WRITE) != 0) opStrs += "WRITE" + if ((op & SelectionKey.OP_CONNECT) != 0) opStrs += "CONNECT" + if ((op & SelectionKey.OP_ACCEPT) != 0) opStrs += "ACCEPT" + if (opStrs.size > 0) opStrs.reduceLeft(_ + " | " + _) else " " + } + + logTrace("Changed key for connection to [" + connection.getRemoteConnectionManagerId() + + "] changed from [" + intToOpStr(lastOps) + "] to [" + intToOpStr(ops) + "]") + } + } + } else { + logInfo("Key not valid ? " + key) + throw new CancelledKeyException() + } + } catch { + case e: CancelledKeyException => { + logInfo("key already cancelled ? " + key, e) + triggerForceCloseByException(key, e) } } } -- cgit v1.2.3 From c446ac31d7065d227f168a7f27010bdf98ef7ad1 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 1 May 2013 00:32:30 +0530 Subject: Spurious commit, reverting gitignore change --- .gitignore | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.gitignore b/.gitignore index 5bb2f33574..155e785b01 100644 --- a/.gitignore +++ b/.gitignore @@ -29,6 +29,8 @@ project/build/target/ project/plugins/target/ project/plugins/lib_managed/ project/plugins/src_managed/ +logs/ +log/ spark-tests.log streaming-tests.log dependency-reduced-pom.xml -- cgit v1.2.3 From 60cabb35cbfd2af0e5ba34c4a416aa2640091acc Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 1 May 2013 01:17:14 +0530 Subject: Add addition catch block for exception too --- core/src/main/scala/spark/network/ConnectionManager.scala | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 03926a6038..0eb03630d0 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -264,6 +264,10 @@ private[spark] class ConnectionManager(port: Int) extends Logging { logInfo("key already cancelled ? " + key, e) triggerForceCloseByException(key, e) } + case e: Exception => { + logError("Exception processing key " + key, e) + triggerForceCloseByException(key, e) + } } } @@ -271,6 +275,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging { try { selector.select() } catch { + // Explicitly only dealing with CancelledKeyException here since other exceptions should be dealt with differently. case e: CancelledKeyException => { // Some keys within the selectors list are invalid/closed. clear them. val allKeys = selector.keys().iterator() @@ -287,6 +292,10 @@ private[spark] class ConnectionManager(port: Int) extends Logging { logInfo("key already cancelled ? " + key, e) triggerForceCloseByException(key, e) } + case e: Exception => { + logError("Exception processing key " + key, e) + triggerForceCloseByException(key, e) + } } } } @@ -330,6 +339,10 @@ private[spark] class ConnectionManager(port: Int) extends Logging { logInfo("key already cancelled ? " + key, e) triggerForceCloseByException(key, e) } + case e: Exception => { + logError("Exception processing key " + key, e) + triggerForceCloseByException(key, e) + } } } } -- cgit v1.2.3 From dd7bef31472e8c7dedc93bc1519be5900784c736 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Apr 2013 15:02:32 -0700 Subject: Two minor fixes according to Ryan LeCompte's review. --- core/src/main/scala/spark/storage/BlockManager.scala | 7 ++----- core/src/main/scala/spark/storage/ShuffleBlockManager.scala | 7 ------- 2 files changed, 2 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 6e0ca9204d..09572b19db 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -296,11 +296,8 @@ class BlockManager( * never deletes (recent) items. */ def getLocalFromDisk(blockId: String, serializer: Serializer): Option[Iterator[Any]] = { - diskStore.getValues(blockId, serializer) match { - case Some(iterator) => Some(iterator) - case None => - throw new Exception("Block " + blockId + " not found on disk, though it should be") - } + diskStore.getValues(blockId, serializer).orElse( + sys.error("Block " + blockId + " not found on disk, though it should be")) } /** diff --git a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala index 2b22dad459..1903df0817 100644 --- a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala @@ -1,10 +1,5 @@ package spark.storage -import java.util.concurrent.{ConcurrentHashMap, ConcurrentLinkedQueue} -import java.util.concurrent.atomic.AtomicInteger - -import scala.collection.mutable.ArrayBuffer - import spark.serializer.Serializer @@ -15,8 +10,6 @@ class ShuffleWriterGroup(val id: Int, val writers: Array[BlockObjectWriter]) private[spark] class ShuffleBlockManager(blockManager: BlockManager) { - val shuffles = new ConcurrentHashMap[Int, Shuffle] - def forShuffle(shuffleId: Int, numBuckets: Int, serializer: Serializer): Shuffle = { new Shuffle(shuffleId, numBuckets, serializer) } -- cgit v1.2.3 From 1d54401d7e41095d8cbeeefd42c9d39ee500cd9f Mon Sep 17 00:00:00 2001 From: unknown Date: Tue, 30 Apr 2013 23:01:32 -0600 Subject: Modified as per TD's suggestions --- ...etworkWordCumulativeCountUpdateStateByKey.scala | 63 ---------------------- .../examples/StatefulNetworkWordCount.scala | 52 ++++++++++++++++++ 2 files changed, 52 insertions(+), 63 deletions(-) delete mode 100644 examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala create mode 100644 examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala b/examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala deleted file mode 100644 index db62246387..0000000000 --- a/examples/src/main/scala/spark/streaming/examples/NetworkWordCumulativeCountUpdateStateByKey.scala +++ /dev/null @@ -1,63 +0,0 @@ -package spark.streaming.examples - -import spark.streaming._ -import spark.streaming.StreamingContext._ - -/** - * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. - * Usage: NetworkWordCumulativeCountUpdateStateByKey - * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. - * and describe the TCP server that Spark Streaming would connect to receive data. - * - * To run this on your local machine, you need to first run a Netcat server - * `$ nc -lk 9999` - * and then run the example - * `$ ./run spark.streaming.examples.NetworkWordCumulativeCountUpdateStateByKey local[2] localhost 9999` - */ -object NetworkWordCumulativeCountUpdateStateByKey { - private def className[A](a: A)(implicit m: Manifest[A]) = m.toString - - def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: NetworkWordCountUpdateStateByKey \n" + - "In local mode, should be 'local[n]' with n > 1") - System.exit(1) - } - - val updateFunc = (values: Seq[Int], state: Option[Int]) => { - val currentCount = values.foldLeft(0)(_ + _) - //println("currentCount: " + currentCount) - - val previousCount = state.getOrElse(0) - //println("previousCount: " + previousCount) - - val cumulative = Some(currentCount + previousCount) - //println("Cumulative: " + cumulative) - - cumulative - } - - // Create the context with a 10 second batch size - val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", Seconds(10), - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) - ssc.checkpoint(".") - - // Create a NetworkInputDStream on target ip:port and count the - // words in input stream of \n delimited test (eg. generated by 'nc') - val lines = ssc.socketTextStream(args(1), args(2).toInt) - val words = lines.flatMap(_.split(" ")) - val wordDstream = words.map(x => (x, 1)) - - // Update the cumulative count using updateStateByKey - // This will give a Dstream made of state (which is the cumulative count of the words) - val stateDstream = wordDstream.updateStateByKey[Int](updateFunc) - - stateDstream.foreach(rdd => { - rdd.foreach(rddVal => { - println("Current Count: " + rddVal) - }) - }) - - ssc.start() - } -} diff --git a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala new file mode 100644 index 0000000000..b662cb1162 --- /dev/null +++ b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -0,0 +1,52 @@ +package spark.streaming.examples + +import spark.streaming._ +import spark.streaming.StreamingContext._ + +/** + * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + * Usage: StatefulNetworkWordCount + * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. + * and describe the TCP server that Spark Streaming would connect to receive data. + * + * To run this on your local machine, you need to first run a Netcat server + * `$ nc -lk 9999` + * and then run the example + * `$ ./run spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` + */ +object StatefulNetworkWordCount { + private def className[A](a: A)(implicit m: Manifest[A]) = m.toString + + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println("Usage: StatefulNetworkWordCount \n" + + "In local mode, should be 'local[n]' with n > 1") + System.exit(1) + } + + val updateFunc = (values: Seq[Int], state: Option[Int]) => { + val currentCount = values.foldLeft(0)(_ + _) + + val previousCount = state.getOrElse(0) + + Some(currentCount + previousCount) + } + + // Create the context with a 10 second batch size + val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", Seconds(10), + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + ssc.checkpoint(".") + + // Create a NetworkInputDStream on target ip:port and count the + // words in input stream of \n delimited test (eg. generated by 'nc') + val lines = ssc.socketTextStream(args(1), args(2).toInt) + val words = lines.flatMap(_.split(" ")) + val wordDstream = words.map(x => (x, 1)) + + // Update the cumulative count using updateStateByKey + // This will give a Dstream made of state (which is the cumulative count of the words) + val stateDstream = wordDstream.updateStateByKey[Int](updateFunc) + stateDstream.print() + ssc.start() + } +} -- cgit v1.2.3 From d960e7e0f83385d8f43129d53c189b3036936daf Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 1 May 2013 20:24:00 +0530 Subject: a) Add support for hyper local scheduling - specific to a host + port - before trying host local scheduling. b) Add some fixes to test code to ensure it passes (and fixes some other issues). c) Fix bug in task scheduling which incorrectly used availableCores instead of all cores on the node. --- core/src/main/scala/spark/SparkEnv.scala | 21 ++- core/src/main/scala/spark/Utils.scala | 17 ++- .../main/scala/spark/deploy/worker/Worker.scala | 5 +- core/src/main/scala/spark/rdd/BlockRDD.scala | 9 +- core/src/main/scala/spark/rdd/ZippedRDD.scala | 2 + .../main/scala/spark/scheduler/DAGScheduler.scala | 7 +- .../main/scala/spark/scheduler/ResultTask.scala | 4 +- .../scala/spark/scheduler/ShuffleMapTask.scala | 4 +- .../spark/scheduler/cluster/ClusterScheduler.scala | 64 ++++++++- .../spark/scheduler/cluster/TaskSetManager.scala | 155 ++++++++++++++------- .../main/scala/spark/storage/BlockManager.scala | 43 ++++-- .../test/scala/spark/MapOutputTrackerSuite.scala | 6 +- .../scala/spark/scheduler/DAGSchedulerSuite.scala | 4 +- .../scala/spark/storage/BlockManagerSuite.scala | 2 + 14 files changed, 244 insertions(+), 99 deletions(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index ffb40bab3a..5b4a464010 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -29,7 +29,11 @@ class SparkEnv ( val blockManager: BlockManager, val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, - val sparkFilesDir: String + val sparkFilesDir: String, + // To be set only as part of initialization of SparkContext. + // (executorId, defaultHostPort) => executorHostPort + // If executorId is NOT found, return defaultHostPort + var executorIdToHostPort: (String, String) => String ) { def stop() { @@ -44,6 +48,17 @@ class SparkEnv ( // down, but let's call it anyway in case it gets fixed in a later release actorSystem.awaitTermination() } + + + def resolveExecutorIdToHostPort(executorId: String, defaultHostPort: String): String = { + val env = SparkEnv.get + if (env.executorIdToHostPort == null) { + // default to using host, not host port. Relevant to non cluster modes. + return defaultHostPort + } + + env.executorIdToHostPort(executorId, defaultHostPort) + } } object SparkEnv extends Logging { @@ -162,7 +177,7 @@ object SparkEnv extends Logging { blockManager, connectionManager, httpFileServer, - sparkFilesDir) + sparkFilesDir, + null) } - } diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 9f48cbe490..279daf04ed 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -357,21 +357,26 @@ private object Utils extends Logging { Utils.logErrorWithStack("Unexpected to have port " + port + " which is not valid in " + hostPort + ". Message " + message) } } + + // Used by DEBUG code : remove when all testing done + def logErrorWithStack(msg: String) { + try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } } + // temp code for debug + System.exit(-1) + } */ // Once testing is complete in various modes, replace with this ? def checkHost(host: String, message: String = "") {} def checkHostPort(hostPort: String, message: String = "") {} - def getUserNameFromEnvironment(): String = { - SparkHadoopUtil.getUserNameFromEnvironment - } - // Used by DEBUG code : remove when all testing done def logErrorWithStack(msg: String) { try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } } - // temp code for debug - System.exit(-1) + } + + def getUserNameFromEnvironment(): String = { + SparkHadoopUtil.getUserNameFromEnvironment } // Typically, this will be of order of number of nodes in cluster diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 1a7da0f7bf..3dc2207170 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -54,7 +54,10 @@ private[spark] class Worker( def createWorkDir() { workDir = Option(workDirPath).map(new File(_)).getOrElse(new File(sparkHome, "work")) try { - if ( (workDir.exists() && !workDir.isDirectory) || (!workDir.exists() && !workDir.mkdirs()) ) { + // This sporadically fails - not sure why ... !workDir.exists() && !workDir.mkdirs() + // So attempting to create and then check if directory was created or not. + workDir.mkdirs() + if ( !workDir.exists() || !workDir.isDirectory) { logError("Failed to create work directory " + workDir) System.exit(1) } diff --git a/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/spark/rdd/BlockRDD.scala index 7348c4f15b..719d4bf03e 100644 --- a/core/src/main/scala/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/spark/rdd/BlockRDD.scala @@ -1,7 +1,7 @@ package spark.rdd -import scala.collection.mutable.HashMap import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} +import spark.storage.BlockManager private[spark] class BlockRDDPartition(val blockId: String, idx: Int) extends Partition { val index = idx @@ -11,12 +11,7 @@ private[spark] class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[String]) extends RDD[T](sc, Nil) { - @transient lazy val locations_ = { - val blockManager = SparkEnv.get.blockManager - /*val locations = blockIds.map(id => blockManager.getLocations(id))*/ - val locations = blockManager.getLocations(blockIds) - HashMap(blockIds.zip(locations):_*) - } + @transient lazy val locations_ = BlockManager.blockIdsToExecutorLocations(blockIds, SparkEnv.get) override def getPartitions: Array[Partition] = (0 until blockIds.size).map(i => { new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition] diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala index 35b0e06785..e80250a99b 100644 --- a/core/src/main/scala/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala @@ -49,6 +49,8 @@ class ZippedRDD[T: ClassManifest, U: ClassManifest]( override def getPreferredLocations(s: Partition): Seq[String] = { val (partition1, partition2) = s.asInstanceOf[ZippedPartition[T, U]].partitions + // TODO: becomes complicated - intersect on hostPort if available, else fallback to host (removing intersected hostPort's). + // Since I am not very sure about this RDD, leaving it to others to comment better ! rdd1.preferredLocations(partition1).intersect(rdd2.preferredLocations(partition2)) } diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 1440b93e65..8072c60bb7 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -12,7 +12,7 @@ import spark.executor.TaskMetrics import spark.partial.ApproximateActionListener import spark.partial.ApproximateEvaluator import spark.partial.PartialResult -import spark.storage.BlockManagerMaster +import spark.storage.{BlockManager, BlockManagerMaster} import spark.util.{MetadataCleaner, TimeStampedHashMap} /** @@ -117,9 +117,8 @@ class DAGScheduler( private def getCacheLocs(rdd: RDD[_]): Array[List[String]] = { if (!cacheLocs.contains(rdd.id)) { val blockIds = rdd.partitions.indices.map(index=> "rdd_%d_%d".format(rdd.id, index)).toArray - cacheLocs(rdd.id) = blockManagerMaster.getLocations(blockIds).map { - locations => locations.map(_.hostPort).toList - }.toArray + val locs = BlockManager.blockIdsToExecutorLocations(blockIds, env) + cacheLocs(rdd.id) = blockIds.map(locs.getOrElse(_, Nil)) } cacheLocs(rdd.id) } diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala index 89dc6640b2..c43cbe5ed4 100644 --- a/core/src/main/scala/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/spark/scheduler/ResultTask.scala @@ -71,11 +71,11 @@ private[spark] class ResultTask[T, U]( } // data locality is on a per host basis, not hyper specific to container (host:port). Unique on set of hosts. - val preferredLocs: Seq[String] = if (locs == null) Nil else locs.map(loc => Utils.parseHostPort(loc)._1).toSet.toSeq + private val preferredLocs: Seq[String] = if (locs == null) Nil else locs.toSet.toSeq { // DEBUG code - preferredLocs.foreach (host => Utils.checkHost(host, "preferredLocs : " + preferredLocs)) + preferredLocs.foreach (hostPort => Utils.checkHost(Utils.parseHostPort(hostPort)._1, "preferredLocs : " + preferredLocs)) } override def run(attemptId: Long): U = { diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 7dc6da4573..0b848af2f3 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -85,11 +85,11 @@ private[spark] class ShuffleMapTask( protected def this() = this(0, null, null, 0, null) // data locality is on a per host basis, not hyper specific to container (host:port). Unique on set of hosts. - private val preferredLocs: Seq[String] = if (locs == null) Nil else locs.map(loc => Utils.parseHostPort(loc)._1).toSet.toSeq + private val preferredLocs: Seq[String] = if (locs == null) Nil else locs.toSet.toSeq { // DEBUG code - preferredLocs.foreach (host => Utils.checkHost(host, "preferredLocs : " + preferredLocs)) + preferredLocs.foreach (hostPort => Utils.checkHost(Utils.parseHostPort(hostPort)._1, "preferredLocs : " + preferredLocs)) } var split = if (rdd == null) { diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index a9d9c5e44c..3c72ce4206 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -79,9 +79,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // The set of executors we have on each host; this is used to compute hostsAlive, which // in turn is used to decide when we can attain data locality on a given host - val executorsByHostPort = new HashMap[String, HashSet[String]] + private val executorsByHostPort = new HashMap[String, HashSet[String]] - val executorIdToHostPort = new HashMap[String, String] + private val executorIdToHostPort = new HashMap[String, String] // JAR server, if any JARs were added by the user to the SparkContext var jarServer: HttpServer = null @@ -102,6 +102,14 @@ private[spark] class ClusterScheduler(val sc: SparkContext) def initialize(context: SchedulerBackend) { backend = context + // resolve executorId to hostPort mapping. + def executorToHostPort(executorId: String, defaultHostPort: String): String = { + executorIdToHostPort.getOrElse(executorId, defaultHostPort) + } + + // Unfortunately, this means that SparkEnv is indirectly referencing ClusterScheduler + // Will that be a design violation ? + SparkEnv.get.executorIdToHostPort = executorToHostPort } def newTaskId(): Long = nextTaskId.getAndIncrement() @@ -209,13 +217,30 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } // Build a list of tasks to assign to each slave val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) + // merge availableCpus into hostToAvailableCpus block ? val availableCpus = offers.map(o => o.cores).toArray + val hostToAvailableCpus = { + val map = new HashMap[String, Int]() + for (offer <- offers) { + val hostPort = offer.hostPort + val cores = offer.cores + // DEBUG code + Utils.checkHostPort(hostPort) + + val host = Utils.parseHostPort(hostPort)._1 + + map.put(host, map.getOrElse(host, 0) + cores) + } + + map + } var launchedTask = false for (manager <- activeTaskSetsQueue.sortBy(m => (m.taskSet.priority, m.taskSet.stageId))) { // Split offers based on host local, rack local and off-rack tasks. + val hyperLocalOffers = new HashMap[String, ArrayBuffer[Int]]() val hostLocalOffers = new HashMap[String, ArrayBuffer[Int]]() val rackLocalOffers = new HashMap[String, ArrayBuffer[Int]]() val otherOffers = new HashMap[String, ArrayBuffer[Int]]() @@ -224,8 +249,17 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val hostPort = offers(i).hostPort // DEBUG code Utils.checkHostPort(hostPort) + + val numHyperLocalTasks = math.max(0, math.min(manager.numPendingTasksForHostPort(hostPort), availableCpus(i))) + if (numHyperLocalTasks > 0){ + val list = hyperLocalOffers.getOrElseUpdate(hostPort, new ArrayBuffer[Int]) + for (j <- 0 until numHyperLocalTasks) list += i + } + val host = Utils.parseHostPort(hostPort)._1 - val numHostLocalTasks = math.max(0, math.min(manager.numPendingTasksForHost(hostPort), availableCpus(i))) + val numHostLocalTasks = math.max(0, + // Remove hyper local tasks (which are also host local btw !) from this + math.min(manager.numPendingTasksForHost(hostPort) - numHyperLocalTasks, hostToAvailableCpus(host))) if (numHostLocalTasks > 0){ val list = hostLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) for (j <- 0 until numHostLocalTasks) list += i @@ -233,7 +267,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val numRackLocalTasks = math.max(0, // Remove host local tasks (which are also rack local btw !) from this - math.min(manager.numRackLocalPendingTasksForHost(hostPort) - numHostLocalTasks, availableCpus(i))) + math.min(manager.numRackLocalPendingTasksForHost(hostPort) - numHyperLocalTasks - numHostLocalTasks, hostToAvailableCpus(host))) if (numRackLocalTasks > 0){ val list = rackLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) for (j <- 0 until numRackLocalTasks) list += i @@ -246,12 +280,19 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } val offersPriorityList = new ArrayBuffer[Int]( - hostLocalOffers.size + rackLocalOffers.size + otherOffers.size) - // First host local, then rack, then others + hyperLocalOffers.size + hostLocalOffers.size + rackLocalOffers.size + otherOffers.size) + + // First hyper local, then host local, then rack, then others + + // numHostLocalOffers contains count of both hyper local and host offers. val numHostLocalOffers = { + val hyperLocalPriorityList = ClusterScheduler.prioritizeContainers(hyperLocalOffers) + offersPriorityList ++= hyperLocalPriorityList + val hostLocalPriorityList = ClusterScheduler.prioritizeContainers(hostLocalOffers) offersPriorityList ++= hostLocalPriorityList - hostLocalPriorityList.size + + hyperLocalPriorityList.size + hostLocalPriorityList.size } val numRackLocalOffers = { val rackLocalPriorityList = ClusterScheduler.prioritizeContainers(rackLocalOffers) @@ -477,6 +518,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } def getExecutorsAliveOnHost(host: String): Option[Set[String]] = { + Utils.checkHost(host) + val retval = hostToAliveHostPorts.get(host) if (retval.isDefined) { return Some(retval.get.toSet) @@ -485,6 +528,13 @@ private[spark] class ClusterScheduler(val sc: SparkContext) None } + def isExecutorAliveOnHostPort(hostPort: String): Boolean = { + // Even if hostPort is a host, it does not matter - it is just a specific check. + // But we do have to ensure that only hostPort get into hostPortsAlive ! + // So no check against Utils.checkHostPort + hostPortsAlive.contains(hostPort) + } + // By default, rack is unknown def getRackForHost(value: String): Option[String] = None diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 27e713e2c4..f5c0058554 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -13,14 +13,18 @@ import spark.scheduler._ import spark.TaskState.TaskState import java.nio.ByteBuffer -private[spark] object TaskLocality extends Enumeration("HOST_LOCAL", "RACK_LOCAL", "ANY") with Logging { +private[spark] object TaskLocality extends Enumeration("HYPER_LOCAL", "HOST_LOCAL", "RACK_LOCAL", "ANY") with Logging { - val HOST_LOCAL, RACK_LOCAL, ANY = Value + // hyper local is expected to be used ONLY within tasksetmanager for now. + val HYPER_LOCAL, HOST_LOCAL, RACK_LOCAL, ANY = Value type TaskLocality = Value def isAllowed(constraint: TaskLocality, condition: TaskLocality): Boolean = { + // Must not be the constraint. + assert (constraint != TaskLocality.HYPER_LOCAL) + constraint match { case TaskLocality.HOST_LOCAL => condition == TaskLocality.HOST_LOCAL case TaskLocality.RACK_LOCAL => condition == TaskLocality.HOST_LOCAL || condition == TaskLocality.RACK_LOCAL @@ -32,7 +36,11 @@ private[spark] object TaskLocality extends Enumeration("HOST_LOCAL", "RACK_LOCAL def parse(str: String): TaskLocality = { // better way to do this ? try { - TaskLocality.withName(str) + val retval = TaskLocality.withName(str) + // Must not specify HYPER_LOCAL ! + assert (retval != TaskLocality.HYPER_LOCAL) + + retval } catch { case nEx: NoSuchElementException => { logWarning("Invalid task locality specified '" + str + "', defaulting to HOST_LOCAL"); @@ -133,35 +141,55 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe addPendingTask(i) } - private def findPreferredLocations(_taskPreferredLocations: Seq[String], scheduler: ClusterScheduler, rackLocal: Boolean = false): ArrayBuffer[String] = { - // DEBUG code - _taskPreferredLocations.foreach(h => Utils.checkHost(h, "taskPreferredLocation " + _taskPreferredLocations)) - - val taskPreferredLocations = if (! rackLocal) _taskPreferredLocations else { - // Expand set to include all 'seen' rack local hosts. - // This works since container allocation/management happens within master - so any rack locality information is updated in msater. - // Best case effort, and maybe sort of kludge for now ... rework it later ? - val hosts = new HashSet[String] - _taskPreferredLocations.foreach(h => { - val rackOpt = scheduler.getRackForHost(h) - if (rackOpt.isDefined) { - val hostsOpt = scheduler.getCachedHostsForRack(rackOpt.get) - if (hostsOpt.isDefined) { - hosts ++= hostsOpt.get + // Note that it follows the hierarchy. + // if we search for HOST_LOCAL, the output will include HYPER_LOCAL and + // if we search for RACK_LOCAL, it will include HYPER_LOCAL & HOST_LOCAL + private def findPreferredLocations(_taskPreferredLocations: Seq[String], scheduler: ClusterScheduler, + taskLocality: TaskLocality.TaskLocality): HashSet[String] = { + + if (TaskLocality.HYPER_LOCAL == taskLocality) { + // straight forward comparison ! Special case it. + val retval = new HashSet[String]() + scheduler.synchronized { + for (location <- _taskPreferredLocations) { + if (scheduler.isExecutorAliveOnHostPort(location)) { + retval += location } } + } - // Ensure that irrespective of what scheduler says, host is always added ! - hosts += h - }) - - hosts + return retval } - val retval = new ArrayBuffer[String] + val taskPreferredLocations = + if (TaskLocality.HOST_LOCAL == taskLocality) { + _taskPreferredLocations + } else { + assert (TaskLocality.RACK_LOCAL == taskLocality) + // Expand set to include all 'seen' rack local hosts. + // This works since container allocation/management happens within master - so any rack locality information is updated in msater. + // Best case effort, and maybe sort of kludge for now ... rework it later ? + val hosts = new HashSet[String] + _taskPreferredLocations.foreach(h => { + val rackOpt = scheduler.getRackForHost(h) + if (rackOpt.isDefined) { + val hostsOpt = scheduler.getCachedHostsForRack(rackOpt.get) + if (hostsOpt.isDefined) { + hosts ++= hostsOpt.get + } + } + + // Ensure that irrespective of what scheduler says, host is always added ! + hosts += h + }) + + hosts + } + + val retval = new HashSet[String] scheduler.synchronized { for (prefLocation <- taskPreferredLocations) { - val aliveLocationsOpt = scheduler.getExecutorsAliveOnHost(prefLocation) + val aliveLocationsOpt = scheduler.getExecutorsAliveOnHost(Utils.parseHostPort(prefLocation)._1) if (aliveLocationsOpt.isDefined) { retval ++= aliveLocationsOpt.get } @@ -175,29 +203,37 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe private def addPendingTask(index: Int) { // We can infer hostLocalLocations from rackLocalLocations by joining it against tasks(index).preferredLocations (with appropriate // hostPort <-> host conversion). But not doing it for simplicity sake. If this becomes a performance issue, modify it. - val hostLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched) - val rackLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, true) + val hyperLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HYPER_LOCAL) + val hostLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HOST_LOCAL) + val rackLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) if (rackLocalLocations.size == 0) { // Current impl ensures this. + assert (hyperLocalLocations.size == 0) assert (hostLocalLocations.size == 0) pendingTasksWithNoPrefs += index } else { - // host locality - for (hostPort <- hostLocalLocations) { + // hyper local locality + for (hostPort <- hyperLocalLocations) { // DEBUG Code Utils.checkHostPort(hostPort) val hostPortList = pendingTasksForHostPort.getOrElseUpdate(hostPort, ArrayBuffer()) hostPortList += index + } + + // host locality (includes hyper local) + for (hostPort <- hostLocalLocations) { + // DEBUG Code + Utils.checkHostPort(hostPort) val host = Utils.parseHostPort(hostPort)._1 val hostList = pendingTasksForHost.getOrElseUpdate(host, ArrayBuffer()) hostList += index } - // rack locality + // rack locality (includes hyper local and host local) for (rackLocalHostPort <- rackLocalLocations) { // DEBUG Code Utils.checkHostPort(rackLocalHostPort) @@ -233,6 +269,11 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe pendingRackLocalTasksForHost.getOrElse(host, ArrayBuffer()) } + // Number of pending tasks for a given host Port (which would be hyper local) + def numPendingTasksForHostPort(hostPort: String): Int = { + getPendingTasksForHostPort(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) + } + // Number of pending tasks for a given host (which would be data local) def numPendingTasksForHost(hostPort: String): Int = { getPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) @@ -270,7 +311,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe if (speculatableTasks.size > 0) { val localTask = speculatableTasks.find { index => - val locations = findPreferredLocations(tasks(index).preferredLocations, sched) + val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HOST_LOCAL) val attemptLocs = taskAttempts(index).map(_.hostPort) (locations.size == 0 || locations.contains(hostPort)) && !attemptLocs.contains(hostPort) } @@ -284,7 +325,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { val rackTask = speculatableTasks.find { index => - val locations = findPreferredLocations(tasks(index).preferredLocations, sched, true) + val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) val attemptLocs = taskAttempts(index).map(_.hostPort) locations.contains(hostPort) && !attemptLocs.contains(hostPort) } @@ -311,6 +352,11 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // Dequeue a pending task for a given node and return its index. // If localOnly is set to false, allow non-local tasks as well. private def findTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = { + val hyperLocalTask = findTaskFromList(getPendingTasksForHostPort(hostPort)) + if (hyperLocalTask != None) { + return hyperLocalTask + } + val localTask = findTaskFromList(getPendingTasksForHost(hostPort)) if (localTask != None) { return localTask @@ -341,30 +387,31 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe return findSpeculativeTask(hostPort, locality) } - // Does a host count as a preferred location for a task? This is true if - // either the task has preferred locations and this host is one, or it has - // no preferred locations (in which we still count the launch as preferred). - private def isPreferredLocation(task: Task[_], hostPort: String): Boolean = { + private def isHyperLocalLocation(task: Task[_], hostPort: String): Boolean = { + Utils.checkHostPort(hostPort) + val locs = task.preferredLocations - // DEBUG code - locs.foreach(h => Utils.checkHost(h, "preferredLocation " + locs)) - if (locs.contains(hostPort) || locs.isEmpty) return true + locs.contains(hostPort) + } + + private def isHostLocalLocation(task: Task[_], hostPort: String): Boolean = { + val locs = task.preferredLocations + + // If no preference, consider it as host local + if (locs.isEmpty) return true val host = Utils.parseHostPort(hostPort)._1 - locs.contains(host) + locs.find(h => Utils.parseHostPort(h)._1 == host).isDefined } // Does a host count as a rack local preferred location for a task? (assumes host is NOT preferred location). // This is true if either the task has preferred locations and this host is one, or it has // no preferred locations (in which we still count the launch as preferred). - def isRackLocalLocation(task: Task[_], hostPort: String): Boolean = { + private def isRackLocalLocation(task: Task[_], hostPort: String): Boolean = { val locs = task.preferredLocations - // DEBUG code - locs.foreach(h => Utils.checkHost(h, "preferredLocation " + locs)) - val preferredRacks = new HashSet[String]() for (preferredHost <- locs) { val rack = sched.getRackForHost(preferredHost) @@ -395,8 +442,11 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe val task = tasks(index) val taskId = sched.newTaskId() // Figure out whether this should count as a preferred launch - val taskLocality = if (isPreferredLocation(task, hostPort)) TaskLocality.HOST_LOCAL else - if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL else TaskLocality.ANY + val taskLocality = + if (isHyperLocalLocation(task, hostPort)) TaskLocality.HYPER_LOCAL else + if (isHostLocalLocation(task, hostPort)) TaskLocality.HOST_LOCAL else + if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL else + TaskLocality.ANY val prefStr = taskLocality.toString logInfo("Starting task %s:%d as TID %s on slave %s: %s (%s)".format( taskSet.id, index, taskId, execId, hostPort, prefStr)) @@ -552,15 +602,22 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe def executorLost(execId: String, hostPort: String) { logInfo("Re-queueing tasks for " + execId + " from TaskSet " + taskSet.id) + // If some task has preferred locations only on hostname, and there are no more executors there, // put it in the no-prefs list to avoid the wait from delay scheduling - for (index <- getPendingTasksForHostPort(hostPort)) { - val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, true) + + // host local tasks - should we push this to rack local or no pref list ? For now, preserving behavior and moving to + // no prefs list. Note, this was done due to impliations related to 'waiting' for data local tasks, etc. + // Note: NOT checking hyper local list - since host local list is super set of that. We need to ad to no prefs only if + // there is no host local node for the task (not if there is no hyper local node for the task) + for (index <- getPendingTasksForHost(Utils.parseHostPort(hostPort)._1)) { + // val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) + val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HOST_LOCAL) if (newLocs.isEmpty) { - assert (findPreferredLocations(tasks(index).preferredLocations, sched).isEmpty) pendingTasksWithNoPrefs += index } } + // Re-enqueue any tasks that ran on the failed executor if this is a shuffle map stage if (tasks(0).isInstanceOf[ShuffleMapTask]) { for ((tid, info) <- taskInfos if info.executorId == execId) { diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 6e861ac734..7a0d6ced3e 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -4,7 +4,7 @@ import java.io.{InputStream, OutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue} -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} +import scala.collection.mutable.{HashMap, ArrayBuffer, HashSet, Queue} import scala.collection.JavaConversions._ import akka.actor.{ActorSystem, Cancellable, Props} @@ -271,23 +271,12 @@ class BlockManager( } - /** - * Get locations of the block. - */ - def getLocations(blockId: String): Seq[String] = { - val startTimeMs = System.currentTimeMillis - var managers = master.getLocations(blockId) - val locations = managers.map(_.hostPort) - logDebug("Got block locations in " + Utils.getUsedTimeMs(startTimeMs)) - return locations - } - /** * Get locations of an array of blocks. */ - def getLocations(blockIds: Array[String]): Array[Seq[String]] = { + def getLocationBlockIds(blockIds: Array[String]): Array[Seq[BlockManagerId]] = { val startTimeMs = System.currentTimeMillis - val locations = master.getLocations(blockIds).map(_.map(_.hostPort).toSeq).toArray + val locations = master.getLocations(blockIds).toArray logDebug("Got multiple block location in " + Utils.getUsedTimeMs(startTimeMs)) return locations } @@ -947,6 +936,32 @@ object BlockManager extends Logging { } } } + + def blockIdsToExecutorLocations(blockIds: Array[String], env: SparkEnv): HashMap[String, List[String]] = { + val blockManager = env.blockManager + /*val locations = blockIds.map(id => blockManager.getLocations(id))*/ + val locationBlockIds = blockManager.getLocationBlockIds(blockIds) + + // Convert from block master locations to executor locations (we need that for task scheduling) + val executorLocations = new HashMap[String, List[String]]() + for (i <- 0 until blockIds.length) { + val blockId = blockIds(i) + val blockLocations = locationBlockIds(i) + + val executors = new HashSet[String]() + + for (bkLocation <- blockLocations) { + val executorHostPort = env.resolveExecutorIdToHostPort(bkLocation.executorId, bkLocation.host) + executors += executorHostPort + // logInfo("bkLocation = " + bkLocation + ", executorHostPort = " + executorHostPort) + } + + executorLocations.put(blockId, executors.toSeq.toList) + } + + executorLocations + } + } class BlockFetcherIterator( diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index 3abc584b6a..875975ca43 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -80,12 +80,14 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { } test("remote fetch") { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", "localhost", 0) + val hostname = "localhost" + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0) + System.setProperty("spark.hostPort", hostname + ":" + boundPort) val masterTracker = new MapOutputTracker() masterTracker.trackerActor = actorSystem.actorOf( Props(new MapOutputTrackerActor(masterTracker)), "MapOutputTracker") - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", "localhost", 0) + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0) val slaveTracker = new MapOutputTracker() slaveTracker.trackerActor = slaveSystem.actorFor( "akka://spark@localhost:" + boundPort + "/user/MapOutputTracker") diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index c0f8986de8..16554eac6e 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -385,12 +385,12 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assert(results === Map(0 -> 42)) } - /** Assert that the supplied TaskSet has exactly the given preferredLocations. */ + /** Assert that the supplied TaskSet has exactly the given preferredLocations. Note, converts taskSet's locations to host only. */ private def assertLocations(taskSet: TaskSet, locations: Seq[Seq[String]]) { assert(locations.size === taskSet.tasks.size) for ((expectLocs, taskLocs) <- taskSet.tasks.map(_.preferredLocations).zip(locations)) { - assert(expectLocs === taskLocs) + assert(expectLocs.map(loc => spark.Utils.parseHostPort(loc)._1) === taskLocs) } } diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index b8c0f6fb76..3fc2825255 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -41,6 +41,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT oldHeartBeat = System.setProperty("spark.storage.disableBlockManagerHeartBeat", "true") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() + // Set some value ... + System.setProperty("spark.hostPort", spark.Utils.localHostName() + ":" + 1111) } after { -- cgit v1.2.3 From 27764a00f40391b94fa05abb11484c442607f6f7 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 1 May 2013 20:56:05 +0530 Subject: Fix some npe introduced accidentally --- .../main/scala/spark/scheduler/DAGScheduler.scala | 2 +- .../main/scala/spark/storage/BlockManager.scala | 30 ++++++++++++++++------ 2 files changed, 23 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 8072c60bb7..b18248d2b5 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -117,7 +117,7 @@ class DAGScheduler( private def getCacheLocs(rdd: RDD[_]): Array[List[String]] = { if (!cacheLocs.contains(rdd.id)) { val blockIds = rdd.partitions.indices.map(index=> "rdd_%d_%d".format(rdd.id, index)).toArray - val locs = BlockManager.blockIdsToExecutorLocations(blockIds, env) + val locs = BlockManager.blockIdsToExecutorLocations(blockIds, env, blockManagerMaster) cacheLocs(rdd.id) = blockIds.map(locs.getOrElse(_, Nil)) } cacheLocs(rdd.id) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 7a0d6ced3e..040082e600 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -937,10 +937,16 @@ object BlockManager extends Logging { } } - def blockIdsToExecutorLocations(blockIds: Array[String], env: SparkEnv): HashMap[String, List[String]] = { - val blockManager = env.blockManager - /*val locations = blockIds.map(id => blockManager.getLocations(id))*/ - val locationBlockIds = blockManager.getLocationBlockIds(blockIds) + def blockIdsToExecutorLocations(blockIds: Array[String], env: SparkEnv, blockManagerMaster: BlockManagerMaster = null): HashMap[String, List[String]] = { + // env == null and blockManagerMaster != null is used in tests + assert (env != null || blockManagerMaster != null) + val locationBlockIds: Seq[Seq[BlockManagerId]] = + if (env != null) { + val blockManager = env.blockManager + blockManager.getLocationBlockIds(blockIds) + } else { + blockManagerMaster.getLocations(blockIds) + } // Convert from block master locations to executor locations (we need that for task scheduling) val executorLocations = new HashMap[String, List[String]]() @@ -950,10 +956,18 @@ object BlockManager extends Logging { val executors = new HashSet[String]() - for (bkLocation <- blockLocations) { - val executorHostPort = env.resolveExecutorIdToHostPort(bkLocation.executorId, bkLocation.host) - executors += executorHostPort - // logInfo("bkLocation = " + bkLocation + ", executorHostPort = " + executorHostPort) + if (env != null) { + for (bkLocation <- blockLocations) { + val executorHostPort = env.resolveExecutorIdToHostPort(bkLocation.executorId, bkLocation.host) + executors += executorHostPort + // logInfo("bkLocation = " + bkLocation + ", executorHostPort = " + executorHostPort) + } + } else { + // Typically while testing, etc - revert to simply using host. + for (bkLocation <- blockLocations) { + executors += bkLocation.host + // logInfo("bkLocation = " + bkLocation + ", executorHostPort = " + executorHostPort) + } } executorLocations.put(blockId, executors.toSeq.toList) -- cgit v1.2.3 From 848156273178bed5763bcbc91baa788bd4a57f6e Mon Sep 17 00:00:00 2001 From: harshars Date: Mon, 25 Mar 2013 20:09:07 -0700 Subject: Merged Ram's commit on removing RDDs. Conflicts: core/src/main/scala/spark/SparkContext.scala --- core/src/main/scala/spark/SparkContext.scala | 62 +++++++++++++++--------- core/src/test/scala/spark/DistributedSuite.scala | 12 +++++ core/src/test/scala/spark/RDDSuite.scala | 7 +++ 3 files changed, 59 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 5f5ec0b0f4..8bee1d65a2 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -1,47 +1,50 @@ package spark import java.io._ -import java.util.concurrent.atomic.AtomicInteger import java.net.URI +import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.atomic.AtomicInteger +import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.generic.Growable -import scala.collection.mutable.HashMap -import scala.collection.JavaConversions._ +import scala.collection.mutable.{ConcurrentMap, HashMap} + +import akka.actor.Actor._ -import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.mapred.InputFormat -import org.apache.hadoop.mapred.SequenceFileInputFormat -import org.apache.hadoop.io.Writable -import org.apache.hadoop.io.IntWritable -import org.apache.hadoop.io.LongWritable -import org.apache.hadoop.io.FloatWritable -import org.apache.hadoop.io.DoubleWritable +import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.ArrayWritable import org.apache.hadoop.io.BooleanWritable import org.apache.hadoop.io.BytesWritable -import org.apache.hadoop.io.ArrayWritable +import org.apache.hadoop.io.DoubleWritable +import org.apache.hadoop.io.FloatWritable +import org.apache.hadoop.io.IntWritable +import org.apache.hadoop.io.LongWritable import org.apache.hadoop.io.NullWritable import org.apache.hadoop.io.Text +import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.FileInputFormat +import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapred.SequenceFileInputFormat import org.apache.hadoop.mapred.TextInputFormat import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} -import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.hadoop.mapreduce.{Job => NewHadoopJob} +import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} + import org.apache.mesos.MesosNativeLibrary -import spark.deploy.{SparkHadoopUtil, LocalSparkCluster} -import spark.partial.ApproximateEvaluator -import spark.partial.PartialResult +import spark.deploy.{LocalSparkCluster, SparkHadoopUtil} +import spark.partial.{ApproximateEvaluator, PartialResult} import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD} -import spark.scheduler._ +import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler} +import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler} import spark.scheduler.local.LocalScheduler -import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler} import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import spark.storage.BlockManagerUI +import spark.storage.{BlockManagerUI, StorageStatus, StorageUtils, RDDInfo} import spark.util.{MetadataCleaner, TimeStampedHashMap} -import spark.storage.{StorageStatus, StorageUtils, RDDInfo} + /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -97,7 +100,7 @@ class SparkContext( private[spark] val addedJars = HashMap[String, Long]() // Keeps track of all persisted RDDs - private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]]() + private[spark] val persistentRdds: ConcurrentMap[Int, RDD[_]] = new ConcurrentHashMap[Int, RDD[_]]() private[spark] val metadataCleaner = new MetadataCleaner("SparkContext", this.cleanup) @@ -520,6 +523,21 @@ class SparkContext( env.blockManager.master.getStorageStatus } + def removeRDD(id: Int): Unit = { + val storageStatusList = getExecutorStorageStatus + val groupedRddBlocks = storageStatusList.flatMap(_.blocks).toMap + logInfo("RDD to remove: " + id) + groupedRddBlocks.foreach(x => { + val k = x._1.substring(0,x._1.lastIndexOf('_')) + val rdd_id = "rdd_" + id + logInfo("RDD to check: " + rdd_id) + if(k.equals(rdd_id)) { + env.blockManager.master.removeBlock(x._1) + } + }) + persistentRdds.remove(id) + } + /** * Clear the job's list of files added by `addFile` so that they do not get downloaded to * any new nodes. @@ -743,7 +761,7 @@ class SparkContext( /** Called by MetadataCleaner to clean up the persistentRdds map periodically */ private[spark] def cleanup(cleanupTime: Long) { - persistentRdds.clearOldValues(cleanupTime) + // do nothing. this needs to be removed. } } diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index c9b4707def..c7f6ab3133 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -252,6 +252,18 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter assert(data2.count === 2) } } + + test("remove RDDs cleanly") { + DistributedSuite.amMaster = true + sc = new SparkContext("local-cluster[3,1,512]", "test") + val data = sc.parallelize(Seq(true, false, false, false), 4) + data.persist(StorageLevel.MEMORY_ONLY_2) + data.count + sc.removeRDD(data.id) + assert(sc.persistentRdds.isEmpty == true) + assert(sc.getRDDStorageInfo.isEmpty == true) + + } } object DistributedSuite { diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 7fbdd44340..88b7ab9f52 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -100,6 +100,13 @@ class RDDSuite extends FunSuite with LocalSparkContext { assert(rdd.collect().toList === List(1, 2, 3, 4)) } + test("remove RDD") { + sc = new SparkContext("local", "test") + val rdd = sc.makeRDD(Array(1,2,3,4), 2).cache() + sc.removeRDD(rdd.id) + assert(sc.persistentRdds.empty == true) + } + test("caching with failures") { sc = new SparkContext("local", "test") val onlySplit = new Partition { override def index: Int = 0 } -- cgit v1.2.3 From 3227ec8edde05cff27c1f9de8861d18b3cda1aae Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 1 May 2013 16:07:44 -0700 Subject: Cleaned up Ram's code. Moved SparkContext.remove to RDD.unpersist. Also updated unit tests to make sure they are properly testing for concurrency. --- core/src/main/scala/spark/RDD.scala | 17 ++++++++++++ core/src/main/scala/spark/SparkContext.scala | 25 ++++-------------- .../main/scala/spark/storage/BlockManagerUI.scala | 4 +-- core/src/test/scala/spark/DistributedSuite.scala | 30 ++++++++++++++++------ core/src/test/scala/spark/RDDSuite.scala | 27 +++++++++++++++---- 5 files changed, 68 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 09e52ebf3e..c77f9915c0 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -140,6 +140,23 @@ abstract class RDD[T: ClassManifest]( /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ def cache(): RDD[T] = persist() + /** Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. */ + def unpersist(): RDD[T] = { + logInfo("Removing RDD " + id + " from persistence list") + val rddBlockPrefix = "rdd_" + id + "_" + // Get the list of blocks in block manager, and remove ones that are part of this RDD. + // The runtime complexity is linear to the number of blocks persisted in the cluster. + // It could be expensive if the cluster is large and has a lot of blocks persisted. + sc.getExecutorStorageStatus().flatMap(_.blocks).foreach { case(blockId, status) => + if (blockId.startsWith(rddBlockPrefix)) { + sc.env.blockManager.master.removeBlock(blockId) + } + } + sc.persistentRdds.remove(id) + storageLevel = StorageLevel.NONE + this + } + /** Get the RDD's current storage level, or StorageLevel.NONE if none is set. */ def getStorageLevel = storageLevel diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 8bee1d65a2..b686c595b8 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -100,7 +100,7 @@ class SparkContext( private[spark] val addedJars = HashMap[String, Long]() // Keeps track of all persisted RDDs - private[spark] val persistentRdds: ConcurrentMap[Int, RDD[_]] = new ConcurrentHashMap[Int, RDD[_]]() + private[spark] val persistentRdds: ConcurrentMap[Int, RDD[_]] = new ConcurrentHashMap[Int, RDD[_]] private[spark] val metadataCleaner = new MetadataCleaner("SparkContext", this.cleanup) @@ -508,36 +508,21 @@ class SparkContext( * Return information about what RDDs are cached, if they are in mem or on disk, how much space * they take, etc. */ - def getRDDStorageInfo : Array[RDDInfo] = { - StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this) + def getRDDStorageInfo(): Array[RDDInfo] = { + StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus(), this) } - def getStageInfo: Map[Stage,StageInfo] = { + def getStageInfo(): Map[Stage,StageInfo] = { dagScheduler.stageToInfos } /** * Return information about blocks stored in all of the slaves */ - def getExecutorStorageStatus : Array[StorageStatus] = { + def getExecutorStorageStatus(): Array[StorageStatus] = { env.blockManager.master.getStorageStatus } - def removeRDD(id: Int): Unit = { - val storageStatusList = getExecutorStorageStatus - val groupedRddBlocks = storageStatusList.flatMap(_.blocks).toMap - logInfo("RDD to remove: " + id) - groupedRddBlocks.foreach(x => { - val k = x._1.substring(0,x._1.lastIndexOf('_')) - val rdd_id = "rdd_" + id - logInfo("RDD to check: " + rdd_id) - if(k.equals(rdd_id)) { - env.blockManager.master.removeBlock(x._1) - } - }) - persistentRdds.remove(id) - } - /** * Clear the job's list of files added by `addFile` so that they do not get downloaded to * any new nodes. diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index 07da572044..c9e4519efe 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -45,7 +45,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, path("") { completeWith { // Request the current storage status from the Master - val storageStatusList = sc.getExecutorStorageStatus + val storageStatusList = sc.getExecutorStorageStatus() // Calculate macro-level statistics val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) @@ -60,7 +60,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, parameter("id") { id => completeWith { val prefix = "rdd_" + id.toString - val storageStatusList = sc.getExecutorStorageStatus + val storageStatusList = sc.getExecutorStorageStatus() val filteredStorageStatusList = StorageUtils. filterStorageStatusByPrefix(storageStatusList, prefix) val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index c7f6ab3133..ab3e197035 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -3,8 +3,10 @@ package spark import network.ConnectionManagerId import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter +import org.scalatest.concurrent.Timeouts._ import org.scalatest.matchers.ShouldMatchers import org.scalatest.prop.Checkers +import org.scalatest.time.{Span, Millis} import org.scalacheck.Arbitrary._ import org.scalacheck.Gen import org.scalacheck.Prop._ @@ -252,24 +254,36 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter assert(data2.count === 2) } } - - test("remove RDDs cleanly") { + + test("unpersist RDDs") { DistributedSuite.amMaster = true sc = new SparkContext("local-cluster[3,1,512]", "test") val data = sc.parallelize(Seq(true, false, false, false), 4) data.persist(StorageLevel.MEMORY_ONLY_2) data.count - sc.removeRDD(data.id) + assert(sc.persistentRdds.isEmpty == false) + data.unpersist() assert(sc.persistentRdds.isEmpty == true) + + failAfter(Span(3000, Millis)) { + try { + while (! sc.getRDDStorageInfo.isEmpty) { + Thread.sleep(200) + } + } catch { + case e: Exception => + // Do nothing. We might see exceptions because block manager + // is racing this thread to remove entries from the driver. + } + } assert(sc.getRDDStorageInfo.isEmpty == true) - } } object DistributedSuite { // Indicates whether this JVM is marked for failure. var mark = false - + // Set by test to remember if we are in the driver program so we can assert // that we are not. var amMaster = false @@ -286,9 +300,9 @@ object DistributedSuite { // Act like an identity function, but if mark was set to true previously, fail, // crashing the entire JVM. def failOnMarkedIdentity(item: Boolean): Boolean = { - if (mark) { + if (mark) { System.exit(42) - } + } item - } + } } diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 88b7ab9f52..cee6312572 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -2,6 +2,8 @@ package spark import scala.collection.mutable.HashMap import org.scalatest.FunSuite +import org.scalatest.concurrent.Timeouts._ +import org.scalatest.time.{Span, Millis} import spark.SparkContext._ import spark.rdd.{CoalescedRDD, CoGroupedRDD, PartitionPruningRDD, ShuffledRDD} @@ -100,11 +102,26 @@ class RDDSuite extends FunSuite with LocalSparkContext { assert(rdd.collect().toList === List(1, 2, 3, 4)) } - test("remove RDD") { - sc = new SparkContext("local", "test") - val rdd = sc.makeRDD(Array(1,2,3,4), 2).cache() - sc.removeRDD(rdd.id) - assert(sc.persistentRdds.empty == true) + test("unpersist RDD") { + sc = new SparkContext("local", "test") + val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() + rdd.count + assert(sc.persistentRdds.isEmpty == false) + rdd.unpersist() + assert(sc.persistentRdds.isEmpty == true) + + failAfter(Span(3000, Millis)) { + try { + while (! sc.getRDDStorageInfo.isEmpty) { + Thread.sleep(200) + } + } catch { + case e: Exception => + // Do nothing. We might see exceptions because block manager + // is racing this thread to remove entries from the driver. + } + } + assert(sc.getRDDStorageInfo.isEmpty == true) } test("caching with failures") { -- cgit v1.2.3 From 34637b97ec7ebdd356653324f15345b00b3a2ac2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 1 May 2013 16:12:37 -0700 Subject: Added SparkContext.cleanup back. Not sure why it was removed before ... --- core/src/main/scala/spark/SparkContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index b686c595b8..401e55d615 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -746,7 +746,7 @@ class SparkContext( /** Called by MetadataCleaner to clean up the persistentRdds map periodically */ private[spark] def cleanup(cleanupTime: Long) { - // do nothing. this needs to be removed. + persistentRdds.clearOldValues(cleanupTime) } } -- cgit v1.2.3 From 204eb32e14e8fce5e4b4cf602375ae9b4ed136c9 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 1 May 2013 16:14:58 -0700 Subject: Changed the type of the persistentRdds hashmap back to TimeStampedHashMap. --- core/src/main/scala/spark/SparkContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 401e55d615..d7d450d958 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -100,7 +100,7 @@ class SparkContext( private[spark] val addedJars = HashMap[String, Long]() // Keeps track of all persisted RDDs - private[spark] val persistentRdds: ConcurrentMap[Int, RDD[_]] = new ConcurrentHashMap[Int, RDD[_]] + private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]] private[spark] val metadataCleaner = new MetadataCleaner("SparkContext", this.cleanup) -- cgit v1.2.3 From 207afe4088219a0c7350b3f80eb60e86c97e140f Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Thu, 18 Apr 2013 12:08:11 -0700 Subject: Remove spark-repl's extraneous dependency on spark-streaming --- project/SparkBuild.scala | 2 +- repl/pom.xml | 14 -------------- 2 files changed, 1 insertion(+), 15 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f2410085d8..190d723435 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -29,7 +29,7 @@ object SparkBuild extends Build { lazy val core = Project("core", file("core"), settings = coreSettings) - lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) dependsOn (streaming) + lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) dependsOn (streaming) diff --git a/repl/pom.xml b/repl/pom.xml index 038da5d988..92a2020b48 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -96,13 +96,6 @@ hadoop1 runtime - - org.spark-project - spark-streaming - ${project.version} - hadoop1 - runtime - org.apache.hadoop hadoop-core @@ -147,13 +140,6 @@ hadoop2 runtime - - org.spark-project - spark-streaming - ${project.version} - hadoop2 - runtime - org.apache.hadoop hadoop-core -- cgit v1.2.3 From 609a817f52d8db05711c0d4529dd1448ed8c4fe0 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Thu, 2 May 2013 06:44:33 +0530 Subject: Integrate review comments on pull request --- core/src/main/scala/spark/SparkEnv.scala | 9 ++-- core/src/main/scala/spark/Utils.scala | 4 +- .../main/scala/spark/scheduler/ResultTask.scala | 1 - .../scala/spark/scheduler/ShuffleMapTask.scala | 1 - .../spark/scheduler/cluster/ClusterScheduler.scala | 32 +++++++------- .../spark/scheduler/cluster/TaskSetManager.scala | 50 +++++++++++----------- 6 files changed, 47 insertions(+), 50 deletions(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 5b4a464010..2ee25e547d 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -33,8 +33,7 @@ class SparkEnv ( // To be set only as part of initialization of SparkContext. // (executorId, defaultHostPort) => executorHostPort // If executorId is NOT found, return defaultHostPort - var executorIdToHostPort: (String, String) => String - ) { + var executorIdToHostPort: Option[(String, String) => String]) { def stop() { httpFileServer.stop() @@ -52,12 +51,12 @@ class SparkEnv ( def resolveExecutorIdToHostPort(executorId: String, defaultHostPort: String): String = { val env = SparkEnv.get - if (env.executorIdToHostPort == null) { + if (env.executorIdToHostPort.isEmpty) { // default to using host, not host port. Relevant to non cluster modes. return defaultHostPort } - env.executorIdToHostPort(executorId, defaultHostPort) + env.executorIdToHostPort.get(executorId, defaultHostPort) } } @@ -178,6 +177,6 @@ object SparkEnv extends Logging { connectionManager, httpFileServer, sparkFilesDir, - null) + None) } } diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 279daf04ed..0e348f8189 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -335,7 +335,6 @@ private object Utils extends Logging { retval } - /* // Used by DEBUG code : remove when all testing done private val ipPattern = Pattern.compile("^[0-9]+(\\.[0-9]+)*$") def checkHost(host: String, message: String = "") { @@ -364,8 +363,8 @@ private object Utils extends Logging { // temp code for debug System.exit(-1) } - */ +/* // Once testing is complete in various modes, replace with this ? def checkHost(host: String, message: String = "") {} def checkHostPort(hostPort: String, message: String = "") {} @@ -374,6 +373,7 @@ private object Utils extends Logging { def logErrorWithStack(msg: String) { try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } } } +*/ def getUserNameFromEnvironment(): String = { SparkHadoopUtil.getUserNameFromEnvironment diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala index c43cbe5ed4..83166bce22 100644 --- a/core/src/main/scala/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/spark/scheduler/ResultTask.scala @@ -70,7 +70,6 @@ private[spark] class ResultTask[T, U]( rdd.partitions(partition) } - // data locality is on a per host basis, not hyper specific to container (host:port). Unique on set of hosts. private val preferredLocs: Seq[String] = if (locs == null) Nil else locs.toSet.toSeq { diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 0b848af2f3..4b36e71c32 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -84,7 +84,6 @@ private[spark] class ShuffleMapTask( protected def this() = this(0, null, null, 0, null) - // data locality is on a per host basis, not hyper specific to container (host:port). Unique on set of hosts. private val preferredLocs: Seq[String] = if (locs == null) Nil else locs.toSet.toSeq { diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 3c72ce4206..49fc449e86 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -73,7 +73,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val activeExecutorIds = new HashSet[String] // TODO: We might want to remove this and merge it with execId datastructures - but later. - // Which hosts in the cluster are alive (contains hostPort's) - used for hyper local and local task locality. + // Which hosts in the cluster are alive (contains hostPort's) - used for instance local and node local task locality. private val hostPortsAlive = new HashSet[String] private val hostToAliveHostPorts = new HashMap[String, HashSet[String]] @@ -109,7 +109,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // Unfortunately, this means that SparkEnv is indirectly referencing ClusterScheduler // Will that be a design violation ? - SparkEnv.get.executorIdToHostPort = executorToHostPort + SparkEnv.get.executorIdToHostPort = Some(executorToHostPort) } def newTaskId(): Long = nextTaskId.getAndIncrement() @@ -240,7 +240,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) for (manager <- activeTaskSetsQueue.sortBy(m => (m.taskSet.priority, m.taskSet.stageId))) { // Split offers based on host local, rack local and off-rack tasks. - val hyperLocalOffers = new HashMap[String, ArrayBuffer[Int]]() + val instanceLocalOffers = new HashMap[String, ArrayBuffer[Int]]() val hostLocalOffers = new HashMap[String, ArrayBuffer[Int]]() val rackLocalOffers = new HashMap[String, ArrayBuffer[Int]]() val otherOffers = new HashMap[String, ArrayBuffer[Int]]() @@ -250,16 +250,16 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // DEBUG code Utils.checkHostPort(hostPort) - val numHyperLocalTasks = math.max(0, math.min(manager.numPendingTasksForHostPort(hostPort), availableCpus(i))) - if (numHyperLocalTasks > 0){ - val list = hyperLocalOffers.getOrElseUpdate(hostPort, new ArrayBuffer[Int]) - for (j <- 0 until numHyperLocalTasks) list += i + val numInstanceLocalTasks = math.max(0, math.min(manager.numPendingTasksForHostPort(hostPort), availableCpus(i))) + if (numInstanceLocalTasks > 0){ + val list = instanceLocalOffers.getOrElseUpdate(hostPort, new ArrayBuffer[Int]) + for (j <- 0 until numInstanceLocalTasks) list += i } val host = Utils.parseHostPort(hostPort)._1 val numHostLocalTasks = math.max(0, - // Remove hyper local tasks (which are also host local btw !) from this - math.min(manager.numPendingTasksForHost(hostPort) - numHyperLocalTasks, hostToAvailableCpus(host))) + // Remove instance local tasks (which are also host local btw !) from this + math.min(manager.numPendingTasksForHost(hostPort) - numInstanceLocalTasks, hostToAvailableCpus(host))) if (numHostLocalTasks > 0){ val list = hostLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) for (j <- 0 until numHostLocalTasks) list += i @@ -267,7 +267,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val numRackLocalTasks = math.max(0, // Remove host local tasks (which are also rack local btw !) from this - math.min(manager.numRackLocalPendingTasksForHost(hostPort) - numHyperLocalTasks - numHostLocalTasks, hostToAvailableCpus(host))) + math.min(manager.numRackLocalPendingTasksForHost(hostPort) - numInstanceLocalTasks - numHostLocalTasks, hostToAvailableCpus(host))) if (numRackLocalTasks > 0){ val list = rackLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) for (j <- 0 until numRackLocalTasks) list += i @@ -280,19 +280,19 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } val offersPriorityList = new ArrayBuffer[Int]( - hyperLocalOffers.size + hostLocalOffers.size + rackLocalOffers.size + otherOffers.size) + instanceLocalOffers.size + hostLocalOffers.size + rackLocalOffers.size + otherOffers.size) - // First hyper local, then host local, then rack, then others + // First instance local, then host local, then rack, then others - // numHostLocalOffers contains count of both hyper local and host offers. + // numHostLocalOffers contains count of both instance local and host offers. val numHostLocalOffers = { - val hyperLocalPriorityList = ClusterScheduler.prioritizeContainers(hyperLocalOffers) - offersPriorityList ++= hyperLocalPriorityList + val instanceLocalPriorityList = ClusterScheduler.prioritizeContainers(instanceLocalOffers) + offersPriorityList ++= instanceLocalPriorityList val hostLocalPriorityList = ClusterScheduler.prioritizeContainers(hostLocalOffers) offersPriorityList ++= hostLocalPriorityList - hyperLocalPriorityList.size + hostLocalPriorityList.size + instanceLocalPriorityList.size + hostLocalPriorityList.size } val numRackLocalOffers = { val rackLocalPriorityList = ClusterScheduler.prioritizeContainers(rackLocalOffers) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index f5c0058554..5f3faaa5c3 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -13,17 +13,17 @@ import spark.scheduler._ import spark.TaskState.TaskState import java.nio.ByteBuffer -private[spark] object TaskLocality extends Enumeration("HYPER_LOCAL", "HOST_LOCAL", "RACK_LOCAL", "ANY") with Logging { +private[spark] object TaskLocality extends Enumeration("INSTANCE_LOCAL", "HOST_LOCAL", "RACK_LOCAL", "ANY") with Logging { - // hyper local is expected to be used ONLY within tasksetmanager for now. - val HYPER_LOCAL, HOST_LOCAL, RACK_LOCAL, ANY = Value + // instance local is expected to be used ONLY within tasksetmanager for now. + val INSTANCE_LOCAL, HOST_LOCAL, RACK_LOCAL, ANY = Value type TaskLocality = Value def isAllowed(constraint: TaskLocality, condition: TaskLocality): Boolean = { // Must not be the constraint. - assert (constraint != TaskLocality.HYPER_LOCAL) + assert (constraint != TaskLocality.INSTANCE_LOCAL) constraint match { case TaskLocality.HOST_LOCAL => condition == TaskLocality.HOST_LOCAL @@ -37,8 +37,8 @@ private[spark] object TaskLocality extends Enumeration("HYPER_LOCAL", "HOST_LOCA // better way to do this ? try { val retval = TaskLocality.withName(str) - // Must not specify HYPER_LOCAL ! - assert (retval != TaskLocality.HYPER_LOCAL) + // Must not specify INSTANCE_LOCAL ! + assert (retval != TaskLocality.INSTANCE_LOCAL) retval } catch { @@ -84,7 +84,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis - // List of pending tasks for each node (hyper local to container). These collections are actually + // List of pending tasks for each node (instance local to container). These collections are actually // treated as stacks, in which new tasks are added to the end of the // ArrayBuffer and removed from the end. This makes it faster to detect // tasks that repeatedly fail because whenever a task failed, it is put @@ -142,12 +142,12 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe } // Note that it follows the hierarchy. - // if we search for HOST_LOCAL, the output will include HYPER_LOCAL and - // if we search for RACK_LOCAL, it will include HYPER_LOCAL & HOST_LOCAL + // if we search for HOST_LOCAL, the output will include INSTANCE_LOCAL and + // if we search for RACK_LOCAL, it will include INSTANCE_LOCAL & HOST_LOCAL private def findPreferredLocations(_taskPreferredLocations: Seq[String], scheduler: ClusterScheduler, taskLocality: TaskLocality.TaskLocality): HashSet[String] = { - if (TaskLocality.HYPER_LOCAL == taskLocality) { + if (TaskLocality.INSTANCE_LOCAL == taskLocality) { // straight forward comparison ! Special case it. val retval = new HashSet[String]() scheduler.synchronized { @@ -203,19 +203,19 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe private def addPendingTask(index: Int) { // We can infer hostLocalLocations from rackLocalLocations by joining it against tasks(index).preferredLocations (with appropriate // hostPort <-> host conversion). But not doing it for simplicity sake. If this becomes a performance issue, modify it. - val hyperLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HYPER_LOCAL) + val instanceLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.INSTANCE_LOCAL) val hostLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HOST_LOCAL) val rackLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) if (rackLocalLocations.size == 0) { // Current impl ensures this. - assert (hyperLocalLocations.size == 0) + assert (instanceLocalLocations.size == 0) assert (hostLocalLocations.size == 0) pendingTasksWithNoPrefs += index } else { - // hyper local locality - for (hostPort <- hyperLocalLocations) { + // instance local locality + for (hostPort <- instanceLocalLocations) { // DEBUG Code Utils.checkHostPort(hostPort) @@ -223,7 +223,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe hostPortList += index } - // host locality (includes hyper local) + // host locality (includes instance local) for (hostPort <- hostLocalLocations) { // DEBUG Code Utils.checkHostPort(hostPort) @@ -233,7 +233,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe hostList += index } - // rack locality (includes hyper local and host local) + // rack locality (includes instance local and host local) for (rackLocalHostPort <- rackLocalLocations) { // DEBUG Code Utils.checkHostPort(rackLocalHostPort) @@ -247,7 +247,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe allPendingTasks += index } - // Return the pending tasks list for a given host port (hyper local), or an empty list if + // Return the pending tasks list for a given host port (instance local), or an empty list if // there is no map entry for that host private def getPendingTasksForHostPort(hostPort: String): ArrayBuffer[Int] = { // DEBUG Code @@ -269,7 +269,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe pendingRackLocalTasksForHost.getOrElse(host, ArrayBuffer()) } - // Number of pending tasks for a given host Port (which would be hyper local) + // Number of pending tasks for a given host Port (which would be instance local) def numPendingTasksForHostPort(hostPort: String): Int = { getPendingTasksForHostPort(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) } @@ -352,9 +352,9 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // Dequeue a pending task for a given node and return its index. // If localOnly is set to false, allow non-local tasks as well. private def findTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = { - val hyperLocalTask = findTaskFromList(getPendingTasksForHostPort(hostPort)) - if (hyperLocalTask != None) { - return hyperLocalTask + val instanceLocalTask = findTaskFromList(getPendingTasksForHostPort(hostPort)) + if (instanceLocalTask != None) { + return instanceLocalTask } val localTask = findTaskFromList(getPendingTasksForHost(hostPort)) @@ -387,7 +387,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe return findSpeculativeTask(hostPort, locality) } - private def isHyperLocalLocation(task: Task[_], hostPort: String): Boolean = { + private def isInstanceLocalLocation(task: Task[_], hostPort: String): Boolean = { Utils.checkHostPort(hostPort) val locs = task.preferredLocations @@ -443,7 +443,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe val taskId = sched.newTaskId() // Figure out whether this should count as a preferred launch val taskLocality = - if (isHyperLocalLocation(task, hostPort)) TaskLocality.HYPER_LOCAL else + if (isInstanceLocalLocation(task, hostPort)) TaskLocality.INSTANCE_LOCAL else if (isHostLocalLocation(task, hostPort)) TaskLocality.HOST_LOCAL else if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL else TaskLocality.ANY @@ -608,8 +608,8 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // host local tasks - should we push this to rack local or no pref list ? For now, preserving behavior and moving to // no prefs list. Note, this was done due to impliations related to 'waiting' for data local tasks, etc. - // Note: NOT checking hyper local list - since host local list is super set of that. We need to ad to no prefs only if - // there is no host local node for the task (not if there is no hyper local node for the task) + // Note: NOT checking instance local list - since host local list is super set of that. We need to ad to no prefs only if + // there is no host local node for the task (not if there is no instance local node for the task) for (index <- getPendingTasksForHost(Utils.parseHostPort(hostPort)._1)) { // val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HOST_LOCAL) -- cgit v1.2.3 From c047f0e3adae59d7e388a1d42d940c3cd5714f82 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 26 Apr 2013 13:28:21 +0800 Subject: filter out Spark streaming block RDD and sort RDDInfo with id --- .../main/scala/spark/storage/StorageUtils.scala | 33 ++++++++++++++-------- 1 file changed, 21 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index dec47a9d41..8f52168c24 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -4,9 +4,9 @@ import spark.{Utils, SparkContext} import BlockManagerMasterActor.BlockStatus private[spark] -case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, +case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, blocks: Map[String, BlockStatus]) { - + def memUsed(blockPrefix: String = "") = { blocks.filterKeys(_.startsWith(blockPrefix)).values.map(_.memSize). reduceOption(_+_).getOrElse(0l) @@ -22,35 +22,40 @@ case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, } case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, - numCachedPartitions: Int, numPartitions: Int, memSize: Long, diskSize: Long) { + numCachedPartitions: Int, numPartitions: Int, memSize: Long, diskSize: Long) + extends Ordered[RDDInfo] { override def toString = { import Utils.memoryBytesToString "RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; DiskSize: %s".format(name, id, storageLevel.toString, numCachedPartitions, numPartitions, memoryBytesToString(memSize), memoryBytesToString(diskSize)) } + + override def compare(that: RDDInfo) = { + this.id - that.id + } } /* Helper methods for storage-related objects */ private[spark] object StorageUtils { - /* Given the current storage status of the BlockManager, returns information for each RDD */ - def rddInfoFromStorageStatus(storageStatusList: Array[StorageStatus], + /* Given the current storage status of the BlockManager, returns information for each RDD */ + def rddInfoFromStorageStatus(storageStatusList: Array[StorageStatus], sc: SparkContext) : Array[RDDInfo] = { - rddInfoFromBlockStatusList(storageStatusList.flatMap(_.blocks).toMap, sc) + rddInfoFromBlockStatusList(storageStatusList.flatMap(_.blocks).toMap, sc) } - /* Given a list of BlockStatus objets, returns information for each RDD */ - def rddInfoFromBlockStatusList(infos: Map[String, BlockStatus], + /* Given a list of BlockStatus objets, returns information for each RDD */ + def rddInfoFromBlockStatusList(infos: Map[String, BlockStatus], sc: SparkContext) : Array[RDDInfo] = { // Group by rddId, ignore the partition name - val groupedRddBlocks = infos.groupBy { case(k, v) => + val groupedRddBlocks = infos.filterKeys(_.startsWith("rdd_")).groupBy { case(k, v) => k.substring(0,k.lastIndexOf('_')) }.mapValues(_.values.toArray) // For each RDD, generate an RDDInfo object - groupedRddBlocks.map { case(rddKey, rddBlocks) => + val rddInfos = groupedRddBlocks.map { case(rddKey, rddBlocks) => // Add up memory and disk sizes val memSize = rddBlocks.map(_.memSize).reduce(_ + _) @@ -65,10 +70,14 @@ object StorageUtils { RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, rdd.partitions.size, memSize, diskSize) }.toArray + + scala.util.Sorting.quickSort(rddInfos) + + rddInfos } - /* Removes all BlockStatus object that are not part of a block prefix */ - def filterStorageStatusByPrefix(storageStatusList: Array[StorageStatus], + /* Removes all BlockStatus object that are not part of a block prefix */ + def filterStorageStatusByPrefix(storageStatusList: Array[StorageStatus], prefix: String) : Array[StorageStatus] = { storageStatusList.map { status => -- cgit v1.2.3 From 1b5aaeadc72ad5197c00897c41f670ea241d0235 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Thu, 2 May 2013 07:30:06 +0530 Subject: Integrate review comments 2 --- .../spark/scheduler/cluster/ClusterScheduler.scala | 78 +++++++++++----------- .../spark/scheduler/cluster/TaskSetManager.scala | 74 ++++++++++---------- .../spark/scheduler/local/LocalScheduler.scala | 2 +- 3 files changed, 77 insertions(+), 77 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 49fc449e86..cf4483f144 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -32,28 +32,28 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val TASK_REVIVAL_INTERVAL = System.getProperty("spark.tasks.revive.interval", "0").toLong /* - This property controls how aggressive we should be to modulate waiting for host local task scheduling. - To elaborate, currently there is a time limit (3 sec def) to ensure that spark attempts to wait for host locality of tasks before + This property controls how aggressive we should be to modulate waiting for node local task scheduling. + To elaborate, currently there is a time limit (3 sec def) to ensure that spark attempts to wait for node locality of tasks before scheduling on other nodes. We have modified this in yarn branch such that offers to task set happen in prioritized order : - host-local, rack-local and then others - But once all available host local (and no pref) tasks are scheduled, instead of waiting for 3 sec before + node-local, rack-local and then others + But once all available node local (and no pref) tasks are scheduled, instead of waiting for 3 sec before scheduling to other nodes (which degrades performance for time sensitive tasks and on larger clusters), we can modulate that : to also allow rack local nodes or any node. The default is still set to HOST - so that previous behavior is maintained. This is to allow tuning the tension between pulling rdd data off node and scheduling computation asap. TODO: rename property ? The value is one of - - HOST_LOCAL (default, no change w.r.t current behavior), + - NODE_LOCAL (default, no change w.r.t current behavior), - RACK_LOCAL and - ANY Note that this property makes more sense when used in conjugation with spark.tasks.revive.interval > 0 : else it is not very effective. Additional Note: For non trivial clusters, there is a 4x - 5x reduction in running time (in some of our experiments) based on whether - it is left at default HOST_LOCAL, RACK_LOCAL (if cluster is configured to be rack aware) or ANY. + it is left at default NODE_LOCAL, RACK_LOCAL (if cluster is configured to be rack aware) or ANY. If cluster is rack aware, then setting it to RACK_LOCAL gives best tradeoff and a 3x - 4x performance improvement while minimizing IO impact. Also, it brings down the variance in running time drastically. */ - val TASK_SCHEDULING_AGGRESSION = TaskLocality.parse(System.getProperty("spark.tasks.schedule.aggression", "HOST_LOCAL")) + val TASK_SCHEDULING_AGGRESSION = TaskLocality.parse(System.getProperty("spark.tasks.schedule.aggression", "NODE_LOCAL")) val activeTaskSets = new HashMap[String, TaskSetManager] var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager] @@ -73,7 +73,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val activeExecutorIds = new HashSet[String] // TODO: We might want to remove this and merge it with execId datastructures - but later. - // Which hosts in the cluster are alive (contains hostPort's) - used for instance local and node local task locality. + // Which hosts in the cluster are alive (contains hostPort's) - used for process local and node local task locality. private val hostPortsAlive = new HashSet[String] private val hostToAliveHostPorts = new HashMap[String, HashSet[String]] @@ -217,9 +217,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } // Build a list of tasks to assign to each slave val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) - // merge availableCpus into hostToAvailableCpus block ? + // merge availableCpus into nodeToAvailableCpus block ? val availableCpus = offers.map(o => o.cores).toArray - val hostToAvailableCpus = { + val nodeToAvailableCpus = { val map = new HashMap[String, Int]() for (offer <- offers) { val hostPort = offer.hostPort @@ -239,9 +239,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext) for (manager <- activeTaskSetsQueue.sortBy(m => (m.taskSet.priority, m.taskSet.stageId))) { - // Split offers based on host local, rack local and off-rack tasks. - val instanceLocalOffers = new HashMap[String, ArrayBuffer[Int]]() - val hostLocalOffers = new HashMap[String, ArrayBuffer[Int]]() + // Split offers based on node local, rack local and off-rack tasks. + val processLocalOffers = new HashMap[String, ArrayBuffer[Int]]() + val nodeLocalOffers = new HashMap[String, ArrayBuffer[Int]]() val rackLocalOffers = new HashMap[String, ArrayBuffer[Int]]() val otherOffers = new HashMap[String, ArrayBuffer[Int]]() @@ -250,29 +250,29 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // DEBUG code Utils.checkHostPort(hostPort) - val numInstanceLocalTasks = math.max(0, math.min(manager.numPendingTasksForHostPort(hostPort), availableCpus(i))) - if (numInstanceLocalTasks > 0){ - val list = instanceLocalOffers.getOrElseUpdate(hostPort, new ArrayBuffer[Int]) - for (j <- 0 until numInstanceLocalTasks) list += i + val numProcessLocalTasks = math.max(0, math.min(manager.numPendingTasksForHostPort(hostPort), availableCpus(i))) + if (numProcessLocalTasks > 0){ + val list = processLocalOffers.getOrElseUpdate(hostPort, new ArrayBuffer[Int]) + for (j <- 0 until numProcessLocalTasks) list += i } val host = Utils.parseHostPort(hostPort)._1 - val numHostLocalTasks = math.max(0, - // Remove instance local tasks (which are also host local btw !) from this - math.min(manager.numPendingTasksForHost(hostPort) - numInstanceLocalTasks, hostToAvailableCpus(host))) - if (numHostLocalTasks > 0){ - val list = hostLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) - for (j <- 0 until numHostLocalTasks) list += i + val numNodeLocalTasks = math.max(0, + // Remove process local tasks (which are also host local btw !) from this + math.min(manager.numPendingTasksForHost(hostPort) - numProcessLocalTasks, nodeToAvailableCpus(host))) + if (numNodeLocalTasks > 0){ + val list = nodeLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) + for (j <- 0 until numNodeLocalTasks) list += i } val numRackLocalTasks = math.max(0, - // Remove host local tasks (which are also rack local btw !) from this - math.min(manager.numRackLocalPendingTasksForHost(hostPort) - numInstanceLocalTasks - numHostLocalTasks, hostToAvailableCpus(host))) + // Remove node local tasks (which are also rack local btw !) from this + math.min(manager.numRackLocalPendingTasksForHost(hostPort) - numProcessLocalTasks - numNodeLocalTasks, nodeToAvailableCpus(host))) if (numRackLocalTasks > 0){ val list = rackLocalOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) for (j <- 0 until numRackLocalTasks) list += i } - if (numHostLocalTasks <= 0 && numRackLocalTasks <= 0){ + if (numNodeLocalTasks <= 0 && numRackLocalTasks <= 0){ // add to others list - spread even this across cluster. val list = otherOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) list += i @@ -280,19 +280,19 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } val offersPriorityList = new ArrayBuffer[Int]( - instanceLocalOffers.size + hostLocalOffers.size + rackLocalOffers.size + otherOffers.size) + processLocalOffers.size + nodeLocalOffers.size + rackLocalOffers.size + otherOffers.size) - // First instance local, then host local, then rack, then others + // First process local, then host local, then rack, then others - // numHostLocalOffers contains count of both instance local and host offers. - val numHostLocalOffers = { - val instanceLocalPriorityList = ClusterScheduler.prioritizeContainers(instanceLocalOffers) - offersPriorityList ++= instanceLocalPriorityList + // numNodeLocalOffers contains count of both process local and host offers. + val numNodeLocalOffers = { + val processLocalPriorityList = ClusterScheduler.prioritizeContainers(processLocalOffers) + offersPriorityList ++= processLocalPriorityList - val hostLocalPriorityList = ClusterScheduler.prioritizeContainers(hostLocalOffers) - offersPriorityList ++= hostLocalPriorityList + val nodeLocalPriorityList = ClusterScheduler.prioritizeContainers(nodeLocalOffers) + offersPriorityList ++= nodeLocalPriorityList - instanceLocalPriorityList.size + hostLocalPriorityList.size + processLocalPriorityList.size + nodeLocalPriorityList.size } val numRackLocalOffers = { val rackLocalPriorityList = ClusterScheduler.prioritizeContainers(rackLocalOffers) @@ -303,8 +303,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) var lastLoop = false val lastLoopIndex = TASK_SCHEDULING_AGGRESSION match { - case TaskLocality.HOST_LOCAL => numHostLocalOffers - case TaskLocality.RACK_LOCAL => numRackLocalOffers + numHostLocalOffers + case TaskLocality.NODE_LOCAL => numNodeLocalOffers + case TaskLocality.RACK_LOCAL => numRackLocalOffers + numNodeLocalOffers case TaskLocality.ANY => offersPriorityList.size } @@ -343,8 +343,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // prevent more looping launchedTask = false } else if (!lastLoop && !launchedTask) { - // Do this only if TASK_SCHEDULING_AGGRESSION != HOST_LOCAL - if (TASK_SCHEDULING_AGGRESSION != TaskLocality.HOST_LOCAL) { + // Do this only if TASK_SCHEDULING_AGGRESSION != NODE_LOCAL + if (TASK_SCHEDULING_AGGRESSION != TaskLocality.NODE_LOCAL) { // fudge launchedTask to ensure we loop once more launchedTask = true // dont loop anymore diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 5f3faaa5c3..ff4790e4cb 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -13,21 +13,21 @@ import spark.scheduler._ import spark.TaskState.TaskState import java.nio.ByteBuffer -private[spark] object TaskLocality extends Enumeration("INSTANCE_LOCAL", "HOST_LOCAL", "RACK_LOCAL", "ANY") with Logging { +private[spark] object TaskLocality extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY") with Logging { - // instance local is expected to be used ONLY within tasksetmanager for now. - val INSTANCE_LOCAL, HOST_LOCAL, RACK_LOCAL, ANY = Value + // process local is expected to be used ONLY within tasksetmanager for now. + val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value type TaskLocality = Value def isAllowed(constraint: TaskLocality, condition: TaskLocality): Boolean = { // Must not be the constraint. - assert (constraint != TaskLocality.INSTANCE_LOCAL) + assert (constraint != TaskLocality.PROCESS_LOCAL) constraint match { - case TaskLocality.HOST_LOCAL => condition == TaskLocality.HOST_LOCAL - case TaskLocality.RACK_LOCAL => condition == TaskLocality.HOST_LOCAL || condition == TaskLocality.RACK_LOCAL + case TaskLocality.NODE_LOCAL => condition == TaskLocality.NODE_LOCAL + case TaskLocality.RACK_LOCAL => condition == TaskLocality.NODE_LOCAL || condition == TaskLocality.RACK_LOCAL // For anything else, allow case _ => true } @@ -37,15 +37,15 @@ private[spark] object TaskLocality extends Enumeration("INSTANCE_LOCAL", "HOST_L // better way to do this ? try { val retval = TaskLocality.withName(str) - // Must not specify INSTANCE_LOCAL ! - assert (retval != TaskLocality.INSTANCE_LOCAL) + // Must not specify PROCESS_LOCAL ! + assert (retval != TaskLocality.PROCESS_LOCAL) retval } catch { case nEx: NoSuchElementException => { - logWarning("Invalid task locality specified '" + str + "', defaulting to HOST_LOCAL"); + logWarning("Invalid task locality specified '" + str + "', defaulting to NODE_LOCAL"); // default to preserve earlier behavior - HOST_LOCAL + NODE_LOCAL } } } @@ -84,7 +84,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis - // List of pending tasks for each node (instance local to container). These collections are actually + // List of pending tasks for each node (process local to container). These collections are actually // treated as stacks, in which new tasks are added to the end of the // ArrayBuffer and removed from the end. This makes it faster to detect // tasks that repeatedly fail because whenever a task failed, it is put @@ -142,12 +142,12 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe } // Note that it follows the hierarchy. - // if we search for HOST_LOCAL, the output will include INSTANCE_LOCAL and - // if we search for RACK_LOCAL, it will include INSTANCE_LOCAL & HOST_LOCAL + // if we search for NODE_LOCAL, the output will include PROCESS_LOCAL and + // if we search for RACK_LOCAL, it will include PROCESS_LOCAL & NODE_LOCAL private def findPreferredLocations(_taskPreferredLocations: Seq[String], scheduler: ClusterScheduler, taskLocality: TaskLocality.TaskLocality): HashSet[String] = { - if (TaskLocality.INSTANCE_LOCAL == taskLocality) { + if (TaskLocality.PROCESS_LOCAL == taskLocality) { // straight forward comparison ! Special case it. val retval = new HashSet[String]() scheduler.synchronized { @@ -162,7 +162,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe } val taskPreferredLocations = - if (TaskLocality.HOST_LOCAL == taskLocality) { + if (TaskLocality.NODE_LOCAL == taskLocality) { _taskPreferredLocations } else { assert (TaskLocality.RACK_LOCAL == taskLocality) @@ -203,19 +203,19 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe private def addPendingTask(index: Int) { // We can infer hostLocalLocations from rackLocalLocations by joining it against tasks(index).preferredLocations (with appropriate // hostPort <-> host conversion). But not doing it for simplicity sake. If this becomes a performance issue, modify it. - val instanceLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.INSTANCE_LOCAL) - val hostLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HOST_LOCAL) + val processLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.PROCESS_LOCAL) + val hostLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) val rackLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) if (rackLocalLocations.size == 0) { // Current impl ensures this. - assert (instanceLocalLocations.size == 0) + assert (processLocalLocations.size == 0) assert (hostLocalLocations.size == 0) pendingTasksWithNoPrefs += index } else { - // instance local locality - for (hostPort <- instanceLocalLocations) { + // process local locality + for (hostPort <- processLocalLocations) { // DEBUG Code Utils.checkHostPort(hostPort) @@ -223,7 +223,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe hostPortList += index } - // host locality (includes instance local) + // host locality (includes process local) for (hostPort <- hostLocalLocations) { // DEBUG Code Utils.checkHostPort(hostPort) @@ -233,7 +233,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe hostList += index } - // rack locality (includes instance local and host local) + // rack locality (includes process local and host local) for (rackLocalHostPort <- rackLocalLocations) { // DEBUG Code Utils.checkHostPort(rackLocalHostPort) @@ -247,7 +247,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe allPendingTasks += index } - // Return the pending tasks list for a given host port (instance local), or an empty list if + // Return the pending tasks list for a given host port (process local), or an empty list if // there is no map entry for that host private def getPendingTasksForHostPort(hostPort: String): ArrayBuffer[Int] = { // DEBUG Code @@ -269,7 +269,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe pendingRackLocalTasksForHost.getOrElse(host, ArrayBuffer()) } - // Number of pending tasks for a given host Port (which would be instance local) + // Number of pending tasks for a given host Port (which would be process local) def numPendingTasksForHostPort(hostPort: String): Int = { getPendingTasksForHostPort(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) } @@ -305,13 +305,13 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // task must have a preference for this host/rack/no preferred locations at all. private def findSpeculativeTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = { - assert (TaskLocality.isAllowed(locality, TaskLocality.HOST_LOCAL)) + assert (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) speculatableTasks.retain(index => !finished(index)) // Remove finished tasks from set if (speculatableTasks.size > 0) { val localTask = speculatableTasks.find { index => - val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HOST_LOCAL) + val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) val attemptLocs = taskAttempts(index).map(_.hostPort) (locations.size == 0 || locations.contains(hostPort)) && !attemptLocs.contains(hostPort) } @@ -352,9 +352,9 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // Dequeue a pending task for a given node and return its index. // If localOnly is set to false, allow non-local tasks as well. private def findTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = { - val instanceLocalTask = findTaskFromList(getPendingTasksForHostPort(hostPort)) - if (instanceLocalTask != None) { - return instanceLocalTask + val processLocalTask = findTaskFromList(getPendingTasksForHostPort(hostPort)) + if (processLocalTask != None) { + return processLocalTask } val localTask = findTaskFromList(getPendingTasksForHost(hostPort)) @@ -387,7 +387,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe return findSpeculativeTask(hostPort, locality) } - private def isInstanceLocalLocation(task: Task[_], hostPort: String): Boolean = { + private def isProcessLocalLocation(task: Task[_], hostPort: String): Boolean = { Utils.checkHostPort(hostPort) val locs = task.preferredLocations @@ -433,7 +433,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe val locality = if (overrideLocality != null) overrideLocality else { // expand only if we have waited for more than LOCALITY_WAIT for a host local task ... val time = System.currentTimeMillis - if (time - lastPreferredLaunchTime < LOCALITY_WAIT) TaskLocality.HOST_LOCAL else TaskLocality.ANY + if (time - lastPreferredLaunchTime < LOCALITY_WAIT) TaskLocality.NODE_LOCAL else TaskLocality.ANY } findTask(hostPort, locality) match { @@ -443,8 +443,8 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe val taskId = sched.newTaskId() // Figure out whether this should count as a preferred launch val taskLocality = - if (isInstanceLocalLocation(task, hostPort)) TaskLocality.INSTANCE_LOCAL else - if (isHostLocalLocation(task, hostPort)) TaskLocality.HOST_LOCAL else + if (isProcessLocalLocation(task, hostPort)) TaskLocality.PROCESS_LOCAL else + if (isHostLocalLocation(task, hostPort)) TaskLocality.NODE_LOCAL else if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL else TaskLocality.ANY val prefStr = taskLocality.toString @@ -456,7 +456,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe val info = new TaskInfo(taskId, index, time, execId, hostPort, taskLocality) taskInfos(taskId) = info taskAttempts(index) = info :: taskAttempts(index) - if (TaskLocality.HOST_LOCAL == taskLocality) { + if (TaskLocality.NODE_LOCAL == taskLocality) { lastPreferredLaunchTime = time } // Serialize and return the task @@ -608,11 +608,11 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe // host local tasks - should we push this to rack local or no pref list ? For now, preserving behavior and moving to // no prefs list. Note, this was done due to impliations related to 'waiting' for data local tasks, etc. - // Note: NOT checking instance local list - since host local list is super set of that. We need to ad to no prefs only if - // there is no host local node for the task (not if there is no instance local node for the task) + // Note: NOT checking process local list - since host local list is super set of that. We need to ad to no prefs only if + // there is no host local node for the task (not if there is no process local node for the task) for (index <- getPendingTasksForHost(Utils.parseHostPort(hostPort)._1)) { // val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) - val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.HOST_LOCAL) + val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) if (newLocs.isEmpty) { pendingTasksWithNoPrefs += index } diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index f060a940a9..53dd6fbe13 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -53,7 +53,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon def runTask(task: Task[_], idInJob: Int, attemptId: Int) { logInfo("Running " + task) - val info = new TaskInfo(attemptId, idInJob, System.currentTimeMillis(), "local", "local:1", TaskLocality.HOST_LOCAL) + val info = new TaskInfo(attemptId, idInJob, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) // Set the Spark execution environment for the worker thread SparkEnv.set(env) try { -- cgit v1.2.3 From dfde9ce9dde0a151d42f7aecb826b40a4c98b459 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Thu, 2 May 2013 07:41:33 +0530 Subject: comment out debug versions of checkHost, etc from Utils - which were used to test --- core/src/main/scala/spark/Utils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 0e348f8189..c1495d5317 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -335,6 +335,7 @@ private object Utils extends Logging { retval } +/* // Used by DEBUG code : remove when all testing done private val ipPattern = Pattern.compile("^[0-9]+(\\.[0-9]+)*$") def checkHost(host: String, message: String = "") { @@ -363,8 +364,8 @@ private object Utils extends Logging { // temp code for debug System.exit(-1) } +*/ -/* // Once testing is complete in various modes, replace with this ? def checkHost(host: String, message: String = "") {} def checkHostPort(hostPort: String, message: String = "") {} @@ -373,7 +374,6 @@ private object Utils extends Logging { def logErrorWithStack(msg: String) { try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } } } -*/ def getUserNameFromEnvironment(): String = { SparkHadoopUtil.getUserNameFromEnvironment -- cgit v1.2.3 From 98df9d28536f5208530488a316df9401e16490bd Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 1 May 2013 20:17:09 -0700 Subject: Added removeRdd function in BlockManager. --- core/src/main/scala/spark/RDD.scala | 15 ++++--------- core/src/main/scala/spark/SparkContext.scala | 8 +++---- .../scala/spark/storage/BlockManagerMaster.scala | 16 ++++++++++++++ .../main/scala/spark/storage/BlockManagerUI.scala | 4 ++-- .../scala/spark/storage/BlockManagerSuite.scala | 25 ++++++++++++++++++++++ 5 files changed, 51 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index c77f9915c0..fd14ef17f1 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -107,7 +107,7 @@ abstract class RDD[T: ClassManifest]( // ======================================================================= /** A unique ID for this RDD (within its SparkContext). */ - val id = sc.newRddId() + val id: Int = sc.newRddId() /** A friendly name for this RDD */ var name: String = null @@ -120,7 +120,8 @@ abstract class RDD[T: ClassManifest]( /** * Set this RDD's storage level to persist its values across operations after the first time - * it is computed. Can only be called once on each RDD. + * it is computed. This can only be used to assign a new storage level if the RDD does not + * have a storage level set yet.. */ def persist(newLevel: StorageLevel): RDD[T] = { // TODO: Handle changes of StorageLevel @@ -143,15 +144,7 @@ abstract class RDD[T: ClassManifest]( /** Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. */ def unpersist(): RDD[T] = { logInfo("Removing RDD " + id + " from persistence list") - val rddBlockPrefix = "rdd_" + id + "_" - // Get the list of blocks in block manager, and remove ones that are part of this RDD. - // The runtime complexity is linear to the number of blocks persisted in the cluster. - // It could be expensive if the cluster is large and has a lot of blocks persisted. - sc.getExecutorStorageStatus().flatMap(_.blocks).foreach { case(blockId, status) => - if (blockId.startsWith(rddBlockPrefix)) { - sc.env.blockManager.master.removeBlock(blockId) - } - } + sc.env.blockManager.master.removeRdd(id) sc.persistentRdds.remove(id) storageLevel = StorageLevel.NONE this diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index d7d450d958..2ae4ad8659 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -508,18 +508,18 @@ class SparkContext( * Return information about what RDDs are cached, if they are in mem or on disk, how much space * they take, etc. */ - def getRDDStorageInfo(): Array[RDDInfo] = { - StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus(), this) + def getRDDStorageInfo: Array[RDDInfo] = { + StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this) } - def getStageInfo(): Map[Stage,StageInfo] = { + def getStageInfo: Map[Stage,StageInfo] = { dagScheduler.stageToInfos } /** * Return information about blocks stored in all of the slaves */ - def getExecutorStorageStatus(): Array[StorageStatus] = { + def getExecutorStorageStatus: Array[StorageStatus] = { env.blockManager.master.getStorageStatus } diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 6fae62d373..ac26c16867 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -15,6 +15,7 @@ import akka.util.duration._ import spark.{Logging, SparkException, Utils} + private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging { val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt @@ -87,6 +88,21 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi askDriverWithReply(RemoveBlock(blockId)) } + /** + * Remove all blocks belonging to the given RDD. + */ + def removeRdd(rddId: Int) { + val rddBlockPrefix = "rdd_" + rddId + "_" + // Get the list of blocks in block manager, and remove ones that are part of this RDD. + // The runtime complexity is linear to the number of blocks persisted in the cluster. + // It could be expensive if the cluster is large and has a lot of blocks persisted. + getStorageStatus.flatMap(_.blocks).foreach { case(blockId, status) => + if (blockId.startsWith(rddBlockPrefix)) { + removeBlock(blockId) + } + } + } + /** * Return the memory status for each block manager, in the form of a map from * the block manager's id to two long values. The first value is the maximum diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index c9e4519efe..07da572044 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -45,7 +45,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, path("") { completeWith { // Request the current storage status from the Master - val storageStatusList = sc.getExecutorStorageStatus() + val storageStatusList = sc.getExecutorStorageStatus // Calculate macro-level statistics val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) @@ -60,7 +60,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, parameter("id") { id => completeWith { val prefix = "rdd_" + id.toString - val storageStatusList = sc.getExecutorStorageStatus() + val storageStatusList = sc.getExecutorStorageStatus val filteredStorageStatusList = StorageUtils. filterStorageStatusByPrefix(storageStatusList, prefix) val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index 5a11a4483b..9fe0de665c 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -207,6 +207,31 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } } + test("removing rdd") { + store = new BlockManager("", actorSystem, master, serializer, 2000) + val a1 = new Array[Byte](400) + val a2 = new Array[Byte](400) + val a3 = new Array[Byte](400) + // Putting a1, a2 and a3 in memory. + store.putSingle("rdd_0_0", a1, StorageLevel.MEMORY_ONLY) + store.putSingle("rdd_0_1", a2, StorageLevel.MEMORY_ONLY) + store.putSingle("nonrddblock", a3, StorageLevel.MEMORY_ONLY) + master.removeRdd(0) + + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + store.getSingle("rdd_0_0") should be (None) + master.getLocations("rdd_0_0") should have size 0 + } + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + store.getSingle("rdd_0_1") should be (None) + master.getLocations("rdd_0_1") should have size 0 + } + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + store.getSingle("nonrddblock") should not be (None) + master.getLocations("nonrddblock") should have size (1) + } + } + test("reregistration on heart beat") { val heartBeat = PrivateMethod[Unit]('heartBeat) store = new BlockManager("", actorSystem, master, serializer, 2000) -- cgit v1.2.3 From 4a318774088f829fe54c3ef0b5f565a845631b4e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 1 May 2013 20:31:54 -0700 Subject: Added the unpersist api to JavaRDD. --- core/src/main/scala/spark/api/java/JavaRDD.scala | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala index e29f1e5899..eb81ed64cd 100644 --- a/core/src/main/scala/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaRDD.scala @@ -14,12 +14,18 @@ JavaRDDLike[T, JavaRDD[T]] { /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ def cache(): JavaRDD[T] = wrapRDD(rdd.cache()) - /** + /** * Set this RDD's storage level to persist its values across operations after the first time - * it is computed. Can only be called once on each RDD. + * it is computed. This can only be used to assign a new storage level if the RDD does not + * have a storage level set yet.. */ def persist(newLevel: StorageLevel): JavaRDD[T] = wrapRDD(rdd.persist(newLevel)) + /** + * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. + */ + def unpersist(): JavaRDD[T] = wrapRDD(rdd.unpersist()) + // Transformations (return a new RDD) /** @@ -31,7 +37,7 @@ JavaRDDLike[T, JavaRDD[T]] { * Return a new RDD containing the distinct elements in this RDD. */ def distinct(numPartitions: Int): JavaRDD[T] = wrapRDD(rdd.distinct(numPartitions)) - + /** * Return a new RDD containing only the elements that satisfy a predicate. */ @@ -54,7 +60,7 @@ JavaRDDLike[T, JavaRDD[T]] { */ def sample(withReplacement: Boolean, fraction: Double, seed: Int): JavaRDD[T] = wrapRDD(rdd.sample(withReplacement, fraction, seed)) - + /** * Return the union of this RDD and another one. Any identical elements will appear multiple * times (use `.distinct()` to eliminate them). @@ -63,7 +69,7 @@ JavaRDDLike[T, JavaRDD[T]] { /** * Return an RDD with the elements from `this` that are not in `other`. - * + * * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting * RDD will be <= us. */ -- cgit v1.2.3 From c847dd3da29483fede326cb9821b0d33f735137e Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Tue, 19 Mar 2013 15:08:22 -0700 Subject: Don't accept generated temp directory names that can't be created successfully. --- core/src/main/scala/spark/storage/DiskStore.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index c9553d2e0f..215c25132b 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -168,8 +168,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) localDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536)) localDir = new File(rootDir, "spark-local-" + localDirId) if (!localDir.exists) { - localDir.mkdirs() - foundLocalDir = true + foundLocalDir = localDir.mkdirs() } } catch { case e: Exception => -- cgit v1.2.3 From 11589c39d9f75e9757ba1717c5202f77d30031b2 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Fri, 3 May 2013 12:23:30 +0530 Subject: Fix ZippedRDD as part Matei's suggestion --- core/src/main/scala/spark/rdd/ZippedRDD.scala | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala index e80250a99b..51573fe68a 100644 --- a/core/src/main/scala/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala @@ -1,6 +1,6 @@ package spark.rdd -import spark.{OneToOneDependency, RDD, SparkContext, Partition, TaskContext} +import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} @@ -49,9 +49,20 @@ class ZippedRDD[T: ClassManifest, U: ClassManifest]( override def getPreferredLocations(s: Partition): Seq[String] = { val (partition1, partition2) = s.asInstanceOf[ZippedPartition[T, U]].partitions - // TODO: becomes complicated - intersect on hostPort if available, else fallback to host (removing intersected hostPort's). - // Since I am not very sure about this RDD, leaving it to others to comment better ! - rdd1.preferredLocations(partition1).intersect(rdd2.preferredLocations(partition2)) + val pref1 = rdd1.preferredLocations(partition1) + val pref2 = rdd2.preferredLocations(partition2) + + // both partitions are instance local. + val instanceLocalLocations = pref1.intersect(pref2) + + // remove locations which are already handled via instanceLocalLocations, and intersect where both partitions are node local. + val nodeLocalPref1 = pref1.filter(loc => ! instanceLocalLocations.contains(loc)).map(loc => Utils.parseHostPort(loc)._1) + val nodeLocalPref2 = pref2.filter(loc => ! instanceLocalLocations.contains(loc)).map(loc => Utils.parseHostPort(loc)._1) + val nodeLocalLocations = nodeLocalPref1.intersect(nodeLocalPref2) + + + // Can have mix of instance local (hostPort) and node local (host) locations as preference ! + instanceLocalLocations ++ nodeLocalLocations } override def clearDependencies() { -- cgit v1.2.3 From 2bc895a829caa459e032e12e1d117994dd510a5c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 3 May 2013 01:02:16 -0700 Subject: Updated according to Matei's code review comment. --- core/src/main/scala/spark/ShuffleFetcher.scala | 2 +- core/src/main/scala/spark/SparkEnv.scala | 10 +++-- core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 3 +- core/src/main/scala/spark/rdd/ShuffledRDD.scala | 5 +-- core/src/main/scala/spark/rdd/SubtractedRDD.scala | 4 +- .../scala/spark/scheduler/ShuffleMapTask.scala | 5 +-- .../main/scala/spark/serializer/Serializer.scala | 42 -------------------- .../scala/spark/serializer/SerializerManager.scala | 45 ++++++++++++++++++++++ core/src/main/scala/spark/storage/DiskStore.scala | 36 +++++++++-------- .../scala/spark/storage/ShuffleBlockManager.scala | 34 ++++++++-------- 10 files changed, 98 insertions(+), 88 deletions(-) create mode 100644 core/src/main/scala/spark/serializer/SerializerManager.scala diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala index 49addc0c10..9513a00126 100644 --- a/core/src/main/scala/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/spark/ShuffleFetcher.scala @@ -10,7 +10,7 @@ private[spark] abstract class ShuffleFetcher { * @return An iterator over the elements of the fetched shuffle outputs. */ def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics, - serializer: Serializer = Serializer.default): Iterator[(K,V)] + serializer: Serializer = SparkEnv.get.serializerManager.default): Iterator[(K,V)] /** Stop the fetcher */ def stop() {} diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 8ba52245fa..2fa97cd829 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -7,7 +7,7 @@ import spark.broadcast.BroadcastManager import spark.storage.BlockManager import spark.storage.BlockManagerMaster import spark.network.ConnectionManager -import spark.serializer.Serializer +import spark.serializer.{Serializer, SerializerManager} import spark.util.AkkaUtils @@ -21,6 +21,7 @@ import spark.util.AkkaUtils class SparkEnv ( val executorId: String, val actorSystem: ActorSystem, + val serializerManager: SerializerManager, val serializer: Serializer, val closureSerializer: Serializer, val cacheManager: CacheManager, @@ -92,10 +93,12 @@ object SparkEnv extends Logging { Class.forName(name, true, classLoader).newInstance().asInstanceOf[T] } - val serializer = Serializer.setDefault( + val serializerManager = new SerializerManager + + val serializer = serializerManager.setDefault( System.getProperty("spark.serializer", "spark.JavaSerializer")) - val closureSerializer = Serializer.get( + val closureSerializer = serializerManager.get( System.getProperty("spark.closure.serializer", "spark.JavaSerializer")) def registerOrLookup(name: String, newActor: => Actor): ActorRef = { @@ -155,6 +158,7 @@ object SparkEnv extends Logging { new SparkEnv( executorId, actorSystem, + serializerManager, serializer, closureSerializer, cacheManager, diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 9e996e9958..7599ba1a02 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -8,7 +8,6 @@ import scala.collection.mutable.ArrayBuffer import spark.{Aggregator, Logging, Partition, Partitioner, RDD, SparkEnv, TaskContext} import spark.{Dependency, OneToOneDependency, ShuffleDependency} -import spark.serializer.Serializer private[spark] sealed trait CoGroupSplitDep extends Serializable @@ -114,7 +113,7 @@ class CoGroupedRDD[K]( } } - val ser = Serializer.get(serializerClass) + val ser = SparkEnv.get.serializerManager.get(serializerClass) for ((dep, depNum) <- split.deps.zipWithIndex) dep match { case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { // Read them from the parent diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala index 8175e23eff..c7d1926b83 100644 --- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala @@ -2,7 +2,6 @@ package spark.rdd import spark.{Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext} import spark.SparkContext._ -import spark.serializer.Serializer private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @@ -32,7 +31,7 @@ class ShuffledRDD[K, V]( override def compute(split: Partition, context: TaskContext): Iterator[(K, V)] = { val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId - SparkEnv.get.shuffleFetcher.fetch[K, V]( - shuffledId, split.index, context.taskMetrics, Serializer.get(serializerClass)) + SparkEnv.get.shuffleFetcher.fetch[K, V](shuffledId, split.index, context.taskMetrics, + SparkEnv.get.serializerManager.get(serializerClass)) } } diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala index f60c35c38e..8a9efc5da2 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -11,7 +11,7 @@ import spark.Partition import spark.SparkEnv import spark.ShuffleDependency import spark.OneToOneDependency -import spark.serializer.Serializer + /** * An optimized version of cogroup for set difference/subtraction. @@ -68,7 +68,7 @@ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassM override def compute(p: Partition, context: TaskContext): Iterator[(K, V)] = { val partition = p.asInstanceOf[CoGroupPartition] - val serializer = Serializer.get(serializerClass) + val serializer = SparkEnv.get.serializerManager.get(serializerClass) val map = new JHashMap[K, ArrayBuffer[V]] def getSeq(k: K): ArrayBuffer[V] = { val seq = map.get(k) diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 124d2d7e26..f097213ab5 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -14,7 +14,6 @@ import com.ning.compress.lzf.LZFOutputStream import spark._ import spark.executor.ShuffleWriteMetrics -import spark.serializer.Serializer import spark.storage._ import spark.util.{TimeStampedHashMap, MetadataCleaner} @@ -139,12 +138,12 @@ private[spark] class ShuffleMapTask( metrics = Some(taskContext.taskMetrics) val blockManager = SparkEnv.get.blockManager - var shuffle: ShuffleBlockManager#Shuffle = null + var shuffle: ShuffleBlocks = null var buckets: ShuffleWriterGroup = null try { // Obtain all the block writers for shuffle blocks. - val ser = Serializer.get(dep.serializerClass) + val ser = SparkEnv.get.serializerManager.get(dep.serializerClass) shuffle = blockManager.shuffleBlockManager.forShuffle(dep.shuffleId, numOutputSplits, ser) buckets = shuffle.acquireWriters(partition) diff --git a/core/src/main/scala/spark/serializer/Serializer.scala b/core/src/main/scala/spark/serializer/Serializer.scala index 77b1a1a434..2ad73b711d 100644 --- a/core/src/main/scala/spark/serializer/Serializer.scala +++ b/core/src/main/scala/spark/serializer/Serializer.scala @@ -2,7 +2,6 @@ package spark.serializer import java.io.{EOFException, InputStream, OutputStream} import java.nio.ByteBuffer -import java.util.concurrent.ConcurrentHashMap import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream @@ -19,47 +18,6 @@ trait Serializer { } -/** - * A singleton object that can be used to fetch serializer objects based on the serializer - * class name. If a previous instance of the serializer object has been created, the get - * method returns that instead of creating a new one. - */ -object Serializer { - - private val serializers = new ConcurrentHashMap[String, Serializer] - private var _default: Serializer = _ - - def default = _default - - def setDefault(clsName: String): Serializer = { - _default = get(clsName) - _default - } - - def get(clsName: String): Serializer = { - if (clsName == null) { - default - } else { - var serializer = serializers.get(clsName) - if (serializer != null) { - // If the serializer has been created previously, reuse that. - serializer - } else this.synchronized { - // Otherwise, create a new one. But make sure no other thread has attempted - // to create another new one at the same time. - serializer = serializers.get(clsName) - if (serializer == null) { - val clsLoader = Thread.currentThread.getContextClassLoader - serializer = Class.forName(clsName, true, clsLoader).newInstance().asInstanceOf[Serializer] - serializers.put(clsName, serializer) - } - serializer - } - } - } -} - - /** * An instance of a serializer, for use by one thread at a time. */ diff --git a/core/src/main/scala/spark/serializer/SerializerManager.scala b/core/src/main/scala/spark/serializer/SerializerManager.scala new file mode 100644 index 0000000000..60b2aac797 --- /dev/null +++ b/core/src/main/scala/spark/serializer/SerializerManager.scala @@ -0,0 +1,45 @@ +package spark.serializer + +import java.util.concurrent.ConcurrentHashMap + + +/** + * A service that returns a serializer object given the serializer's class name. If a previous + * instance of the serializer object has been created, the get method returns that instead of + * creating a new one. + */ +private[spark] class SerializerManager { + + private val serializers = new ConcurrentHashMap[String, Serializer] + private var _default: Serializer = _ + + def default = _default + + def setDefault(clsName: String): Serializer = { + _default = get(clsName) + _default + } + + def get(clsName: String): Serializer = { + if (clsName == null) { + default + } else { + var serializer = serializers.get(clsName) + if (serializer != null) { + // If the serializer has been created previously, reuse that. + serializer + } else this.synchronized { + // Otherwise, create a new one. But make sure no other thread has attempted + // to create another new one at the same time. + serializer = serializers.get(clsName) + if (serializer == null) { + val clsLoader = Thread.currentThread.getContextClassLoader + serializer = + Class.forName(clsName, true, clsLoader).newInstance().asInstanceOf[Serializer] + serializers.put(clsName, serializer) + } + serializer + } + } + } +} diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 4cddcc86fc..498bc9eeb6 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -2,6 +2,7 @@ package spark.storage import java.io.{File, FileOutputStream, OutputStream, RandomAccessFile} import java.nio.ByteBuffer +import java.nio.channels.FileChannel import java.nio.channels.FileChannel.MapMode import java.util.{Random, Date} import java.text.SimpleDateFormat @@ -26,14 +27,16 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) private val f: File = createFile(blockId /*, allowAppendExisting */) - private var repositionableStream: FastBufferedOutputStream = null + // The file channel, used for repositioning / truncating the file. + private var channel: FileChannel = null private var bs: OutputStream = null private var objOut: SerializationStream = null - private var validLength = 0L + private var lastValidPosition = 0L override def open(): DiskBlockObjectWriter = { - repositionableStream = new FastBufferedOutputStream(new FileOutputStream(f)) - bs = blockManager.wrapForCompression(blockId, repositionableStream) + val fos = new FileOutputStream(f, true) + channel = fos.getChannel() + bs = blockManager.wrapForCompression(blockId, new FastBufferedOutputStream(fos)) objOut = serializer.newInstance().serializeStream(bs) this } @@ -41,9 +44,9 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) override def close() { objOut.close() bs.close() - objOut = null + channel = null bs = null - repositionableStream = null + objOut = null // Invoke the close callback handler. super.close() } @@ -54,25 +57,23 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) // Return the number of bytes written for this commit. override def commit(): Long = { bs.flush() - validLength = repositionableStream.position() - validLength + val prevPos = lastValidPosition + lastValidPosition = channel.position() + lastValidPosition - prevPos } override def revertPartialWrites() { - // Flush the outstanding writes and delete the file. - objOut.close() - bs.close() - objOut = null - bs = null - repositionableStream = null - f.delete() + // Discard current writes. We do this by flushing the outstanding writes and + // truncate the file to the last valid position. + bs.flush() + channel.truncate(lastValidPosition) } override def write(value: Any) { objOut.writeObject(value) } - override def size(): Long = validLength + override def size(): Long = lastValidPosition } val MAX_DIR_CREATION_ATTEMPTS: Int = 10 @@ -86,7 +87,8 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) addShutdownHook() - def getBlockWriter(blockId: String, serializer: Serializer, bufferSize: Int): BlockObjectWriter = { + def getBlockWriter(blockId: String, serializer: Serializer, bufferSize: Int) + : BlockObjectWriter = { new DiskBlockObjectWriter(blockId, serializer, bufferSize) } diff --git a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala index 1903df0817..49eabfb0d2 100644 --- a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala @@ -8,26 +8,30 @@ class ShuffleWriterGroup(val id: Int, val writers: Array[BlockObjectWriter]) private[spark] -class ShuffleBlockManager(blockManager: BlockManager) { +trait ShuffleBlocks { + def acquireWriters(mapId: Int): ShuffleWriterGroup + def releaseWriters(group: ShuffleWriterGroup) +} - def forShuffle(shuffleId: Int, numBuckets: Int, serializer: Serializer): Shuffle = { - new Shuffle(shuffleId, numBuckets, serializer) - } - class Shuffle(shuffleId: Int, numBuckets: Int, serializer: Serializer) { +private[spark] +class ShuffleBlockManager(blockManager: BlockManager) { - // Get a group of writers for a map task. - def acquireWriters(mapId: Int): ShuffleWriterGroup = { - val bufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024 - val writers = Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => - val blockId = ShuffleBlockManager.blockId(shuffleId, bucketId, mapId) - blockManager.getDiskBlockWriter(blockId, serializer, bufferSize).open() + def forShuffle(shuffleId: Int, numBuckets: Int, serializer: Serializer): ShuffleBlocks = { + new ShuffleBlocks { + // Get a group of writers for a map task. + override def acquireWriters(mapId: Int): ShuffleWriterGroup = { + val bufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024 + val writers = Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => + val blockId = ShuffleBlockManager.blockId(shuffleId, bucketId, mapId) + blockManager.getDiskBlockWriter(blockId, serializer, bufferSize).open() + } + new ShuffleWriterGroup(mapId, writers) } - new ShuffleWriterGroup(mapId, writers) - } - def releaseWriters(group: ShuffleWriterGroup) = { - // Nothing really to release here. + override def releaseWriters(group: ShuffleWriterGroup) = { + // Nothing really to release here. + } } } } -- cgit v1.2.3 From 6fae936088d2a50606ba5082cee4a3c3a98a2b01 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 2 May 2013 22:30:06 -0700 Subject: applications (aka drivers) send their webUI address to master when registering so it can be displayed in the master web ui --- .../scala/spark/deploy/ApplicationDescription.scala | 4 +++- .../src/main/scala/spark/deploy/client/TestClient.scala | 2 +- .../scala/spark/deploy/master/ApplicationInfo.scala | 8 +++++++- core/src/main/scala/spark/deploy/master/Master.scala | 2 +- .../scheduler/cluster/SparkDeploySchedulerBackend.scala | 3 ++- core/src/main/scala/spark/storage/BlockManagerUI.scala | 17 +++++++++-------- .../twirl/spark/deploy/master/app_details.scala.html | 1 + 7 files changed, 24 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/spark/deploy/ApplicationDescription.scala index 6659e53b25..bb9e7b3bba 100644 --- a/core/src/main/scala/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/spark/deploy/ApplicationDescription.scala @@ -5,7 +5,9 @@ private[spark] class ApplicationDescription( val cores: Int, val memoryPerSlave: Int, val command: Command, - val sparkHome: String) + val sparkHome: String, + val appUIHost: String, + val appUIPort: Int) extends Serializable { val user = System.getProperty("user.name", "") diff --git a/core/src/main/scala/spark/deploy/client/TestClient.scala b/core/src/main/scala/spark/deploy/client/TestClient.scala index ad92532b58..e4ab01dd2a 100644 --- a/core/src/main/scala/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/spark/deploy/client/TestClient.scala @@ -25,7 +25,7 @@ private[spark] object TestClient { val url = args(0) val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0) val desc = new ApplicationDescription( - "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home") + "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "localhost", 0) val listener = new TestListener val client = new Client(actorSystem, url, desc, listener) client.start() diff --git a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala index 3591a94072..3ee1b60351 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala @@ -10,7 +10,9 @@ private[spark] class ApplicationInfo( val id: String, val desc: ApplicationDescription, val submitDate: Date, - val driver: ActorRef) + val driver: ActorRef, + val appUIHost: String, + val appUIPort: Int) { var state = ApplicationState.WAITING var executors = new mutable.HashMap[Int, ExecutorInfo] @@ -60,4 +62,8 @@ private[spark] class ApplicationInfo( System.currentTimeMillis() - startTime } } + + + def appUIAddress = "http://" + this.appUIHost + ":" + this.appUIPort + } diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 160afe5239..9f2d3da495 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -244,7 +244,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act def addApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = { val now = System.currentTimeMillis() val date = new Date(now) - val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver) + val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUIHost, desc.appUIPort) apps += app idToApp(app.id) = app actorToApp(driver) = app diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 0b8922d139..5d7d1feb74 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -31,7 +31,8 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command("spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome().getOrElse( throw new IllegalArgumentException("must supply spark home for spark standalone")) - val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome) + val appDesc = + new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, sc.ui.host, sc.ui.port) client = new Client(sc.env.actorSystem, master, appDesc, this) client.start() diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index 07da572044..13158e4262 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -20,19 +20,20 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, val STATIC_RESOURCE_DIR = "spark/deploy/static" implicit val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + val host = Utils.localHostName() + val port = if (System.getProperty("spark.ui.port") != null) { + System.getProperty("spark.ui.port").toInt + } else { + // TODO: Unfortunately, it's not possible to pass port 0 to spray and figure out which + // random port it bound to, so we have to try to find a local one by creating a socket. + Utils.findFreePort() + } /** Start a HTTP server to run the Web interface */ def start() { try { - val port = if (System.getProperty("spark.ui.port") != null) { - System.getProperty("spark.ui.port").toInt - } else { - // TODO: Unfortunately, it's not possible to pass port 0 to spray and figure out which - // random port it bound to, so we have to try to find a local one by creating a socket. - Utils.findFreePort() - } AkkaUtils.startSprayServer(actorSystem, "0.0.0.0", port, handler, "BlockManagerHTTPServer") - logInfo("Started BlockManager web UI at http://%s:%d".format(Utils.localHostName(), port)) + logInfo("Started BlockManager web UI at http://%s:%d".format(host, port)) } catch { case e: Exception => logError("Failed to create BlockManager WebUI", e) diff --git a/core/src/main/twirl/spark/deploy/master/app_details.scala.html b/core/src/main/twirl/spark/deploy/master/app_details.scala.html index 301a7e2124..02086b476f 100644 --- a/core/src/main/twirl/spark/deploy/master/app_details.scala.html +++ b/core/src/main/twirl/spark/deploy/master/app_details.scala.html @@ -22,6 +22,7 @@
  • Memory per Slave: @app.desc.memoryPerSlave
  • Submit Date: @app.submitDate
  • State: @app.state
  • +
  • Application Detail UI
  • -- cgit v1.2.3 From bb8a434f9db177d764d169a7273c66ed01c066c1 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 3 May 2013 15:14:02 -0700 Subject: Add zipPartitions to Java API. --- .../main/scala/spark/api/java/JavaRDDLike.scala | 15 +++++++++++++ .../spark/api/java/function/FlatMapFunction2.scala | 11 +++++++++ core/src/test/scala/spark/JavaAPISuite.java | 26 ++++++++++++++++++++++ 3 files changed, 52 insertions(+) create mode 100644 core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index d884529d7a..9b74d1226f 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -182,6 +182,21 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { JavaPairRDD.fromRDD(rdd.zip(other.rdd)(other.classManifest))(classManifest, other.classManifest) } + /** + * Zip this RDD's partitions with one (or more) RDD(s) and return a new RDD by + * applying a function to the zipped partitions. Assumes that all the RDDs have the + * *same number of partitions*, but does *not* require them to have the same number + * of elements in each partition. + */ + def zipPartitions[U, V]( + f: FlatMapFunction2[java.util.Iterator[T], java.util.Iterator[U], V], + other: JavaRDDLike[U, _]): JavaRDD[V] = { + def fn = (x: Iterator[T], y: Iterator[U]) => asScalaIterator( + f.apply(asJavaIterator(x), asJavaIterator(y)).iterator()) + JavaRDD.fromRDD( + rdd.zipPartitions(fn, other.rdd)(other.classManifest, f.elementType()))(f.elementType()) + } + // Actions (launch a job to return a value to the user program) /** diff --git a/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala b/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala new file mode 100644 index 0000000000..6044043add --- /dev/null +++ b/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala @@ -0,0 +1,11 @@ +package spark.api.java.function + +/** + * A function that takes two inputs and returns zero or more output records. + */ +abstract class FlatMapFunction2[A, B, C] extends Function2[A, B, java.lang.Iterable[C]] { + @throws(classOf[Exception]) + def call(a: A, b:B) : java.lang.Iterable[C] + + def elementType() : ClassManifest[C] = ClassManifest.Any.asInstanceOf[ClassManifest[C]] +} diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index d3dcd3bbeb..93bb69b41c 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -632,6 +632,32 @@ public class JavaAPISuite implements Serializable { zipped.count(); } + @Test + public void zipPartitions() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6), 2); + JavaRDD rdd2 = sc.parallelize(Arrays.asList("1", "2", "3", "4"), 2); + FlatMapFunction2, Iterator, Integer> sizesFn = + new FlatMapFunction2, Iterator, Integer>() { + @Override + public Iterable call(Iterator i, Iterator s) { + int sizeI = 0; + int sizeS = 0; + while (i.hasNext()) { + sizeI += 1; + i.next(); + } + while (s.hasNext()) { + sizeS += 1; + s.next(); + } + return Arrays.asList(sizeI, sizeS); + } + }; + + JavaRDD sizes = rdd1.zipPartitions(sizesFn, rdd2); + Assert.assertEquals("[3, 2, 3, 2]", sizes.collect().toString()); + } + @Test public void accumulators() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); -- cgit v1.2.3 From 2274ad0786b758e3170e96815e9693ea27635f06 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 3 May 2013 16:29:36 -0700 Subject: Fix flaky test by changing catch and adding sleep --- core/src/test/scala/spark/DistributedSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index ab3e197035..a13c88cfb4 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -261,9 +261,9 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter val data = sc.parallelize(Seq(true, false, false, false), 4) data.persist(StorageLevel.MEMORY_ONLY_2) data.count - assert(sc.persistentRdds.isEmpty == false) + assert(sc.persistentRdds.isEmpty === false) data.unpersist() - assert(sc.persistentRdds.isEmpty == true) + assert(sc.persistentRdds.isEmpty === true) failAfter(Span(3000, Millis)) { try { @@ -271,12 +271,12 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter Thread.sleep(200) } } catch { - case e: Exception => + case _ => { Thread.sleep(10) } // Do nothing. We might see exceptions because block manager // is racing this thread to remove entries from the driver. } } - assert(sc.getRDDStorageInfo.isEmpty == true) + assert(sc.getRDDStorageInfo.isEmpty === true) } } -- cgit v1.2.3 From edb57c8331738403d66c15ed99996e8bfb0488f7 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Sat, 4 May 2013 19:47:45 +0530 Subject: Add support for instance local in getPreferredLocations of ZippedPartitionsBaseRDD. Add comments to both ZippedPartitionsBaseRDD and ZippedRDD to better describe the potential problem with the approach --- .../main/scala/spark/rdd/ZippedPartitionsRDD.scala | 28 +++++++++++++++++++--- core/src/main/scala/spark/rdd/ZippedRDD.scala | 20 ++++++++++------ 2 files changed, 38 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala index fc3f29ffcd..dd9f3c2680 100644 --- a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala @@ -1,6 +1,6 @@ package spark.rdd -import spark.{OneToOneDependency, RDD, SparkContext, Partition, TaskContext} +import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} private[spark] class ZippedPartitionsPartition( @@ -38,9 +38,31 @@ abstract class ZippedPartitionsBaseRDD[V: ClassManifest]( } override def getPreferredLocations(s: Partition): Seq[String] = { + // Note that as number of rdd's increase and/or number of slaves in cluster increase, the computed preferredLocations below + // become diminishingly small : so we might need to look at alternate strategies to alleviate this. + // If there are no (or very small number of preferred locations), we will end up transferred the blocks to 'any' node in the + // cluster - paying with n/w and cache cost. + // Maybe pick a node which figures max amount of time ? + // Choose node which is hosting 'larger' of some subset of blocks ? + // Look at rack locality to ensure chosen host is atleast rack local to both hosting node ?, etc (would be good to defer this if possible) val splits = s.asInstanceOf[ZippedPartitionsPartition].partitions - val preferredLocations = rdds.zip(splits).map(x => x._1.preferredLocations(x._2)) - preferredLocations.reduce((x, y) => x.intersect(y)) + val rddSplitZip = rdds.zip(splits) + + // exact match. + val exactMatchPreferredLocations = rddSplitZip.map(x => x._1.preferredLocations(x._2)) + val exactMatchLocations = exactMatchPreferredLocations.reduce((x, y) => x.intersect(y)) + + // Remove exact match and then do host local match. + val otherNodePreferredLocations = rddSplitZip.map(x => { + x._1.preferredLocations(x._2).map(hostPort => { + val host = Utils.parseHostPort(hostPort)._1 + + if (exactMatchLocations.contains(host)) null else host + }).filter(_ != null) + }) + val otherNodeLocalLocations = otherNodePreferredLocations.reduce((x, y) => x.intersect(y)) + + otherNodeLocalLocations ++ exactMatchLocations } override def clearDependencies() { diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala index 51573fe68a..f728e93d24 100644 --- a/core/src/main/scala/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala @@ -48,21 +48,27 @@ class ZippedRDD[T: ClassManifest, U: ClassManifest]( } override def getPreferredLocations(s: Partition): Seq[String] = { + // Note that as number of slaves in cluster increase, the computed preferredLocations can become small : so we might need + // to look at alternate strategies to alleviate this. (If there are no (or very small number of preferred locations), we + // will end up transferred the blocks to 'any' node in the cluster - paying with n/w and cache cost. + // Maybe pick one or the other ? (so that atleast one block is local ?). + // Choose node which is hosting 'larger' of the blocks ? + // Look at rack locality to ensure chosen host is atleast rack local to both hosting node ?, etc (would be good to defer this if possible) val (partition1, partition2) = s.asInstanceOf[ZippedPartition[T, U]].partitions val pref1 = rdd1.preferredLocations(partition1) val pref2 = rdd2.preferredLocations(partition2) - // both partitions are instance local. - val instanceLocalLocations = pref1.intersect(pref2) + // exact match - instance local and host local. + val exactMatchLocations = pref1.intersect(pref2) - // remove locations which are already handled via instanceLocalLocations, and intersect where both partitions are node local. - val nodeLocalPref1 = pref1.filter(loc => ! instanceLocalLocations.contains(loc)).map(loc => Utils.parseHostPort(loc)._1) - val nodeLocalPref2 = pref2.filter(loc => ! instanceLocalLocations.contains(loc)).map(loc => Utils.parseHostPort(loc)._1) - val nodeLocalLocations = nodeLocalPref1.intersect(nodeLocalPref2) + // remove locations which are already handled via exactMatchLocations, and intersect where both partitions are node local. + val otherNodeLocalPref1 = pref1.filter(loc => ! exactMatchLocations.contains(loc)).map(loc => Utils.parseHostPort(loc)._1) + val otherNodeLocalPref2 = pref2.filter(loc => ! exactMatchLocations.contains(loc)).map(loc => Utils.parseHostPort(loc)._1) + val otherNodeLocalLocations = otherNodeLocalPref1.intersect(otherNodeLocalPref2) // Can have mix of instance local (hostPort) and node local (host) locations as preference ! - instanceLocalLocations ++ nodeLocalLocations + exactMatchLocations ++ otherNodeLocalLocations } override def clearDependencies() { -- cgit v1.2.3 From 02e8cfa61792f296555c7ed16613a91d895181a1 Mon Sep 17 00:00:00 2001 From: Ethan Jewett Date: Sat, 4 May 2013 12:31:30 -0500 Subject: HBase example --- .../src/main/scala/spark/examples/HBaseTest.scala | 34 ++++++++++++++++++++++ project/SparkBuild.scala | 6 +++- 2 files changed, 39 insertions(+), 1 deletion(-) create mode 100644 examples/src/main/scala/spark/examples/HBaseTest.scala diff --git a/examples/src/main/scala/spark/examples/HBaseTest.scala b/examples/src/main/scala/spark/examples/HBaseTest.scala new file mode 100644 index 0000000000..90ff64b483 --- /dev/null +++ b/examples/src/main/scala/spark/examples/HBaseTest.scala @@ -0,0 +1,34 @@ +package spark.examples + +import spark._ +import spark.rdd.NewHadoopRDD +import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor, HColumnDescriptor} +import org.apache.hadoop.hbase.client.HBaseAdmin +import org.apache.hadoop.hbase.mapreduce.TableInputFormat + +object HBaseTest { + def main(args: Array[String]) { + val sc = new SparkContext(args(0), "HBaseTest", + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + + val conf = HBaseConfiguration.create() + conf.set(TableInputFormat.INPUT_TABLE, args(1)) + + // Initialize hBase tables if necessary + val admin = new HBaseAdmin(conf) + if(!admin.isTableAvailable(args(1))) { + val colDesc = new HColumnDescriptor(args(2)) + val tableDesc = new HTableDescriptor(args(1)) + tableDesc.addFamily(colDesc) + admin.createTable(tableDesc) + } + + val hBaseRDD = new NewHadoopRDD(sc, classOf[TableInputFormat], + classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable], + classOf[org.apache.hadoop.hbase.client.Result], conf) + + hBaseRDD.count() + + System.exit(0) + } +} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 190d723435..6f5607d31c 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -200,7 +200,11 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", - libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.11") + resolvers ++= Seq("Apache HBase" at "https://repository.apache.org/content/repositories/releases"), + libraryDependencies ++= Seq( + "com.twitter" % "algebird-core_2.9.2" % "0.1.11", + "org.apache.hbase" % "hbase" % "0.94.6" + ) ) def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") -- cgit v1.2.3 From 9290f16430f92c66d4ec3b1ec76e491ae7cf26dc Mon Sep 17 00:00:00 2001 From: Ethan Jewett Date: Sat, 4 May 2013 12:39:14 -0500 Subject: Remove unnecessary column family config --- examples/src/main/scala/spark/examples/HBaseTest.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/examples/src/main/scala/spark/examples/HBaseTest.scala b/examples/src/main/scala/spark/examples/HBaseTest.scala index 90ff64b483..37aedde302 100644 --- a/examples/src/main/scala/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/spark/examples/HBaseTest.scala @@ -2,7 +2,7 @@ package spark.examples import spark._ import spark.rdd.NewHadoopRDD -import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor, HColumnDescriptor} +import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor} import org.apache.hadoop.hbase.client.HBaseAdmin import org.apache.hadoop.hbase.mapreduce.TableInputFormat @@ -14,12 +14,10 @@ object HBaseTest { val conf = HBaseConfiguration.create() conf.set(TableInputFormat.INPUT_TABLE, args(1)) - // Initialize hBase tables if necessary + // Initialize hBase table if necessary val admin = new HBaseAdmin(conf) if(!admin.isTableAvailable(args(1))) { - val colDesc = new HColumnDescriptor(args(2)) val tableDesc = new HTableDescriptor(args(1)) - tableDesc.addFamily(colDesc) admin.createTable(tableDesc) } -- cgit v1.2.3 From 7cff7e789723b646d1692fc71ef99f89a862bdc6 Mon Sep 17 00:00:00 2001 From: Ethan Jewett Date: Sat, 4 May 2013 14:56:55 -0500 Subject: Fix indents and mention other configuration options --- examples/src/main/scala/spark/examples/HBaseTest.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/examples/src/main/scala/spark/examples/HBaseTest.scala b/examples/src/main/scala/spark/examples/HBaseTest.scala index 37aedde302..d94b25828d 100644 --- a/examples/src/main/scala/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/spark/examples/HBaseTest.scala @@ -12,6 +12,9 @@ object HBaseTest { System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) val conf = HBaseConfiguration.create() + + // Other options for configuring scan behavior are available. More information available at + // http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html conf.set(TableInputFormat.INPUT_TABLE, args(1)) // Initialize hBase table if necessary @@ -22,8 +25,8 @@ object HBaseTest { } val hBaseRDD = new NewHadoopRDD(sc, classOf[TableInputFormat], - classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable], - classOf[org.apache.hadoop.hbase.client.Result], conf) + classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable], + classOf[org.apache.hadoop.hbase.client.Result], conf) hBaseRDD.count() -- cgit v1.2.3 From d48e9fde01cec2a7db794edf4cbe66c2228531aa Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 4 May 2013 12:36:47 -0700 Subject: Fix SPARK-629: weird number of cores in job details page. --- core/src/main/scala/spark/deploy/ApplicationDescription.scala | 2 +- core/src/main/scala/spark/deploy/JsonProtocol.scala | 4 ++-- core/src/main/scala/spark/deploy/master/ApplicationInfo.scala | 2 +- .../src/main/twirl/spark/deploy/master/app_details.scala.html | 11 ++++------- 4 files changed, 8 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/spark/deploy/ApplicationDescription.scala index 6659e53b25..b6b9f9bf9d 100644 --- a/core/src/main/scala/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/spark/deploy/ApplicationDescription.scala @@ -2,7 +2,7 @@ package spark.deploy private[spark] class ApplicationDescription( val name: String, - val cores: Int, + val maxCores: Int, /* Integer.MAX_VALUE denotes an unlimited number of cores */ val memoryPerSlave: Int, val command: Command, val sparkHome: String) diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index 71a641a9ef..ea832101d2 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -26,7 +26,7 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol { "starttime" -> JsNumber(obj.startTime), "id" -> JsString(obj.id), "name" -> JsString(obj.desc.name), - "cores" -> JsNumber(obj.desc.cores), + "cores" -> JsNumber(obj.desc.maxCores), "user" -> JsString(obj.desc.user), "memoryperslave" -> JsNumber(obj.desc.memoryPerSlave), "submitdate" -> JsString(obj.submitDate.toString)) @@ -35,7 +35,7 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol { implicit object AppDescriptionJsonFormat extends RootJsonWriter[ApplicationDescription] { def write(obj: ApplicationDescription) = JsObject( "name" -> JsString(obj.name), - "cores" -> JsNumber(obj.cores), + "cores" -> JsNumber(obj.maxCores), "memoryperslave" -> JsNumber(obj.memoryPerSlave), "user" -> JsString(obj.user) ) diff --git a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala index 3591a94072..70e5caab66 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala @@ -37,7 +37,7 @@ private[spark] class ApplicationInfo( coresGranted -= exec.cores } - def coresLeft: Int = desc.cores - coresGranted + def coresLeft: Int = desc.maxCores - coresGranted private var _retryCount = 0 diff --git a/core/src/main/twirl/spark/deploy/master/app_details.scala.html b/core/src/main/twirl/spark/deploy/master/app_details.scala.html index 301a7e2124..66147e213f 100644 --- a/core/src/main/twirl/spark/deploy/master/app_details.scala.html +++ b/core/src/main/twirl/spark/deploy/master/app_details.scala.html @@ -9,15 +9,12 @@
  • ID: @app.id
  • Description: @app.desc.name
  • User: @app.desc.user
  • -
  • Cores: - @app.desc.cores - (@app.coresGranted Granted - @if(app.desc.cores == Integer.MAX_VALUE) { - +
  • Cores: + @if(app.desc.maxCores == Integer.MAX_VALUE) { + Unlimited (@app.coresGranted granted) } else { - , @app.coresLeft + @app.desc.maxCores (@app.coresGranted granted, @app.coresLeft left) } - )
  • Memory per Slave: @app.desc.memoryPerSlave
  • Submit Date: @app.submitDate
  • -- cgit v1.2.3 From c0688451a6a91f596d9c596383026ddbdcbb8bb0 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 4 May 2013 12:37:04 -0700 Subject: Fix wrong closing tags in web UI HTML. --- core/src/main/twirl/spark/deploy/master/executor_row.scala.html | 2 +- core/src/main/twirl/spark/deploy/master/worker_row.scala.html | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/twirl/spark/deploy/master/executor_row.scala.html b/core/src/main/twirl/spark/deploy/master/executor_row.scala.html index d2d80fad48..21e72c7aab 100644 --- a/core/src/main/twirl/spark/deploy/master/executor_row.scala.html +++ b/core/src/main/twirl/spark/deploy/master/executor_row.scala.html @@ -3,7 +3,7 @@ @executor.id - @executor.worker.id + @executor.worker.id @executor.cores @executor.memory diff --git a/core/src/main/twirl/spark/deploy/master/worker_row.scala.html b/core/src/main/twirl/spark/deploy/master/worker_row.scala.html index be69e9bf02..46277ca421 100644 --- a/core/src/main/twirl/spark/deploy/master/worker_row.scala.html +++ b/core/src/main/twirl/spark/deploy/master/worker_row.scala.html @@ -4,7 +4,7 @@ - @worker.id + @worker.id @{worker.host}:@{worker.port} @worker.state -- cgit v1.2.3 From 42b1953c5385135467b87e79266a0f6c23d63e7b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 4 May 2013 13:23:31 -0700 Subject: Fix SPARK-630: app details page shows finished executors as running. --- core/src/main/scala/spark/deploy/master/Master.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 160afe5239..707fe57983 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -275,6 +275,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act for (exec <- app.executors.values) { exec.worker.removeExecutor(exec) exec.worker.actor ! KillExecutor(exec.application.id, exec.id) + exec.state = ExecutorState.KILLED } app.markFinished(state) app.driver ! ApplicationRemoved(state.toString) -- cgit v1.2.3 From 0a2bed356b9ea604d317be97a6747588c5af29e4 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 4 May 2013 21:50:08 -0700 Subject: Fixed flaky unpersist test in DistributedSuite. --- core/src/test/scala/spark/DistributedSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index a13c88cfb4..4df3bb5b67 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -276,7 +276,6 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter // is racing this thread to remove entries from the driver. } } - assert(sc.getRDDStorageInfo.isEmpty === true) } } -- cgit v1.2.3 From e014c1d1cb4bd1037dc674ef474d0197267b399b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 5 May 2013 11:30:36 -0700 Subject: Fix SPARK-670: EC2 start command should require -i option. --- ec2/spark_ec2.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 9f2daad2b6..7affe6fffc 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -103,7 +103,7 @@ def parse_args(): parser.print_help() sys.exit(1) (action, cluster_name) = args - if opts.identity_file == None and action in ['launch', 'login']: + if opts.identity_file == None and action in ['launch', 'login', 'start']: print >> stderr, ("ERROR: The -i or --identity-file argument is " + "required for " + action) sys.exit(1) -- cgit v1.2.3 From 22a5063ae45300cdd759f51877dac850008e16ee Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Sun, 5 May 2013 12:19:11 -0700 Subject: switch from separating appUI host & port to combining into just appUiUrl --- core/src/main/scala/spark/deploy/ApplicationDescription.scala | 3 +-- core/src/main/scala/spark/deploy/client/TestClient.scala | 2 +- core/src/main/scala/spark/deploy/master/ApplicationInfo.scala | 6 +----- core/src/main/scala/spark/deploy/master/Master.scala | 2 +- .../scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala | 2 +- core/src/main/scala/spark/storage/BlockManagerUI.scala | 2 ++ core/src/main/twirl/spark/deploy/master/app_details.scala.html | 2 +- 7 files changed, 8 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/spark/deploy/ApplicationDescription.scala index bb9e7b3bba..4aff0aedc1 100644 --- a/core/src/main/scala/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/spark/deploy/ApplicationDescription.scala @@ -6,8 +6,7 @@ private[spark] class ApplicationDescription( val memoryPerSlave: Int, val command: Command, val sparkHome: String, - val appUIHost: String, - val appUIPort: Int) + val appUiUrl: String) extends Serializable { val user = System.getProperty("user.name", "") diff --git a/core/src/main/scala/spark/deploy/client/TestClient.scala b/core/src/main/scala/spark/deploy/client/TestClient.scala index e4ab01dd2a..f195082808 100644 --- a/core/src/main/scala/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/spark/deploy/client/TestClient.scala @@ -25,7 +25,7 @@ private[spark] object TestClient { val url = args(0) val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0) val desc = new ApplicationDescription( - "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "localhost", 0) + "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "ignored") val listener = new TestListener val client = new Client(actorSystem, url, desc, listener) client.start() diff --git a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala index 3ee1b60351..e28b007e30 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala @@ -11,8 +11,7 @@ private[spark] class ApplicationInfo( val desc: ApplicationDescription, val submitDate: Date, val driver: ActorRef, - val appUIHost: String, - val appUIPort: Int) + val appUiUrl: String) { var state = ApplicationState.WAITING var executors = new mutable.HashMap[Int, ExecutorInfo] @@ -63,7 +62,4 @@ private[spark] class ApplicationInfo( } } - - def appUIAddress = "http://" + this.appUIHost + ":" + this.appUIPort - } diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 9f2d3da495..6f58ad16af 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -244,7 +244,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act def addApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = { val now = System.currentTimeMillis() val date = new Date(now) - val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUIHost, desc.appUIPort) + val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUiUrl) apps += app idToApp(app.id) = app actorToApp(driver) = app diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 5d7d1feb74..955ee5d806 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -32,7 +32,7 @@ private[spark] class SparkDeploySchedulerBackend( val sparkHome = sc.getSparkHome().getOrElse( throw new IllegalArgumentException("must supply spark home for spark standalone")) val appDesc = - new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, sc.ui.host, sc.ui.port) + new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, sc.ui.appUIAddress) client = new Client(sc.env.actorSystem, master, appDesc, this) client.start() diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index 13158e4262..e02281344a 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -74,4 +74,6 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, } } } + + private[spark] def appUIAddress = "http://" + host + ":" + port } diff --git a/core/src/main/twirl/spark/deploy/master/app_details.scala.html b/core/src/main/twirl/spark/deploy/master/app_details.scala.html index 02086b476f..15eabc9834 100644 --- a/core/src/main/twirl/spark/deploy/master/app_details.scala.html +++ b/core/src/main/twirl/spark/deploy/master/app_details.scala.html @@ -22,7 +22,7 @@
  • Memory per Slave: @app.desc.memoryPerSlave
  • Submit Date: @app.submitDate
  • State: @app.state
  • -
  • Application Detail UI
  • +
  • Application Detail UI
  • -- cgit v1.2.3 From cbf6a5ee1e7d290d04a0c5dac78d360266d415a4 Mon Sep 17 00:00:00 2001 From: unknown Date: Mon, 6 May 2013 08:05:45 -0600 Subject: Removed unused code, clarified intent of the program, batch size to 1 second --- .../scala/spark/streaming/examples/StatefulNetworkWordCount.scala | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala index b662cb1162..51c3c9f9b4 100644 --- a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -4,7 +4,7 @@ import spark.streaming._ import spark.streaming.StreamingContext._ /** - * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + * Counts words cumulatively in UTF8 encoded, '\n' delimited text received from the network every second. * Usage: StatefulNetworkWordCount * is the Spark master URL. In local mode, should be 'local[n]' with n > 1. * and describe the TCP server that Spark Streaming would connect to receive data. @@ -15,8 +15,6 @@ import spark.streaming.StreamingContext._ * `$ ./run spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` */ object StatefulNetworkWordCount { - private def className[A](a: A)(implicit m: Manifest[A]) = m.toString - def main(args: Array[String]) { if (args.length < 3) { System.err.println("Usage: StatefulNetworkWordCount \n" + @@ -32,8 +30,8 @@ object StatefulNetworkWordCount { Some(currentCount + previousCount) } - // Create the context with a 10 second batch size - val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", Seconds(10), + // Create the context with a 1 second batch size + val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", Seconds(1), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) ssc.checkpoint(".") -- cgit v1.2.3 From 0fd84965f66aa37d2ae14da799b86a5c8ed1cb32 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 6 May 2013 15:35:18 -0700 Subject: Added EmptyRDD. --- core/src/main/scala/spark/rdd/EmptyRDD.scala | 16 ++++++++++++++++ core/src/test/scala/spark/RDDSuite.scala | 14 +++++++++++++- 2 files changed, 29 insertions(+), 1 deletion(-) create mode 100644 core/src/main/scala/spark/rdd/EmptyRDD.scala diff --git a/core/src/main/scala/spark/rdd/EmptyRDD.scala b/core/src/main/scala/spark/rdd/EmptyRDD.scala new file mode 100644 index 0000000000..e4dd3a7fa7 --- /dev/null +++ b/core/src/main/scala/spark/rdd/EmptyRDD.scala @@ -0,0 +1,16 @@ +package spark.rdd + +import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} + + +/** + * An RDD that is empty, i.e. has no element in it. + */ +class EmptyRDD[T: ClassManifest](sc: SparkContext) extends RDD[T](sc, Nil) { + + override def getPartitions: Array[Partition] = Array.empty + + override def compute(split: Partition, context: TaskContext): Iterator[T] = { + throw new UnsupportedOperationException("empty RDD") + } +} diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index cee6312572..2ce757b13c 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -5,7 +5,7 @@ import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts._ import org.scalatest.time.{Span, Millis} import spark.SparkContext._ -import spark.rdd.{CoalescedRDD, CoGroupedRDD, PartitionPruningRDD, ShuffledRDD} +import spark.rdd.{CoalescedRDD, CoGroupedRDD, EmptyRDD, PartitionPruningRDD, ShuffledRDD} class RDDSuite extends FunSuite with LocalSparkContext { @@ -147,6 +147,18 @@ class RDDSuite extends FunSuite with LocalSparkContext { assert(rdd.collect().toList === List(1, 2, 3, 4)) } + test("empty RDD") { + sc = new SparkContext("local", "test") + val empty = new EmptyRDD[Int](sc) + assert(empty.count === 0) + assert(empty.collect().size === 0) + + val thrown = intercept[UnsupportedOperationException]{ + empty.reduce(_+_) + } + assert(thrown.getMessage.contains("empty")) + } + test("cogrouped RDDs") { sc = new SparkContext("local", "test") val rdd1 = sc.makeRDD(Array((1, "one"), (1, "another one"), (2, "two"), (3, "three")), 2) -- cgit v1.2.3 From 64d4d2b036447f42bfcd3bac5687c79a3b0661ca Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 6 May 2013 16:30:46 -0700 Subject: Added tests for joins, cogroups, and unions for EmptyRDD. --- core/src/test/scala/spark/RDDSuite.scala | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 2ce757b13c..a761dd77c5 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -157,6 +157,14 @@ class RDDSuite extends FunSuite with LocalSparkContext { empty.reduce(_+_) } assert(thrown.getMessage.contains("empty")) + + val emptyKv = new EmptyRDD[(Int, Int)](sc) + val rdd = sc.parallelize(1 to 2, 2).map(x => (x, x)) + assert(rdd.join(emptyKv).collect().size === 0) + assert(rdd.rightOuterJoin(emptyKv).collect().size === 0) + assert(rdd.leftOuterJoin(emptyKv).collect().size === 2) + assert(rdd.cogroup(emptyKv).collect().size === 2) + assert(rdd.union(emptyKv).collect().size === 2) } test("cogrouped RDDs") { -- cgit v1.2.3 From 4d8919d33056a006ebf6b1ddb0509aeccaa828d7 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Thu, 18 Apr 2013 14:58:38 -0700 Subject: Update Maven build to Scala 2.9.3 --- core/pom.xml | 4 ++-- pom.xml | 11 +++++------ 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index da26d674ec..9a019b5a42 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -73,7 +73,7 @@
    cc.spray - spray-json_${scala.version} + spray-json_2.9.2 org.tomdz.twirl @@ -81,7 +81,7 @@ com.github.scala-incubator.io - scala-io-file_${scala.version} + scala-io-file_2.9.2 org.apache.mesos diff --git a/pom.xml b/pom.xml index c3323ffad0..3936165d78 100644 --- a/pom.xml +++ b/pom.xml @@ -51,7 +51,7 @@ UTF-8 1.5 - 2.9.2 + 2.9.3 0.9.0-incubating 2.0.3 1.0-M2.1 @@ -238,7 +238,7 @@ cc.spray - spray-json_${scala.version} + spray-json_2.9.2 ${spray.json.version} @@ -248,7 +248,7 @@ com.github.scala-incubator.io - scala-io-file_${scala.version} + scala-io-file_2.9.2 0.4.1 @@ -277,7 +277,7 @@ org.scalatest scalatest_${scala.version} - 1.8 + 1.9.1 test @@ -289,7 +289,7 @@ org.scalacheck scalacheck_${scala.version} - 1.9 + 1.10.0 test @@ -513,7 +513,6 @@ hadoop1 - 1 -- cgit v1.2.3 From a3d5f922109caa878f8350fe0634514b8af55cbc Mon Sep 17 00:00:00 2001 From: Ethan Jewett Date: Tue, 7 May 2013 11:43:06 -0500 Subject: Switch to using SparkContext method to create RDD --- examples/src/main/scala/spark/examples/HBaseTest.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/examples/src/main/scala/spark/examples/HBaseTest.scala b/examples/src/main/scala/spark/examples/HBaseTest.scala index d94b25828d..9bad876860 100644 --- a/examples/src/main/scala/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/spark/examples/HBaseTest.scala @@ -24,9 +24,9 @@ object HBaseTest { admin.createTable(tableDesc) } - val hBaseRDD = new NewHadoopRDD(sc, classOf[TableInputFormat], + val hBaseRDD = sc.newAPIHadoopRDD(conf, classOf[TableInputFormat], classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable], - classOf[org.apache.hadoop.hbase.client.Result], conf) + classOf[org.apache.hadoop.hbase.client.Result]) hBaseRDD.count() -- cgit v1.2.3 From aacca1b8a85bd073ce185a06d6470b070761b2f4 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Thu, 18 Apr 2013 14:58:38 -0700 Subject: Update Maven build to Scala 2.9.3 --- core/pom.xml | 4 ++-- pom.xml | 11 +++++------ 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index da26d674ec..9a019b5a42 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -73,7 +73,7 @@ cc.spray - spray-json_${scala.version} + spray-json_2.9.2 org.tomdz.twirl @@ -81,7 +81,7 @@ com.github.scala-incubator.io - scala-io-file_${scala.version} + scala-io-file_2.9.2 org.apache.mesos diff --git a/pom.xml b/pom.xml index c3323ffad0..3936165d78 100644 --- a/pom.xml +++ b/pom.xml @@ -51,7 +51,7 @@ UTF-8 1.5 - 2.9.2 + 2.9.3 0.9.0-incubating 2.0.3 1.0-M2.1 @@ -238,7 +238,7 @@ cc.spray - spray-json_${scala.version} + spray-json_2.9.2 ${spray.json.version} @@ -248,7 +248,7 @@ com.github.scala-incubator.io - scala-io-file_${scala.version} + scala-io-file_2.9.2 0.4.1 @@ -277,7 +277,7 @@ org.scalatest scalatest_${scala.version} - 1.8 + 1.9.1 test @@ -289,7 +289,7 @@ org.scalacheck scalacheck_${scala.version} - 1.9 + 1.10.0 test @@ -513,7 +513,6 @@ hadoop1 - 1 -- cgit v1.2.3 From 8b7948517182ced5b3681dfc668732364ebccc38 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 7 May 2013 17:02:32 -0700 Subject: Moved BlockFetcherIterator to its own file. --- .../scala/spark/storage/BlockFetcherIterator.scala | 361 +++++++++++++++++++++ .../main/scala/spark/storage/BlockManager.scala | 340 +------------------ 2 files changed, 362 insertions(+), 339 deletions(-) create mode 100644 core/src/main/scala/spark/storage/BlockFetcherIterator.scala diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala new file mode 100644 index 0000000000..30990d9a38 --- /dev/null +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -0,0 +1,361 @@ +package spark.storage + +import java.nio.ByteBuffer +import java.util.concurrent.LinkedBlockingQueue + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashSet +import scala.collection.mutable.Queue + +import spark.Logging +import spark.Utils +import spark.SparkException + +import spark.network.BufferMessage +import spark.network.ConnectionManagerId +import spark.network.netty.ShuffleCopier + +import spark.serializer.Serializer +import io.netty.buffer.ByteBuf + + +trait BlockFetcherIterator extends Iterator[(String, Option[Iterator[Any]])] + with Logging with BlockFetchTracker { + + def initialize() + +} + + + +object BlockFetcherIterator { + + // A request to fetch one or more blocks, complete with their sizes + class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) { + val size = blocks.map(_._2).sum + } + + // A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize + // the block (since we want all deserializaton to happen in the calling thread); can also + // represent a fetch failure if size == -1. + class FetchResult(val blockId: String, val size: Long, val deserialize: () => Iterator[Any]) { + def failed: Boolean = size == -1 + } + + class BasicBlockFetcherIterator( + private val blockManager: BlockManager, + val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], + serializer: Serializer + ) extends BlockFetcherIterator { + + import blockManager._ + + private var _remoteBytesRead = 0l + private var _remoteFetchTime = 0l + private var _fetchWaitTime = 0l + + if (blocksByAddress == null) { + throw new IllegalArgumentException("BlocksByAddress is null") + } + var totalBlocks = blocksByAddress.map(_._2.size).sum + logDebug("Getting " + totalBlocks + " blocks") + var startTime = System.currentTimeMillis + val localBlockIds = new ArrayBuffer[String]() + val remoteBlockIds = new HashSet[String]() + + // A queue to hold our results. + val results = new LinkedBlockingQueue[FetchResult] + + // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that + // the number of bytes in flight is limited to maxBytesInFlight + val fetchRequests = new Queue[FetchRequest] + + // Current bytes in flight from our requests + var bytesInFlight = 0L + + def sendRequest(req: FetchRequest) { + logDebug("Sending request for %d blocks (%s) from %s".format( + req.blocks.size, Utils.memoryBytesToString(req.size), req.address.hostPort)) + val cmId = new ConnectionManagerId(req.address.host, req.address.port) + val blockMessageArray = new BlockMessageArray(req.blocks.map { + case (blockId, size) => BlockMessage.fromGetBlock(GetBlock(blockId)) + }) + bytesInFlight += req.size + val sizeMap = req.blocks.toMap // so we can look up the size of each blockID + val fetchStart = System.currentTimeMillis() + val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) + future.onSuccess { + case Some(message) => { + val fetchDone = System.currentTimeMillis() + _remoteFetchTime += fetchDone - fetchStart + val bufferMessage = message.asInstanceOf[BufferMessage] + val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) + for (blockMessage <- blockMessageArray) { + if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) { + throw new SparkException( + "Unexpected message " + blockMessage.getType + " received from " + cmId) + } + val blockId = blockMessage.getId + results.put(new FetchResult(blockId, sizeMap(blockId), + () => dataDeserialize(blockId, blockMessage.getData, serializer))) + _remoteBytesRead += req.size + logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) + } + } + case None => { + logError("Could not get block(s) from " + cmId) + for ((blockId, size) <- req.blocks) { + results.put(new FetchResult(blockId, -1, null)) + } + } + } + } + + def splitLocalRemoteBlocks():ArrayBuffer[FetchRequest] = { + // Split local and remote blocks. Remote blocks are further split into FetchRequests of size + // at most maxBytesInFlight in order to limit the amount of data in flight. + val remoteRequests = new ArrayBuffer[FetchRequest] + for ((address, blockInfos) <- blocksByAddress) { + if (address == blockManagerId) { + localBlockIds ++= blockInfos.map(_._1) + } else { + remoteBlockIds ++= blockInfos.map(_._1) + // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + val minRequestSize = math.max(maxBytesInFlight / 5, 1L) + logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) + val iterator = blockInfos.iterator + var curRequestSize = 0L + var curBlocks = new ArrayBuffer[(String, Long)] + while (iterator.hasNext) { + val (blockId, size) = iterator.next() + curBlocks += ((blockId, size)) + curRequestSize += size + if (curRequestSize >= minRequestSize) { + // Add this FetchRequest + remoteRequests += new FetchRequest(address, curBlocks) + curRequestSize = 0 + curBlocks = new ArrayBuffer[(String, Long)] + } + } + // Add in the final request + if (!curBlocks.isEmpty) { + remoteRequests += new FetchRequest(address, curBlocks) + } + } + } + remoteRequests + } + + def getLocalBlocks(){ + // Get the local blocks while remote blocks are being fetched. Note that it's okay to do + // these all at once because they will just memory-map some files, so they won't consume + // any memory that might exceed our maxBytesInFlight + for (id <- localBlockIds) { + getLocal(id) match { + case Some(iter) => { + results.put(new FetchResult(id, 0, () => iter)) // Pass 0 as size since it's not in flight + logDebug("Got local block " + id) + } + case None => { + throw new BlockException(id, "Could not get block " + id + " from local machine") + } + } + } + } + + override def initialize(){ + // Split local and remote blocks. + val remoteRequests = splitLocalRemoteBlocks() + // Add the remote requests into our queue in a random order + fetchRequests ++= Utils.randomize(remoteRequests) + + // Send out initial requests for blocks, up to our maxBytesInFlight + while (!fetchRequests.isEmpty && + (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { + sendRequest(fetchRequests.dequeue()) + } + + val numGets = remoteBlockIds.size - fetchRequests.size + logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime)) + + // Get Local Blocks + startTime = System.currentTimeMillis + getLocalBlocks() + logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") + + } + + //an iterator that will read fetched blocks off the queue as they arrive. + @volatile protected var resultsGotten = 0 + + def hasNext: Boolean = resultsGotten < totalBlocks + + def next(): (String, Option[Iterator[Any]]) = { + resultsGotten += 1 + val startFetchWait = System.currentTimeMillis() + val result = results.take() + val stopFetchWait = System.currentTimeMillis() + _fetchWaitTime += (stopFetchWait - startFetchWait) + if (! result.failed) bytesInFlight -= result.size + while (!fetchRequests.isEmpty && + (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { + sendRequest(fetchRequests.dequeue()) + } + (result.blockId, if (result.failed) None else Some(result.deserialize())) + } + + + //methods to profile the block fetching + def numLocalBlocks = localBlockIds.size + def numRemoteBlocks = remoteBlockIds.size + + def remoteFetchTime = _remoteFetchTime + def fetchWaitTime = _fetchWaitTime + + def remoteBytesRead = _remoteBytesRead + + } + + + class NettyBlockFetcherIterator( + blockManager: BlockManager, + blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], + serializer: Serializer + ) extends BasicBlockFetcherIterator(blockManager,blocksByAddress,serializer) { + + import blockManager._ + + val fetchRequestsSync = new LinkedBlockingQueue[FetchRequest] + + def putResult(blockId:String, blockSize:Long, blockData:ByteBuffer, + results : LinkedBlockingQueue[FetchResult]){ + results.put(new FetchResult( + blockId, blockSize, () => dataDeserialize(blockId, blockData, serializer) )) + } + + def startCopiers (numCopiers: Int): List [ _ <: Thread]= { + (for ( i <- Range(0,numCopiers) ) yield { + val copier = new Thread { + override def run(){ + try { + while(!isInterrupted && !fetchRequestsSync.isEmpty) { + sendRequest(fetchRequestsSync.take()) + } + } catch { + case x: InterruptedException => logInfo("Copier Interrupted") + //case _ => throw new SparkException("Exception Throw in Shuffle Copier") + } + } + } + copier.start + copier + }).toList + } + + //keep this to interrupt the threads when necessary + def stopCopiers(copiers : List[_ <: Thread]) { + for (copier <- copiers) { + copier.interrupt() + } + } + + override def sendRequest(req: FetchRequest) { + logDebug("Sending request for %d blocks (%s) from %s".format( + req.blocks.size, Utils.memoryBytesToString(req.size), req.address.host)) + val cmId = new ConnectionManagerId(req.address.host, System.getProperty("spark.shuffle.sender.port", "6653").toInt) + val cpier = new ShuffleCopier + cpier.getBlocks(cmId,req.blocks,(blockId:String,blockSize:Long,blockData:ByteBuf) => putResult(blockId,blockSize,blockData.nioBuffer,results)) + logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.host ) + } + + override def splitLocalRemoteBlocks() : ArrayBuffer[FetchRequest] = { + // Split local and remote blocks. Remote blocks are further split into FetchRequests of size + // at most maxBytesInFlight in order to limit the amount of data in flight. + val originalTotalBlocks = totalBlocks; + val remoteRequests = new ArrayBuffer[FetchRequest] + for ((address, blockInfos) <- blocksByAddress) { + if (address == blockManagerId) { + localBlockIds ++= blockInfos.map(_._1) + } else { + remoteBlockIds ++= blockInfos.map(_._1) + // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + val minRequestSize = math.max(maxBytesInFlight / 5, 1L) + logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) + val iterator = blockInfos.iterator + var curRequestSize = 0L + var curBlocks = new ArrayBuffer[(String, Long)] + while (iterator.hasNext) { + val (blockId, size) = iterator.next() + if (size > 0) { + curBlocks += ((blockId, size)) + curRequestSize += size + } else if (size == 0){ + //here we changes the totalBlocks + totalBlocks -= 1 + } else { + throw new SparkException("Negative block size "+blockId) + } + if (curRequestSize >= minRequestSize) { + // Add this FetchRequest + remoteRequests += new FetchRequest(address, curBlocks) + curRequestSize = 0 + curBlocks = new ArrayBuffer[(String, Long)] + } + } + // Add in the final request + if (!curBlocks.isEmpty) { + remoteRequests += new FetchRequest(address, curBlocks) + } + } + } + logInfo("Getting " + totalBlocks + " non 0-byte blocks out of " + originalTotalBlocks + " blocks") + remoteRequests + } + + var copiers : List[_ <: Thread] = null + + override def initialize(){ + // Split Local Remote Blocks and adjust totalBlocks to include only the non 0-byte blocks + val remoteRequests = splitLocalRemoteBlocks() + // Add the remote requests into our queue in a random order + for (request <- Utils.randomize(remoteRequests)) { + fetchRequestsSync.put(request) + } + + copiers = startCopiers(System.getProperty("spark.shuffle.copier.threads", "6").toInt) + logInfo("Started " + fetchRequestsSync.size + " remote gets in " + Utils.getUsedTimeMs(startTime)) + + // Get Local Blocks + startTime = System.currentTimeMillis + getLocalBlocks() + logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") + } + + override def next(): (String, Option[Iterator[Any]]) = { + resultsGotten += 1 + val result = results.take() + // if all the results has been retrieved + // shutdown the copiers + if (resultsGotten == totalBlocks) { + if( copiers != null ) + stopCopiers(copiers) + } + (result.blockId, if (result.failed) None else Some(result.deserialize())) + } + } + + def apply(t: String, + blockManager: BlockManager, + blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], + serializer: Serializer): BlockFetcherIterator = { + val iter = if (t == "netty") { new NettyBlockFetcherIterator(blockManager,blocksByAddress, serializer) } + else { new BasicBlockFetcherIterator(blockManager,blocksByAddress, serializer) } + iter.initialize() + iter + } +} + diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 433e939656..a189c1a025 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -2,9 +2,8 @@ package spark.storage import java.io.{InputStream, OutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} -import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue} -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} +import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import akka.actor.{ActorSystem, Cancellable, Props} @@ -23,8 +22,6 @@ import spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStam import sun.nio.ch.DirectBuffer -import spark.network.netty.ShuffleCopier -import io.netty.buffer.ByteBuf private[spark] class BlockManager( @@ -977,338 +974,3 @@ object BlockManager extends Logging { } } - -trait BlockFetcherIterator extends Iterator[(String, Option[Iterator[Any]])] with Logging with BlockFetchTracker { - def initialize -} - -object BlockFetcherIterator { - - // A request to fetch one or more blocks, complete with their sizes - class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) { - val size = blocks.map(_._2).sum - } - - // A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize - // the block (since we want all deserializaton to happen in the calling thread); can also - // represent a fetch failure if size == -1. - class FetchResult(val blockId: String, val size: Long, val deserialize: () => Iterator[Any]) { - def failed: Boolean = size == -1 - } - -class BasicBlockFetcherIterator( - private val blockManager: BlockManager, - val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], - serializer: Serializer -) extends BlockFetcherIterator { - - import blockManager._ - - private var _remoteBytesRead = 0l - private var _remoteFetchTime = 0l - private var _fetchWaitTime = 0l - - if (blocksByAddress == null) { - throw new IllegalArgumentException("BlocksByAddress is null") - } - var totalBlocks = blocksByAddress.map(_._2.size).sum - logDebug("Getting " + totalBlocks + " blocks") - var startTime = System.currentTimeMillis - val localBlockIds = new ArrayBuffer[String]() - val remoteBlockIds = new HashSet[String]() - - // A queue to hold our results. - val results = new LinkedBlockingQueue[FetchResult] - - // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that - // the number of bytes in flight is limited to maxBytesInFlight - val fetchRequests = new Queue[FetchRequest] - - // Current bytes in flight from our requests - var bytesInFlight = 0L - - def sendRequest(req: FetchRequest) { - logDebug("Sending request for %d blocks (%s) from %s".format( - req.blocks.size, Utils.memoryBytesToString(req.size), req.address.hostPort)) - val cmId = new ConnectionManagerId(req.address.host, req.address.port) - val blockMessageArray = new BlockMessageArray(req.blocks.map { - case (blockId, size) => BlockMessage.fromGetBlock(GetBlock(blockId)) - }) - bytesInFlight += req.size - val sizeMap = req.blocks.toMap // so we can look up the size of each blockID - val fetchStart = System.currentTimeMillis() - val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) - future.onSuccess { - case Some(message) => { - val fetchDone = System.currentTimeMillis() - _remoteFetchTime += fetchDone - fetchStart - val bufferMessage = message.asInstanceOf[BufferMessage] - val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) - for (blockMessage <- blockMessageArray) { - if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) { - throw new SparkException( - "Unexpected message " + blockMessage.getType + " received from " + cmId) - } - val blockId = blockMessage.getId - results.put(new FetchResult(blockId, sizeMap(blockId), - () => dataDeserialize(blockId, blockMessage.getData, serializer))) - _remoteBytesRead += req.size - logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) - } - } - case None => { - logError("Could not get block(s) from " + cmId) - for ((blockId, size) <- req.blocks) { - results.put(new FetchResult(blockId, -1, null)) - } - } - } - } - - def splitLocalRemoteBlocks():ArrayBuffer[FetchRequest] = { - // Split local and remote blocks. Remote blocks are further split into FetchRequests of size - // at most maxBytesInFlight in order to limit the amount of data in flight. - val remoteRequests = new ArrayBuffer[FetchRequest] - for ((address, blockInfos) <- blocksByAddress) { - if (address == blockManagerId) { - localBlockIds ++= blockInfos.map(_._1) - } else { - remoteBlockIds ++= blockInfos.map(_._1) - // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them - // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 - // nodes, rather than blocking on reading output from one node. - val minRequestSize = math.max(maxBytesInFlight / 5, 1L) - logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) - val iterator = blockInfos.iterator - var curRequestSize = 0L - var curBlocks = new ArrayBuffer[(String, Long)] - while (iterator.hasNext) { - val (blockId, size) = iterator.next() - curBlocks += ((blockId, size)) - curRequestSize += size - if (curRequestSize >= minRequestSize) { - // Add this FetchRequest - remoteRequests += new FetchRequest(address, curBlocks) - curRequestSize = 0 - curBlocks = new ArrayBuffer[(String, Long)] - } - } - // Add in the final request - if (!curBlocks.isEmpty) { - remoteRequests += new FetchRequest(address, curBlocks) - } - } - } - remoteRequests - } - - def getLocalBlocks(){ - // Get the local blocks while remote blocks are being fetched. Note that it's okay to do - // these all at once because they will just memory-map some files, so they won't consume - // any memory that might exceed our maxBytesInFlight - for (id <- localBlockIds) { - getLocal(id) match { - case Some(iter) => { - results.put(new FetchResult(id, 0, () => iter)) // Pass 0 as size since it's not in flight - logDebug("Got local block " + id) - } - case None => { - throw new BlockException(id, "Could not get block " + id + " from local machine") - } - } - } - } - - def initialize(){ - // Split local and remote blocks. - val remoteRequests = splitLocalRemoteBlocks() - // Add the remote requests into our queue in a random order - fetchRequests ++= Utils.randomize(remoteRequests) - - // Send out initial requests for blocks, up to our maxBytesInFlight - while (!fetchRequests.isEmpty && - (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - sendRequest(fetchRequests.dequeue()) - } - - val numGets = remoteBlockIds.size - fetchRequests.size - logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime)) - - // Get Local Blocks - startTime = System.currentTimeMillis - getLocalBlocks() - logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") - - } - - //an iterator that will read fetched blocks off the queue as they arrive. - @volatile private var resultsGotten = 0 - - def hasNext: Boolean = resultsGotten < totalBlocks - - def next(): (String, Option[Iterator[Any]]) = { - resultsGotten += 1 - val startFetchWait = System.currentTimeMillis() - val result = results.take() - val stopFetchWait = System.currentTimeMillis() - _fetchWaitTime += (stopFetchWait - startFetchWait) - if (! result.failed) bytesInFlight -= result.size - while (!fetchRequests.isEmpty && - (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - sendRequest(fetchRequests.dequeue()) - } - (result.blockId, if (result.failed) None else Some(result.deserialize())) - } - - - //methods to profile the block fetching - def numLocalBlocks = localBlockIds.size - def numRemoteBlocks = remoteBlockIds.size - - def remoteFetchTime = _remoteFetchTime - def fetchWaitTime = _fetchWaitTime - - def remoteBytesRead = _remoteBytesRead - -} - -class NettyBlockFetcherIterator( - blockManager: BlockManager, - blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], - serializer: Serializer -) extends BasicBlockFetcherIterator(blockManager,blocksByAddress,serializer) { - - import blockManager._ - - val fetchRequestsSync = new LinkedBlockingQueue[FetchRequest] - - def putResult(blockId:String, blockSize:Long, blockData:ByteBuffer, - results : LinkedBlockingQueue[FetchResult]){ - results.put(new FetchResult( - blockId, blockSize, () => dataDeserialize(blockId, blockData, serializer) )) - } - - def startCopiers (numCopiers: Int): List [ _ <: Thread]= { - (for ( i <- Range(0,numCopiers) ) yield { - val copier = new Thread { - override def run(){ - try { - while(!isInterrupted && !fetchRequestsSync.isEmpty) { - sendRequest(fetchRequestsSync.take()) - } - } catch { - case x: InterruptedException => logInfo("Copier Interrupted") - //case _ => throw new SparkException("Exception Throw in Shuffle Copier") - } - } - } - copier.start - copier - }).toList - } - - //keep this to interrupt the threads when necessary - def stopCopiers(copiers : List[_ <: Thread]) { - for (copier <- copiers) { - copier.interrupt() - } - } - - override def sendRequest(req: FetchRequest) { - logDebug("Sending request for %d blocks (%s) from %s".format( - req.blocks.size, Utils.memoryBytesToString(req.size), req.address.ip)) - val cmId = new ConnectionManagerId(req.address.ip, System.getProperty("spark.shuffle.sender.port", "6653").toInt) - val cpier = new ShuffleCopier - cpier.getBlocks(cmId,req.blocks,(blockId:String,blockSize:Long,blockData:ByteBuf) => putResult(blockId,blockSize,blockData.nioBuffer,results)) - logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.ip ) - } - - override def splitLocalRemoteBlocks() : ArrayBuffer[FetchRequest] = { - // Split local and remote blocks. Remote blocks are further split into FetchRequests of size - // at most maxBytesInFlight in order to limit the amount of data in flight. - val originalTotalBlocks = totalBlocks; - val remoteRequests = new ArrayBuffer[FetchRequest] - for ((address, blockInfos) <- blocksByAddress) { - if (address == blockManagerId) { - localBlockIds ++= blockInfos.map(_._1) - } else { - remoteBlockIds ++= blockInfos.map(_._1) - // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them - // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 - // nodes, rather than blocking on reading output from one node. - val minRequestSize = math.max(maxBytesInFlight / 5, 1L) - logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) - val iterator = blockInfos.iterator - var curRequestSize = 0L - var curBlocks = new ArrayBuffer[(String, Long)] - while (iterator.hasNext) { - val (blockId, size) = iterator.next() - if (size > 0) { - curBlocks += ((blockId, size)) - curRequestSize += size - } else if (size == 0){ - //here we changes the totalBlocks - totalBlocks -= 1 - } else { - throw new SparkException("Negative block size "+blockId) - } - if (curRequestSize >= minRequestSize) { - // Add this FetchRequest - remoteRequests += new FetchRequest(address, curBlocks) - curRequestSize = 0 - curBlocks = new ArrayBuffer[(String, Long)] - } - } - // Add in the final request - if (!curBlocks.isEmpty) { - remoteRequests += new FetchRequest(address, curBlocks) - } - } - } - logInfo("Getting " + totalBlocks + " non 0-byte blocks out of " + originalTotalBlocks + " blocks") - remoteRequests - } - - var copiers : List[_ <: Thread] = null - - override def initialize(){ - // Split Local Remote Blocks and adjust totalBlocks to include only the non 0-byte blocks - val remoteRequests = splitLocalRemoteBlocks() - // Add the remote requests into our queue in a random order - for (request <- Utils.randomize(remoteRequests)) { - fetchRequestsSync.put(request) - } - - copiers = startCopiers(System.getProperty("spark.shuffle.copier.threads", "6").toInt) - logInfo("Started " + fetchRequestsSync.size + " remote gets in " + Utils.getUsedTimeMs(startTime)) - - // Get Local Blocks - startTime = System.currentTimeMillis - getLocalBlocks() - logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") - } - - override def next(): (String, Option[Iterator[Any]]) = { - resultsGotten += 1 - val result = results.take() - // if all the results has been retrieved - // shutdown the copiers - if (resultsGotten == totalBlocks) { - if( copiers != null ) - stopCopiers(copiers) - } - (result.blockId, if (result.failed) None else Some(result.deserialize())) - } - } - - def apply(t: String, - blockManager: BlockManager, - blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], - serializer: Serializer): BlockFetcherIterator = { - val iter = if (t == "netty") { new NettyBlockFetcherIterator(blockManager,blocksByAddress, serializer) } - else { new BasicBlockFetcherIterator(blockManager,blocksByAddress, serializer) } - iter.initialize - iter - } - -} -- cgit v1.2.3 From 0e5cc30868bcf933f2980c4cfe29abc3d8fe5887 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 7 May 2013 18:18:24 -0700 Subject: Cleaned up BlockManager and BlockFetcherIterator from Shane's PR. --- .../scala/spark/storage/BlockFetchTracker.scala | 12 +- .../scala/spark/storage/BlockFetcherIterator.scala | 167 ++++++++++----------- .../main/scala/spark/storage/BlockManager.scala | 22 +-- 3 files changed, 102 insertions(+), 99 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockFetchTracker.scala b/core/src/main/scala/spark/storage/BlockFetchTracker.scala index 993aece1f7..0718156b1b 100644 --- a/core/src/main/scala/spark/storage/BlockFetchTracker.scala +++ b/core/src/main/scala/spark/storage/BlockFetchTracker.scala @@ -1,10 +1,10 @@ package spark.storage private[spark] trait BlockFetchTracker { - def totalBlocks : Int - def numLocalBlocks: Int - def numRemoteBlocks: Int - def remoteFetchTime : Long - def fetchWaitTime: Long - def remoteBytesRead : Long + def totalBlocks : Int + def numLocalBlocks: Int + def numRemoteBlocks: Int + def remoteFetchTime : Long + def fetchWaitTime: Long + def remoteBytesRead : Long } diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index 30990d9a38..43f835237c 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -7,27 +7,36 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashSet import scala.collection.mutable.Queue +import io.netty.buffer.ByteBuf + import spark.Logging import spark.Utils import spark.SparkException - import spark.network.BufferMessage import spark.network.ConnectionManagerId import spark.network.netty.ShuffleCopier - import spark.serializer.Serializer -import io.netty.buffer.ByteBuf +/** + * A block fetcher iterator interface. There are two implementations: + * + * BasicBlockFetcherIterator: uses a custom-built NIO communication layer. + * NettyBlockFetcherIterator: uses Netty (OIO) as the communication layer. + * + * Eventually we would like the two to converge and use a single NIO-based communication layer, + * but extensive tests show that under some circumstances (e.g. large shuffles with lots of cores), + * NIO would perform poorly and thus the need for the Netty OIO one. + */ + +private[storage] trait BlockFetcherIterator extends Iterator[(String, Option[Iterator[Any]])] with Logging with BlockFetchTracker { - def initialize() - } - +private[storage] object BlockFetcherIterator { // A request to fetch one or more blocks, complete with their sizes @@ -45,8 +54,8 @@ object BlockFetcherIterator { class BasicBlockFetcherIterator( private val blockManager: BlockManager, val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], - serializer: Serializer - ) extends BlockFetcherIterator { + serializer: Serializer) + extends BlockFetcherIterator { import blockManager._ @@ -57,23 +66,24 @@ object BlockFetcherIterator { if (blocksByAddress == null) { throw new IllegalArgumentException("BlocksByAddress is null") } - var totalBlocks = blocksByAddress.map(_._2.size).sum - logDebug("Getting " + totalBlocks + " blocks") - var startTime = System.currentTimeMillis - val localBlockIds = new ArrayBuffer[String]() - val remoteBlockIds = new HashSet[String]() + + protected var _totalBlocks = blocksByAddress.map(_._2.size).sum + logDebug("Getting " + _totalBlocks + " blocks") + protected var startTime = System.currentTimeMillis + protected val localBlockIds = new ArrayBuffer[String]() + protected val remoteBlockIds = new HashSet[String]() // A queue to hold our results. - val results = new LinkedBlockingQueue[FetchResult] + protected val results = new LinkedBlockingQueue[FetchResult] // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that // the number of bytes in flight is limited to maxBytesInFlight - val fetchRequests = new Queue[FetchRequest] + private val fetchRequests = new Queue[FetchRequest] // Current bytes in flight from our requests - var bytesInFlight = 0L + private var bytesInFlight = 0L - def sendRequest(req: FetchRequest) { + protected def sendRequest(req: FetchRequest) { logDebug("Sending request for %d blocks (%s) from %s".format( req.blocks.size, Utils.memoryBytesToString(req.size), req.address.hostPort)) val cmId = new ConnectionManagerId(req.address.host, req.address.port) @@ -111,7 +121,7 @@ object BlockFetcherIterator { } } - def splitLocalRemoteBlocks():ArrayBuffer[FetchRequest] = { + protected def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { // Split local and remote blocks. Remote blocks are further split into FetchRequests of size // at most maxBytesInFlight in order to limit the amount of data in flight. val remoteRequests = new ArrayBuffer[FetchRequest] @@ -148,14 +158,15 @@ object BlockFetcherIterator { remoteRequests } - def getLocalBlocks(){ + protected def getLocalBlocks() { // Get the local blocks while remote blocks are being fetched. Note that it's okay to do // these all at once because they will just memory-map some files, so they won't consume // any memory that might exceed our maxBytesInFlight for (id <- localBlockIds) { getLocal(id) match { case Some(iter) => { - results.put(new FetchResult(id, 0, () => iter)) // Pass 0 as size since it's not in flight + // Pass 0 as size since it's not in flight + results.put(new FetchResult(id, 0, () => iter)) logDebug("Got local block " + id) } case None => { @@ -165,7 +176,7 @@ object BlockFetcherIterator { } } - override def initialize(){ + override def initialize() { // Split local and remote blocks. val remoteRequests = splitLocalRemoteBlocks() // Add the remote requests into our queue in a random order @@ -184,15 +195,14 @@ object BlockFetcherIterator { startTime = System.currentTimeMillis getLocalBlocks() logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") - } //an iterator that will read fetched blocks off the queue as they arrive. @volatile protected var resultsGotten = 0 - def hasNext: Boolean = resultsGotten < totalBlocks + override def hasNext: Boolean = resultsGotten < _totalBlocks - def next(): (String, Option[Iterator[Any]]) = { + override def next(): (String, Option[Iterator[Any]]) = { resultsGotten += 1 val startFetchWait = System.currentTimeMillis() val result = results.take() @@ -206,74 +216,73 @@ object BlockFetcherIterator { (result.blockId, if (result.failed) None else Some(result.deserialize())) } - - //methods to profile the block fetching - def numLocalBlocks = localBlockIds.size - def numRemoteBlocks = remoteBlockIds.size - - def remoteFetchTime = _remoteFetchTime - def fetchWaitTime = _fetchWaitTime - - def remoteBytesRead = _remoteBytesRead - + // Implementing BlockFetchTracker trait. + override def totalBlocks: Int = _totalBlocks + override def numLocalBlocks: Int = localBlockIds.size + override def numRemoteBlocks: Int = remoteBlockIds.size + override def remoteFetchTime: Long = _remoteFetchTime + override def fetchWaitTime: Long = _fetchWaitTime + override def remoteBytesRead: Long = _remoteBytesRead } - + // End of BasicBlockFetcherIterator class NettyBlockFetcherIterator( blockManager: BlockManager, blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], - serializer: Serializer - ) extends BasicBlockFetcherIterator(blockManager,blocksByAddress,serializer) { + serializer: Serializer) + extends BasicBlockFetcherIterator(blockManager, blocksByAddress, serializer) { import blockManager._ val fetchRequestsSync = new LinkedBlockingQueue[FetchRequest] - def putResult(blockId:String, blockSize:Long, blockData:ByteBuffer, - results : LinkedBlockingQueue[FetchResult]){ - results.put(new FetchResult( - blockId, blockSize, () => dataDeserialize(blockId, blockData, serializer) )) - } - - def startCopiers (numCopiers: Int): List [ _ <: Thread]= { + private def startCopiers(numCopiers: Int): List[_ <: Thread] = { (for ( i <- Range(0,numCopiers) ) yield { - val copier = new Thread { - override def run(){ - try { - while(!isInterrupted && !fetchRequestsSync.isEmpty) { + val copier = new Thread { + override def run(){ + try { + while(!isInterrupted && !fetchRequestsSync.isEmpty) { sendRequest(fetchRequestsSync.take()) - } - } catch { - case x: InterruptedException => logInfo("Copier Interrupted") - //case _ => throw new SparkException("Exception Throw in Shuffle Copier") } - } + } catch { + case x: InterruptedException => logInfo("Copier Interrupted") + //case _ => throw new SparkException("Exception Throw in Shuffle Copier") + } } - copier.start - copier + } + copier.start + copier }).toList } //keep this to interrupt the threads when necessary - def stopCopiers(copiers : List[_ <: Thread]) { + private def stopCopiers() { for (copier <- copiers) { copier.interrupt() } } - override def sendRequest(req: FetchRequest) { + override protected def sendRequest(req: FetchRequest) { + + def putResult(blockId: String, blockSize: Long, blockData: ByteBuf) { + val fetchResult = new FetchResult(blockId, blockSize, + () => dataDeserialize(blockId, blockData.nioBuffer, serializer)) + results.put(fetchResult) + } + logDebug("Sending request for %d blocks (%s) from %s".format( req.blocks.size, Utils.memoryBytesToString(req.size), req.address.host)) - val cmId = new ConnectionManagerId(req.address.host, System.getProperty("spark.shuffle.sender.port", "6653").toInt) + val cmId = new ConnectionManagerId( + req.address.host, System.getProperty("spark.shuffle.sender.port", "6653").toInt) val cpier = new ShuffleCopier - cpier.getBlocks(cmId,req.blocks,(blockId:String,blockSize:Long,blockData:ByteBuf) => putResult(blockId,blockSize,blockData.nioBuffer,results)) + cpier.getBlocks(cmId, req.blocks, putResult) logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.host ) } - override def splitLocalRemoteBlocks() : ArrayBuffer[FetchRequest] = { + override protected def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { // Split local and remote blocks. Remote blocks are further split into FetchRequests of size // at most maxBytesInFlight in order to limit the amount of data in flight. - val originalTotalBlocks = totalBlocks; + val originalTotalBlocks = _totalBlocks; val remoteRequests = new ArrayBuffer[FetchRequest] for ((address, blockInfos) <- blocksByAddress) { if (address == blockManagerId) { @@ -293,11 +302,11 @@ object BlockFetcherIterator { if (size > 0) { curBlocks += ((blockId, size)) curRequestSize += size - } else if (size == 0){ + } else if (size == 0) { //here we changes the totalBlocks - totalBlocks -= 1 + _totalBlocks -= 1 } else { - throw new SparkException("Negative block size "+blockId) + throw new BlockException(blockId, "Negative block size " + size) } if (curRequestSize >= minRequestSize) { // Add this FetchRequest @@ -312,13 +321,14 @@ object BlockFetcherIterator { } } } - logInfo("Getting " + totalBlocks + " non 0-byte blocks out of " + originalTotalBlocks + " blocks") + logInfo("Getting " + _totalBlocks + " non-zero-bytes blocks out of " + + originalTotalBlocks + " blocks") remoteRequests } - var copiers : List[_ <: Thread] = null + private var copiers: List[_ <: Thread] = null - override def initialize(){ + override def initialize() { // Split Local Remote Blocks and adjust totalBlocks to include only the non 0-byte blocks val remoteRequests = splitLocalRemoteBlocks() // Add the remote requests into our queue in a random order @@ -327,7 +337,8 @@ object BlockFetcherIterator { } copiers = startCopiers(System.getProperty("spark.shuffle.copier.threads", "6").toInt) - logInfo("Started " + fetchRequestsSync.size + " remote gets in " + Utils.getUsedTimeMs(startTime)) + logInfo("Started " + fetchRequestsSync.size + " remote gets in " + + Utils.getUsedTimeMs(startTime)) // Get Local Blocks startTime = System.currentTimeMillis @@ -338,24 +349,12 @@ object BlockFetcherIterator { override def next(): (String, Option[Iterator[Any]]) = { resultsGotten += 1 val result = results.take() - // if all the results has been retrieved - // shutdown the copiers - if (resultsGotten == totalBlocks) { - if( copiers != null ) - stopCopiers(copiers) + // if all the results has been retrieved, shutdown the copiers + if (resultsGotten == _totalBlocks && copiers != null) { + stopCopiers() } (result.blockId, if (result.failed) None else Some(result.deserialize())) } } - - def apply(t: String, - blockManager: BlockManager, - blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], - serializer: Serializer): BlockFetcherIterator = { - val iter = if (t == "netty") { new NettyBlockFetcherIterator(blockManager,blocksByAddress, serializer) } - else { new BasicBlockFetcherIterator(blockManager,blocksByAddress, serializer) } - iter.initialize() - iter - } + // End of NettyBlockFetcherIterator } - diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index a189c1a025..e0dec3a8bb 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -23,8 +23,7 @@ import spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStam import sun.nio.ch.DirectBuffer -private[spark] -class BlockManager( +private[spark] class BlockManager( executorId: String, actorSystem: ActorSystem, val master: BlockManagerMaster, @@ -494,11 +493,16 @@ class BlockManager( def getMultiple( blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], serializer: Serializer) : BlockFetcherIterator = { - if(System.getProperty("spark.shuffle.use.netty", "false").toBoolean){ - return BlockFetcherIterator("netty",this, blocksByAddress, serializer) - } else { - return BlockFetcherIterator("", this, blocksByAddress, serializer) - } + + val iter = + if (System.getProperty("spark.shuffle.use.netty", "false").toBoolean) { + new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer) + } else { + new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer) + } + + iter.initialize() + iter } def put(blockId: String, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean) @@ -942,8 +946,8 @@ class BlockManager( } } -private[spark] -object BlockManager extends Logging { + +private[spark] object BlockManager extends Logging { val ID_GENERATOR = new IdGenerator -- cgit v1.2.3 From 9e64396ca4c24804d5fd4e96212eed54530ca409 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 7 May 2013 18:30:54 -0700 Subject: Cleaned up the Java files from Shane's PR. --- .../main/java/spark/network/netty/FileClient.java | 45 ++++++------- .../netty/FileClientChannelInitializer.java | 11 +--- .../spark/network/netty/FileClientHandler.java | 11 ++-- .../main/java/spark/network/netty/FileServer.java | 73 ++++++++++------------ .../netty/FileServerChannelInitializer.java | 22 +++---- .../spark/network/netty/FileServerHandler.java | 33 +++++----- .../java/spark/network/netty/PathResolver.java | 4 +- core/src/main/scala/spark/storage/DiskStore.scala | 4 +- 8 files changed, 85 insertions(+), 118 deletions(-) diff --git a/core/src/main/java/spark/network/netty/FileClient.java b/core/src/main/java/spark/network/netty/FileClient.java index d0c5081dd2..3a62dacbc8 100644 --- a/core/src/main/java/spark/network/netty/FileClient.java +++ b/core/src/main/java/spark/network/netty/FileClient.java @@ -1,42 +1,40 @@ package spark.network.netty; import io.netty.bootstrap.Bootstrap; -import io.netty.channel.AbstractChannel; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelOption; -import io.netty.channel.nio.NioEventLoopGroup; -import io.netty.channel.socket.nio.NioSocketChannel; import io.netty.channel.oio.OioEventLoopGroup; import io.netty.channel.socket.oio.OioSocketChannel; -import java.util.Arrays; -public class FileClient { +class FileClient { private FileClientHandler handler = null; private Channel channel = null; private Bootstrap bootstrap = null; - public FileClient(FileClientHandler handler){ + public FileClient(FileClientHandler handler) { this.handler = handler; } - - public void init(){ - bootstrap = new Bootstrap(); - bootstrap.group(new OioEventLoopGroup()) + + public void init() { + bootstrap = new Bootstrap(); + bootstrap.group(new OioEventLoopGroup()) .channel(OioSocketChannel.class) .option(ChannelOption.SO_KEEPALIVE, true) .option(ChannelOption.TCP_NODELAY, true) .handler(new FileClientChannelInitializer(handler)); - } + } public static final class ChannelCloseListener implements ChannelFutureListener { private FileClient fc = null; + public ChannelCloseListener(FileClient fc){ this.fc = fc; } + @Override public void operationComplete(ChannelFuture future) { if (fc.bootstrap!=null){ @@ -46,44 +44,39 @@ public class FileClient { } } - public void connect(String host, int port){ + public void connect(String host, int port) { try { - // Start the connection attempt. channel = bootstrap.connect(host, port).sync().channel(); // ChannelFuture cf = channel.closeFuture(); //cf.addListener(new ChannelCloseListener(this)); } catch (InterruptedException e) { close(); - } + } } - - public void waitForClose(){ + + public void waitForClose() { try { channel.closeFuture().sync(); } catch (InterruptedException e){ e.printStackTrace(); } - } + } - public void sendRequest(String file){ + public void sendRequest(String file) { //assert(file == null); //assert(channel == null); - channel.write(file+"\r\n"); + channel.write(file + "\r\n"); } - public void close(){ + public void close() { if(channel != null) { - channel.close(); - channel = null; + channel.close(); + channel = null; } if ( bootstrap!=null) { bootstrap.shutdown(); bootstrap = null; } } - - } - - diff --git a/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java b/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java index 50e5704619..af25baf641 100644 --- a/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java +++ b/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java @@ -3,15 +3,10 @@ package spark.network.netty; import io.netty.buffer.BufType; import io.netty.channel.ChannelInitializer; import io.netty.channel.socket.SocketChannel; -import io.netty.handler.codec.LengthFieldBasedFrameDecoder; import io.netty.handler.codec.string.StringEncoder; -import io.netty.util.CharsetUtil; -import io.netty.handler.logging.LoggingHandler; -import io.netty.handler.logging.LogLevel; -public class FileClientChannelInitializer extends - ChannelInitializer { +class FileClientChannelInitializer extends ChannelInitializer { private FileClientHandler fhandler; @@ -23,7 +18,7 @@ public class FileClientChannelInitializer extends public void initChannel(SocketChannel channel) { // file no more than 2G channel.pipeline() - .addLast("encoder", new StringEncoder(BufType.BYTE)) - .addLast("handler", fhandler); + .addLast("encoder", new StringEncoder(BufType.BYTE)) + .addLast("handler", fhandler); } } diff --git a/core/src/main/java/spark/network/netty/FileClientHandler.java b/core/src/main/java/spark/network/netty/FileClientHandler.java index 911c8b32b5..2069dee5ca 100644 --- a/core/src/main/java/spark/network/netty/FileClientHandler.java +++ b/core/src/main/java/spark/network/netty/FileClientHandler.java @@ -3,12 +3,9 @@ package spark.network.netty; import io.netty.buffer.ByteBuf; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundByteHandlerAdapter; -import io.netty.util.CharsetUtil; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.logging.Logger; -public abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter { +abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter { private FileHeader currentHeader = null; @@ -19,7 +16,7 @@ public abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter // Use direct buffer if possible. return ctx.alloc().ioBuffer(); } - + @Override public void inboundBufferUpdated(ChannelHandlerContext ctx, ByteBuf in) { // get header @@ -27,8 +24,8 @@ public abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter currentHeader = FileHeader.create(in.readBytes(FileHeader.HEADER_SIZE())); } // get file - if(in.readableBytes() >= currentHeader.fileLen()){ - handle(ctx,in,currentHeader); + if(in.readableBytes() >= currentHeader.fileLen()) { + handle(ctx, in, currentHeader); currentHeader = null; ctx.close(); } diff --git a/core/src/main/java/spark/network/netty/FileServer.java b/core/src/main/java/spark/network/netty/FileServer.java index 38af305096..647b26bf8a 100644 --- a/core/src/main/java/spark/network/netty/FileServer.java +++ b/core/src/main/java/spark/network/netty/FileServer.java @@ -1,58 +1,51 @@ package spark.network.netty; -import java.io.File; import io.netty.bootstrap.ServerBootstrap; -import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelOption; import io.netty.channel.Channel; -import io.netty.channel.nio.NioEventLoopGroup; -import io.netty.channel.socket.nio.NioServerSocketChannel; import io.netty.channel.oio.OioEventLoopGroup; import io.netty.channel.socket.oio.OioServerSocketChannel; -import io.netty.handler.logging.LogLevel; -import io.netty.handler.logging.LoggingHandler; + /** * Server that accept the path of a file an echo back its content. */ -public class FileServer { +class FileServer { + + private ServerBootstrap bootstrap = null; + private Channel channel = null; + private PathResolver pResolver; - private ServerBootstrap bootstrap = null; - private Channel channel = null; - private PathResolver pResolver; + public FileServer(PathResolver pResolver) { + this.pResolver = pResolver; + } - public FileServer(PathResolver pResolver){ - this.pResolver = pResolver; + public void run(int port) { + // Configure the server. + bootstrap = new ServerBootstrap(); + try { + bootstrap.group(new OioEventLoopGroup(), new OioEventLoopGroup()) + .channel(OioServerSocketChannel.class) + .option(ChannelOption.SO_BACKLOG, 100) + .option(ChannelOption.SO_RCVBUF, 1500) + .childHandler(new FileServerChannelInitializer(pResolver)); + // Start the server. + channel = bootstrap.bind(port).sync().channel(); + channel.closeFuture().sync(); + } catch (InterruptedException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } finally{ + bootstrap.shutdown(); } + } - public void run(int port) { - // Configure the server. - bootstrap = new ServerBootstrap(); - try { - bootstrap.group(new OioEventLoopGroup(), new OioEventLoopGroup()) - .channel(OioServerSocketChannel.class) - .option(ChannelOption.SO_BACKLOG, 100) - .option(ChannelOption.SO_RCVBUF, 1500) - .childHandler(new FileServerChannelInitializer(pResolver)); - // Start the server. - channel = bootstrap.bind(port).sync().channel(); - channel.closeFuture().sync(); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } finally{ - bootstrap.shutdown(); - } + public void stop() { + if (channel!=null) { + channel.close(); } - - public void stop(){ - if (channel!=null){ - channel.close(); - } - if (bootstrap != null){ - bootstrap.shutdown(); - } + if (bootstrap != null) { + bootstrap.shutdown(); } + } } - - diff --git a/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java b/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java index 9d0618ff1c..8f1f5c65cd 100644 --- a/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java +++ b/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java @@ -1,21 +1,15 @@ package spark.network.netty; -import java.io.File; -import io.netty.buffer.BufType; import io.netty.channel.ChannelInitializer; import io.netty.channel.socket.SocketChannel; -import io.netty.handler.codec.string.StringDecoder; -import io.netty.handler.codec.string.StringEncoder; import io.netty.handler.codec.DelimiterBasedFrameDecoder; import io.netty.handler.codec.Delimiters; -import io.netty.util.CharsetUtil; -import io.netty.handler.logging.LoggingHandler; -import io.netty.handler.logging.LogLevel; +import io.netty.handler.codec.string.StringDecoder; + -public class FileServerChannelInitializer extends - ChannelInitializer { +class FileServerChannelInitializer extends ChannelInitializer { - PathResolver pResolver; + PathResolver pResolver; public FileServerChannelInitializer(PathResolver pResolver) { this.pResolver = pResolver; @@ -24,10 +18,8 @@ public class FileServerChannelInitializer extends @Override public void initChannel(SocketChannel channel) { channel.pipeline() - .addLast("framer", new DelimiterBasedFrameDecoder( - 8192, Delimiters.lineDelimiter())) - .addLast("strDecoder", new StringDecoder()) - .addLast("handler", new FileServerHandler(pResolver)); - + .addLast("framer", new DelimiterBasedFrameDecoder(8192, Delimiters.lineDelimiter())) + .addLast("strDecoder", new StringDecoder()) + .addLast("handler", new FileServerHandler(pResolver)); } } diff --git a/core/src/main/java/spark/network/netty/FileServerHandler.java b/core/src/main/java/spark/network/netty/FileServerHandler.java index e1083e87a2..a78eddb1b5 100644 --- a/core/src/main/java/spark/network/netty/FileServerHandler.java +++ b/core/src/main/java/spark/network/netty/FileServerHandler.java @@ -1,17 +1,17 @@ package spark.network.netty; +import java.io.File; +import java.io.FileInputStream; + import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundMessageHandlerAdapter; import io.netty.channel.DefaultFileRegion; -import io.netty.handler.stream.ChunkedFile; -import java.io.File; -import java.io.FileInputStream; -public class FileServerHandler extends - ChannelInboundMessageHandlerAdapter { - PathResolver pResolver; - +class FileServerHandler extends ChannelInboundMessageHandlerAdapter { + + PathResolver pResolver; + public FileServerHandler(PathResolver pResolver){ this.pResolver = pResolver; } @@ -21,8 +21,8 @@ public class FileServerHandler extends String path = pResolver.getAbsolutePath(blockId); // if getFilePath returns null, close the channel if (path == null) { - //ctx.close(); - return; + //ctx.close(); + return; } File file = new File(path); if (file.exists()) { @@ -33,23 +33,21 @@ public class FileServerHandler extends return; } long length = file.length(); - if (length > Integer.MAX_VALUE || length <= 0 ) { + if (length > Integer.MAX_VALUE || length <= 0) { //logger.info("too large file : " + file.getAbsolutePath() + " of size "+ length); ctx.write(new FileHeader(0, blockId).buffer()); ctx.flush(); - return; + return; } int len = new Long(length).intValue(); //logger.info("Sending block "+blockId+" filelen = "+len); //logger.info("header = "+ (new FileHeader(len, blockId)).buffer()); ctx.write((new FileHeader(len, blockId)).buffer()); try { - ctx.sendFile(new DefaultFileRegion(new FileInputStream(file) - .getChannel(), 0, file.length())); + ctx.sendFile(new DefaultFileRegion(new FileInputStream(file) + .getChannel(), 0, file.length())); } catch (Exception e) { - // TODO Auto-generated catch block - //logger.warning("Exception when sending file : " - //+ file.getAbsolutePath()); + //logger.warning("Exception when sending file : " + file.getAbsolutePath()); e.printStackTrace(); } } else { @@ -58,8 +56,7 @@ public class FileServerHandler extends } ctx.flush(); } - - + @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { cause.printStackTrace(); diff --git a/core/src/main/java/spark/network/netty/PathResolver.java b/core/src/main/java/spark/network/netty/PathResolver.java index 5d5eda006e..302411672c 100755 --- a/core/src/main/java/spark/network/netty/PathResolver.java +++ b/core/src/main/java/spark/network/netty/PathResolver.java @@ -1,12 +1,12 @@ package spark.network.netty; + public interface PathResolver { /** * Get the absolute path of the file - * + * * @param fileId * @return the absolute path of file */ public String getAbsolutePath(String fileId); - } diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 82bcbd5bc2..be33d4260e 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -288,7 +288,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) val port = System.getProperty("spark.shuffle.sender.port", "6653").toInt val pResolver = new PathResolver { - def getAbsolutePath(blockId:String):String = { + override def getAbsolutePath(blockId: String): String = { if (!blockId.startsWith("shuffle_")) { return null } @@ -298,7 +298,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) shuffleSender = new Thread { override def run() = { val sender = new ShuffleSender(port,pResolver) - logInfo("created ShuffleSender binding to port : "+ port) + logInfo("Created ShuffleSender binding to port : "+ port) sender.start } } -- cgit v1.2.3 From 547dcbe494ce7a888f636cf2596243be37b567b1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 7 May 2013 18:39:33 -0700 Subject: Cleaned up Scala files in network/netty from Shane's PR. --- .../scala/spark/network/netty/ShuffleCopier.scala | 50 ++++++++++------------ .../scala/spark/network/netty/ShuffleSender.scala | 18 +++++--- 2 files changed, 35 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala index d8d35bfeec..a91f5a886d 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala @@ -1,23 +1,21 @@ package spark.network.netty +import java.util.concurrent.Executors + import io.netty.buffer.ByteBuf import io.netty.channel.ChannelHandlerContext -import io.netty.channel.ChannelInboundByteHandlerAdapter import io.netty.util.CharsetUtil -import java.util.concurrent.atomic.AtomicInteger -import java.util.logging.Logger import spark.Logging import spark.network.ConnectionManagerId -import java.util.concurrent.Executors + private[spark] class ShuffleCopier extends Logging { - def getBlock(cmId: ConnectionManagerId, - blockId: String, - resultCollectCallback: (String, Long, ByteBuf) => Unit) = { + def getBlock(cmId: ConnectionManagerId, blockId: String, + resultCollectCallback: (String, Long, ByteBuf) => Unit) { - val handler = new ShuffleClientHandler(resultCollectCallback) + val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) val fc = new FileClient(handler) fc.init() fc.connect(cmId.host, cmId.port) @@ -28,29 +26,28 @@ private[spark] class ShuffleCopier extends Logging { def getBlocks(cmId: ConnectionManagerId, blocks: Seq[(String, Long)], - resultCollectCallback: (String, Long, ByteBuf) => Unit) = { + resultCollectCallback: (String, Long, ByteBuf) => Unit) { - blocks.map { - case(blockId,size) => { - getBlock(cmId,blockId,resultCollectCallback) - } + for ((blockId, size) <- blocks) { + getBlock(cmId, blockId, resultCollectCallback) } } } -private[spark] class ShuffleClientHandler(val resultCollectCallBack: (String, Long, ByteBuf) => Unit ) extends FileClientHandler with Logging { - - def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) { - logDebug("Received Block: " + header.blockId + " (" + header.fileLen + "B)"); - resultCollectCallBack(header.blockId, header.fileLen.toLong, in.readBytes(header.fileLen)) - } -} private[spark] object ShuffleCopier extends Logging { - def echoResultCollectCallBack(blockId: String, size: Long, content: ByteBuf) = { - logInfo("File: " + blockId + " content is : \" " - + content.toString(CharsetUtil.UTF_8) + "\"") + private class ShuffleClientHandler(resultCollectCallBack: (String, Long, ByteBuf) => Unit) + extends FileClientHandler with Logging { + + override def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) { + logDebug("Received Block: " + header.blockId + " (" + header.fileLen + "B)"); + resultCollectCallBack(header.blockId, header.fileLen.toLong, in.readBytes(header.fileLen)) + } + } + + def echoResultCollectCallBack(blockId: String, size: Long, content: ByteBuf) { + logInfo("File: " + blockId + " content is : \" " + content.toString(CharsetUtil.UTF_8) + "\"") } def runGetBlock(host:String, port:Int, file:String){ @@ -71,18 +68,17 @@ private[spark] object ShuffleCopier extends Logging { val host = args(0) val port = args(1).toInt val file = args(2) - val threads = if (args.length>3) args(3).toInt else 10 + val threads = if (args.length > 3) args(3).toInt else 10 val copiers = Executors.newFixedThreadPool(80) - for (i <- Range(0,threads)){ + for (i <- Range(0, threads)) { val runnable = new Runnable() { def run() { - runGetBlock(host,port,file) + runGetBlock(host, port, file) } } copiers.execute(runnable) } copiers.shutdown } - } diff --git a/core/src/main/scala/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/spark/network/netty/ShuffleSender.scala index c1986812e9..dc87fefc56 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleSender.scala +++ b/core/src/main/scala/spark/network/netty/ShuffleSender.scala @@ -1,12 +1,13 @@ package spark.network.netty -import spark.Logging import java.io.File +import spark.Logging + -private[spark] class ShuffleSender(val port: Int, val pResolver:PathResolver) extends Logging { +private[spark] class ShuffleSender(val port: Int, val pResolver: PathResolver) extends Logging { val server = new FileServer(pResolver) - + Runtime.getRuntime().addShutdownHook( new Thread() { override def run() { @@ -20,17 +21,22 @@ private[spark] class ShuffleSender(val port: Int, val pResolver:PathResolver) ex } } + private[spark] object ShuffleSender { + def main(args: Array[String]) { if (args.length < 3) { - System.err.println("Usage: ShuffleSender ") + System.err.println( + "Usage: ShuffleSender ") System.exit(1) } + val port = args(0).toInt val subDirsPerLocalDir = args(1).toInt - val localDirs = args.drop(2) map {new File(_)} + val localDirs = args.drop(2).map(new File(_)) + val pResovler = new PathResolver { - def getAbsolutePath(blockId:String):String = { + override def getAbsolutePath(blockId: String): String = { if (!blockId.startsWith("shuffle_")) { throw new Exception("Block " + blockId + " is not a shuffle block") } -- cgit v1.2.3 From 8388e8dd7ab5e55ea67b329d9359ba2147d796b0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 7 May 2013 18:40:35 -0700 Subject: Minor style fix in DiskStore... --- core/src/main/scala/spark/storage/DiskStore.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index be33d4260e..933eeaa216 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -283,7 +283,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) }) } - private def startShuffleBlockSender (){ + private def startShuffleBlockSender() { try { val port = System.getProperty("spark.shuffle.sender.port", "6653").toInt @@ -297,7 +297,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) } shuffleSender = new Thread { override def run() = { - val sender = new ShuffleSender(port,pResolver) + val sender = new ShuffleSender(port, pResolver) logInfo("Created ShuffleSender binding to port : "+ port) sender.start } -- cgit v1.2.3 From 5d70ee4663b5589611aef107f914a94f301c7d2a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 7 May 2013 22:42:15 -0700 Subject: Cleaned up connection manager (moved many classes to their own files). --- .../main/scala/spark/network/BufferMessage.scala | 94 +++++++++++ core/src/main/scala/spark/network/Connection.scala | 137 +++++++++------- .../scala/spark/network/ConnectionManager.scala | 53 +++--- .../scala/spark/network/ConnectionManagerId.scala | 21 +++ core/src/main/scala/spark/network/Message.scala | 179 ++------------------- .../main/scala/spark/network/MessageChunk.scala | 25 +++ .../scala/spark/network/MessageChunkHeader.scala | 58 +++++++ 7 files changed, 315 insertions(+), 252 deletions(-) create mode 100644 core/src/main/scala/spark/network/BufferMessage.scala create mode 100644 core/src/main/scala/spark/network/ConnectionManagerId.scala create mode 100644 core/src/main/scala/spark/network/MessageChunk.scala create mode 100644 core/src/main/scala/spark/network/MessageChunkHeader.scala diff --git a/core/src/main/scala/spark/network/BufferMessage.scala b/core/src/main/scala/spark/network/BufferMessage.scala new file mode 100644 index 0000000000..7b0e489a6c --- /dev/null +++ b/core/src/main/scala/spark/network/BufferMessage.scala @@ -0,0 +1,94 @@ +package spark.network + +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer + +import spark.storage.BlockManager + + +private[spark] +class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: Int) + extends Message(Message.BUFFER_MESSAGE, id_) { + + val initialSize = currentSize() + var gotChunkForSendingOnce = false + + def size = initialSize + + def currentSize() = { + if (buffers == null || buffers.isEmpty) { + 0 + } else { + buffers.map(_.remaining).reduceLeft(_ + _) + } + } + + def getChunkForSending(maxChunkSize: Int): Option[MessageChunk] = { + if (maxChunkSize <= 0) { + throw new Exception("Max chunk size is " + maxChunkSize) + } + + if (size == 0 && gotChunkForSendingOnce == false) { + val newChunk = new MessageChunk( + new MessageChunkHeader(typ, id, 0, 0, ackId, senderAddress), null) + gotChunkForSendingOnce = true + return Some(newChunk) + } + + while(!buffers.isEmpty) { + val buffer = buffers(0) + if (buffer.remaining == 0) { + BlockManager.dispose(buffer) + buffers -= buffer + } else { + val newBuffer = if (buffer.remaining <= maxChunkSize) { + buffer.duplicate() + } else { + buffer.slice().limit(maxChunkSize).asInstanceOf[ByteBuffer] + } + buffer.position(buffer.position + newBuffer.remaining) + val newChunk = new MessageChunk(new MessageChunkHeader( + typ, id, size, newBuffer.remaining, ackId, senderAddress), newBuffer) + gotChunkForSendingOnce = true + return Some(newChunk) + } + } + None + } + + def getChunkForReceiving(chunkSize: Int): Option[MessageChunk] = { + // STRONG ASSUMPTION: BufferMessage created when receiving data has ONLY ONE data buffer + if (buffers.size > 1) { + throw new Exception("Attempting to get chunk from message with multiple data buffers") + } + val buffer = buffers(0) + if (buffer.remaining > 0) { + if (buffer.remaining < chunkSize) { + throw new Exception("Not enough space in data buffer for receiving chunk") + } + val newBuffer = buffer.slice().limit(chunkSize).asInstanceOf[ByteBuffer] + buffer.position(buffer.position + newBuffer.remaining) + val newChunk = new MessageChunk(new MessageChunkHeader( + typ, id, size, newBuffer.remaining, ackId, senderAddress), newBuffer) + return Some(newChunk) + } + None + } + + def flip() { + buffers.foreach(_.flip) + } + + def hasAckId() = (ackId != 0) + + def isCompletelyReceived() = !buffers(0).hasRemaining + + override def toString = { + if (hasAckId) { + "BufferAckMessage(aid = " + ackId + ", id = " + id + ", size = " + size + ")" + } else { + "BufferMessage(id = " + id + ", size = " + size + ")" + } + } +} \ No newline at end of file diff --git a/core/src/main/scala/spark/network/Connection.scala b/core/src/main/scala/spark/network/Connection.scala index 00a0433a44..6e28f677a3 100644 --- a/core/src/main/scala/spark/network/Connection.scala +++ b/core/src/main/scala/spark/network/Connection.scala @@ -13,12 +13,13 @@ import java.net._ private[spark] abstract class Connection(val channel: SocketChannel, val selector: Selector, - val socketRemoteConnectionManagerId: ConnectionManagerId) extends Logging { + val socketRemoteConnectionManagerId: ConnectionManagerId) + extends Logging { + def this(channel_ : SocketChannel, selector_ : Selector) = { this(channel_, selector_, - ConnectionManagerId.fromSocketAddress( - channel_.socket.getRemoteSocketAddress().asInstanceOf[InetSocketAddress] - )) + ConnectionManagerId.fromSocketAddress( + channel_.socket.getRemoteSocketAddress().asInstanceOf[InetSocketAddress])) } channel.configureBlocking(false) @@ -32,17 +33,19 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, var onKeyInterestChangeCallback: (Connection, Int) => Unit = null val remoteAddress = getRemoteAddress() - + // Read channels typically do not register for write and write does not for read // Now, we do have write registering for read too (temporarily), but this is to detect // channel close NOT to actually read/consume data on it ! // How does this work if/when we move to SSL ? - + // What is the interest to register with selector for when we want this connection to be selected def registerInterest() - // What is the interest to register with selector for when we want this connection to be de-selected - // Traditionally, 0 - but in our case, for example, for close-detection on SendingConnection hack, it will be - // SelectionKey.OP_READ (until we fix it properly) + + // What is the interest to register with selector for when we want this connection to + // be de-selected + // Traditionally, 0 - but in our case, for example, for close-detection on SendingConnection hack, + // it will be SelectionKey.OP_READ (until we fix it properly) def unregisterInterest() // On receiving a read event, should we change the interest for this channel or not ? @@ -64,12 +67,14 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, // Returns whether we have to register for further reads or not. def read(): Boolean = { - throw new UnsupportedOperationException("Cannot read on connection of type " + this.getClass.toString) + throw new UnsupportedOperationException( + "Cannot read on connection of type " + this.getClass.toString) } // Returns whether we have to register for further writes or not. def write(): Boolean = { - throw new UnsupportedOperationException("Cannot write on connection of type " + this.getClass.toString) + throw new UnsupportedOperationException( + "Cannot write on connection of type " + this.getClass.toString) } def close() { @@ -81,11 +86,17 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, callOnCloseCallback() } - def onClose(callback: Connection => Unit) {onCloseCallback = callback} + def onClose(callback: Connection => Unit) { + onCloseCallback = callback + } - def onException(callback: (Connection, Exception) => Unit) {onExceptionCallback = callback} + def onException(callback: (Connection, Exception) => Unit) { + onExceptionCallback = callback + } - def onKeyInterestChange(callback: (Connection, Int) => Unit) {onKeyInterestChangeCallback = callback} + def onKeyInterestChange(callback: (Connection, Int) => Unit) { + onKeyInterestChangeCallback = callback + } def callOnExceptionCallback(e: Exception) { if (onExceptionCallback != null) { @@ -95,7 +106,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, " and OnExceptionCallback not registered", e) } } - + def callOnCloseCallback() { if (onCloseCallback != null) { onCloseCallback(this) @@ -132,24 +143,25 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, print(" (" + position + ", " + length + ")") buffer.position(curPosition) } - } -private[spark] class SendingConnection(val address: InetSocketAddress, selector_ : Selector, - remoteId_ : ConnectionManagerId) -extends Connection(SocketChannel.open, selector_, remoteId_) { +private[spark] +class SendingConnection(val address: InetSocketAddress, selector_ : Selector, + remoteId_ : ConnectionManagerId) + extends Connection(SocketChannel.open, selector_, remoteId_) { class Outbox(fair: Int = 0) { val messages = new Queue[Message]() - val defaultChunkSize = 65536 //32768 //16384 + val defaultChunkSize = 65536 //32768 //16384 var nextMessageToBeUsed = 0 def addMessage(message: Message) { - messages.synchronized{ + messages.synchronized{ /*messages += message*/ messages.enqueue(message) - logDebug("Added [" + message + "] to outbox for sending to [" + getRemoteConnectionManagerId() + "]") + logDebug("Added [" + message + "] to outbox for sending to " + + "[" + getRemoteConnectionManagerId() + "]") } } @@ -174,7 +186,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { message.started = true message.startTime = System.currentTimeMillis } - return chunk + return chunk } else { /*logInfo("Finished sending [" + message + "] to [" + getRemoteConnectionManagerId() + "]")*/ message.finishTime = System.currentTimeMillis @@ -185,7 +197,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { } None } - + private def getChunkRR(): Option[MessageChunk] = { messages.synchronized { while (!messages.isEmpty) { @@ -197,12 +209,14 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { messages.enqueue(message) nextMessageToBeUsed = nextMessageToBeUsed + 1 if (!message.started) { - logDebug("Starting to send [" + message + "] to [" + getRemoteConnectionManagerId() + "]") + logDebug( + "Starting to send [" + message + "] to [" + getRemoteConnectionManagerId() + "]") message.started = true message.startTime = System.currentTimeMillis } - logTrace("Sending chunk from [" + message+ "] to [" + getRemoteConnectionManagerId() + "]") - return chunk + logTrace( + "Sending chunk from [" + message+ "] to [" + getRemoteConnectionManagerId() + "]") + return chunk } else { message.finishTime = System.currentTimeMillis logDebug("Finished sending [" + message + "] to [" + getRemoteConnectionManagerId() + @@ -213,7 +227,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { None } } - + private val outbox = new Outbox(1) val currentBuffers = new ArrayBuffer[ByteBuffer]() @@ -228,11 +242,11 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { // it does - so let us keep it for now. changeConnectionKeyInterest(SelectionKey.OP_WRITE | DEFAULT_INTEREST) } - + override def unregisterInterest() { changeConnectionKeyInterest(DEFAULT_INTEREST) } - + def send(message: Message) { outbox.synchronized { outbox.addMessage(message) @@ -262,12 +276,14 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { // selection - though need not necessarily always complete successfully. val connected = channel.finishConnect if (!force && !connected) { - logInfo("finish connect failed [" + address + "], " + outbox.messages.size + " messages pending") + logInfo( + "finish connect failed [" + address + "], " + outbox.messages.size + " messages pending") return false } // Fallback to previous behavior - assume finishConnect completed - // This will happen only when finishConnect failed for some repeated number of times (10 or so) + // This will happen only when finishConnect failed for some repeated number of times + // (10 or so) // Is highly unlikely unless there was an unclean close of socket, etc registerInterest() logInfo("Connected to [" + address + "], " + outbox.messages.size + " messages pending") @@ -283,13 +299,13 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { } override def write(): Boolean = { - try{ - while(true) { + try { + while (true) { if (currentBuffers.size == 0) { outbox.synchronized { outbox.getChunk() match { case Some(chunk) => { - currentBuffers ++= chunk.buffers + currentBuffers ++= chunk.buffers } case None => { // changeConnectionKeyInterest(0) @@ -299,7 +315,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { } } } - + if (currentBuffers.size > 0) { val buffer = currentBuffers(0) val remainingBytes = buffer.remaining @@ -314,7 +330,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { } } } catch { - case e: Exception => { + case e: Exception => { logWarning("Error writing in connection to " + getRemoteConnectionManagerId(), e) callOnExceptionCallback(e) close() @@ -336,7 +352,8 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { if (length == -1) { // EOF close() } else if (length > 0) { - logWarning("Unexpected data read from SendingConnection to " + getRemoteConnectionManagerId()) + logWarning( + "Unexpected data read from SendingConnection to " + getRemoteConnectionManagerId()) } } catch { case e: Exception => @@ -355,30 +372,32 @@ extends Connection(SocketChannel.open, selector_, remoteId_) { // Must be created within selector loop - else deadlock -private[spark] class ReceivingConnection(channel_ : SocketChannel, selector_ : Selector) -extends Connection(channel_, selector_) { - +private[spark] class ReceivingConnection(channel_ : SocketChannel, selector_ : Selector) + extends Connection(channel_, selector_) { + class Inbox() { val messages = new HashMap[Int, BufferMessage]() - + def getChunk(header: MessageChunkHeader): Option[MessageChunk] = { - + def createNewMessage: BufferMessage = { val newMessage = Message.create(header).asInstanceOf[BufferMessage] newMessage.started = true newMessage.startTime = System.currentTimeMillis - logDebug("Starting to receive [" + newMessage + "] from [" + getRemoteConnectionManagerId() + "]") + logDebug( + "Starting to receive [" + newMessage + "] from [" + getRemoteConnectionManagerId() + "]") messages += ((newMessage.id, newMessage)) newMessage } - + val message = messages.getOrElseUpdate(header.id, createNewMessage) - logTrace("Receiving chunk of [" + message + "] from [" + getRemoteConnectionManagerId() + "]") + logTrace( + "Receiving chunk of [" + message + "] from [" + getRemoteConnectionManagerId() + "]") message.getChunkForReceiving(header.chunkSize) } - + def getMessageForChunk(chunk: MessageChunk): Option[BufferMessage] = { - messages.get(chunk.header.id) + messages.get(chunk.header.id) } def removeMessage(message: Message) { @@ -387,12 +406,14 @@ extends Connection(channel_, selector_) { } @volatile private var inferredRemoteManagerId: ConnectionManagerId = null + override def getRemoteConnectionManagerId(): ConnectionManagerId = { val currId = inferredRemoteManagerId if (currId != null) currId else super.getRemoteConnectionManagerId() } - // The reciever's remote address is the local socket on remote side : which is NOT the connection manager id of the receiver. + // The reciever's remote address is the local socket on remote side : which is NOT + // the connection manager id of the receiver. // We infer that from the messages we receive on the receiver socket. private def processConnectionManagerId(header: MessageChunkHeader) { val currId = inferredRemoteManagerId @@ -428,7 +449,8 @@ extends Connection(channel_, selector_) { } headerBuffer.flip if (headerBuffer.remaining != MessageChunkHeader.HEADER_SIZE) { - throw new Exception("Unexpected number of bytes (" + headerBuffer.remaining + ") in the header") + throw new Exception( + "Unexpected number of bytes (" + headerBuffer.remaining + ") in the header") } val header = MessageChunkHeader.create(headerBuffer) headerBuffer.clear() @@ -451,9 +473,9 @@ extends Connection(channel_, selector_) { case _ => throw new Exception("Message of unknown type received") } } - + if (currentChunk == null) throw new Exception("No message chunk to receive data") - + val bytesRead = channel.read(currentChunk.buffer) if (bytesRead == 0) { // re-register for read event ... @@ -464,14 +486,15 @@ extends Connection(channel_, selector_) { } /*logDebug("Read " + bytesRead + " bytes for the buffer")*/ - + if (currentChunk.buffer.remaining == 0) { /*println("Filled buffer at " + System.currentTimeMillis)*/ val bufferMessage = inbox.getMessageForChunk(currentChunk).get if (bufferMessage.isCompletelyReceived) { bufferMessage.flip bufferMessage.finishTime = System.currentTimeMillis - logDebug("Finished receiving [" + bufferMessage + "] from [" + getRemoteConnectionManagerId() + "] in " + bufferMessage.timeTaken) + logDebug("Finished receiving [" + bufferMessage + "] from " + + "[" + getRemoteConnectionManagerId() + "] in " + bufferMessage.timeTaken) if (onReceiveCallback != null) { onReceiveCallback(this, bufferMessage) } @@ -481,7 +504,7 @@ extends Connection(channel_, selector_) { } } } catch { - case e: Exception => { + case e: Exception => { logWarning("Error reading from connection to " + getRemoteConnectionManagerId(), e) callOnExceptionCallback(e) close() @@ -491,7 +514,7 @@ extends Connection(channel_, selector_) { // should not happen - to keep scala compiler happy return true } - + def onReceive(callback: (Connection, Message) => Unit) {onReceiveCallback = callback} override def changeInterestForRead(): Boolean = true @@ -505,7 +528,7 @@ extends Connection(channel_, selector_) { // it does - so let us keep it for now. changeConnectionKeyInterest(SelectionKey.OP_READ) } - + override def unregisterInterest() { changeConnectionKeyInterest(0) } diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 0eb03630d0..624a094856 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -18,20 +18,7 @@ import akka.dispatch.{Await, Promise, ExecutionContext, Future} import akka.util.Duration import akka.util.duration._ -private[spark] case class ConnectionManagerId(host: String, port: Int) { - // DEBUG code - Utils.checkHost(host) - assert (port > 0) - def toSocketAddress() = new InetSocketAddress(host, port) -} - -private[spark] object ConnectionManagerId { - def fromSocketAddress(socketAddress: InetSocketAddress): ConnectionManagerId = { - new ConnectionManagerId(socketAddress.getHostName(), socketAddress.getPort()) - } -} - private[spark] class ConnectionManager(port: Int) extends Logging { class MessageStatus( @@ -45,7 +32,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging { def markDone() { completionHandler(this) } } - + private val selector = SelectorProvider.provider.openSelector() private val handleMessageExecutor = new ThreadPoolExecutor( @@ -80,7 +67,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging { serverChannel.configureBlocking(false) serverChannel.socket.setReuseAddress(true) - serverChannel.socket.setReceiveBufferSize(256 * 1024) + serverChannel.socket.setReceiveBufferSize(256 * 1024) serverChannel.socket.bind(new InetSocketAddress(port)) serverChannel.register(selector, SelectionKey.OP_ACCEPT) @@ -351,7 +338,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging { case e: Exception => logError("Error in select loop", e) } } - + def acceptConnection(key: SelectionKey) { val serverChannel = key.channel.asInstanceOf[ServerSocketChannel] @@ -463,7 +450,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging { def receiveMessage(connection: Connection, message: Message) { val connectionManagerId = ConnectionManagerId.fromSocketAddress(message.senderAddress) - logDebug("Received [" + message + "] from [" + connectionManagerId + "]") + logDebug("Received [" + message + "] from [" + connectionManagerId + "]") val runnable = new Runnable() { val creationTime = System.currentTimeMillis def run() { @@ -483,11 +470,11 @@ private[spark] class ConnectionManager(port: Int) extends Logging { if (bufferMessage.hasAckId) { val sentMessageStatus = messageStatuses.synchronized { messageStatuses.get(bufferMessage.ackId) match { - case Some(status) => { - messageStatuses -= bufferMessage.ackId + case Some(status) => { + messageStatuses -= bufferMessage.ackId status } - case None => { + case None => { throw new Exception("Could not find reference for received ack message " + message.id) null } @@ -507,7 +494,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging { logDebug("Not calling back as callback is null") None } - + if (ackMessage.isDefined) { if (!ackMessage.get.isInstanceOf[BufferMessage]) { logDebug("Response to " + bufferMessage + " is not a buffer message, it is of type " + ackMessage.get.getClass()) @@ -517,7 +504,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging { } } - sendMessage(connectionManagerId, ackMessage.getOrElse { + sendMessage(connectionManagerId, ackMessage.getOrElse { Message.createBufferMessage(bufferMessage.id) }) } @@ -588,17 +575,17 @@ private[spark] object ConnectionManager { def main(args: Array[String]) { val manager = new ConnectionManager(9999) - manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { println("Received [" + msg + "] from [" + id + "]") None }) - + /*testSequentialSending(manager)*/ /*System.gc()*/ /*testParallelSending(manager)*/ /*System.gc()*/ - + /*testParallelDecreasingSending(manager)*/ /*System.gc()*/ @@ -610,9 +597,9 @@ private[spark] object ConnectionManager { println("--------------------------") println("Sequential Sending") println("--------------------------") - val size = 10 * 1024 * 1024 + val size = 10 * 1024 * 1024 val count = 10 - + val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) buffer.flip @@ -628,7 +615,7 @@ private[spark] object ConnectionManager { println("--------------------------") println("Parallel Sending") println("--------------------------") - val size = 10 * 1024 * 1024 + val size = 10 * 1024 * 1024 val count = 10 val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) @@ -643,12 +630,12 @@ private[spark] object ConnectionManager { if (!g.isDefined) println("Failed") }) val finishTime = System.currentTimeMillis - + val mb = size * count / 1024.0 / 1024.0 val ms = finishTime - startTime val tput = mb * 1000.0 / ms println("--------------------------") - println("Started at " + startTime + ", finished at " + finishTime) + println("Started at " + startTime + ", finished at " + finishTime) println("Sent " + count + " messages of size " + size + " in " + ms + " ms (" + tput + " MB/s)") println("--------------------------") println() @@ -658,7 +645,7 @@ private[spark] object ConnectionManager { println("--------------------------") println("Parallel Decreasing Sending") println("--------------------------") - val size = 10 * 1024 * 1024 + val size = 10 * 1024 * 1024 val count = 10 val buffers = Array.tabulate(count)(i => ByteBuffer.allocate(size * (i + 1)).put(Array.tabulate[Byte](size * (i + 1))(x => x.toByte))) buffers.foreach(_.flip) @@ -673,7 +660,7 @@ private[spark] object ConnectionManager { if (!g.isDefined) println("Failed") }) val finishTime = System.currentTimeMillis - + val ms = finishTime - startTime val tput = mb * 1000.0 / ms println("--------------------------") @@ -687,7 +674,7 @@ private[spark] object ConnectionManager { println("--------------------------") println("Continuous Sending") println("--------------------------") - val size = 10 * 1024 * 1024 + val size = 10 * 1024 * 1024 val count = 10 val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) diff --git a/core/src/main/scala/spark/network/ConnectionManagerId.scala b/core/src/main/scala/spark/network/ConnectionManagerId.scala new file mode 100644 index 0000000000..b554e84251 --- /dev/null +++ b/core/src/main/scala/spark/network/ConnectionManagerId.scala @@ -0,0 +1,21 @@ +package spark.network + +import java.net.InetSocketAddress + +import spark.Utils + + +private[spark] case class ConnectionManagerId(host: String, port: Int) { + // DEBUG code + Utils.checkHost(host) + assert (port > 0) + + def toSocketAddress() = new InetSocketAddress(host, port) +} + + +private[spark] object ConnectionManagerId { + def fromSocketAddress(socketAddress: InetSocketAddress): ConnectionManagerId = { + new ConnectionManagerId(socketAddress.getHostName(), socketAddress.getPort()) + } +} diff --git a/core/src/main/scala/spark/network/Message.scala b/core/src/main/scala/spark/network/Message.scala index 34fac9e776..d4f03610eb 100644 --- a/core/src/main/scala/spark/network/Message.scala +++ b/core/src/main/scala/spark/network/Message.scala @@ -1,56 +1,10 @@ package spark.network -import spark._ - -import scala.collection.mutable.ArrayBuffer - import java.nio.ByteBuffer -import java.net.InetAddress import java.net.InetSocketAddress -import storage.BlockManager - -private[spark] class MessageChunkHeader( - val typ: Long, - val id: Int, - val totalSize: Int, - val chunkSize: Int, - val other: Int, - val address: InetSocketAddress) { - lazy val buffer = { - // No need to change this, at 'use' time, we do a reverse lookup of the hostname. Refer to network.Connection - val ip = address.getAddress.getAddress() - val port = address.getPort() - ByteBuffer. - allocate(MessageChunkHeader.HEADER_SIZE). - putLong(typ). - putInt(id). - putInt(totalSize). - putInt(chunkSize). - putInt(other). - putInt(ip.size). - put(ip). - putInt(port). - position(MessageChunkHeader.HEADER_SIZE). - flip.asInstanceOf[ByteBuffer] - } - - override def toString = "" + this.getClass.getSimpleName + ":" + id + " of type " + typ + - " and sizes " + totalSize + " / " + chunkSize + " bytes" -} -private[spark] class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) { - val size = if (buffer == null) 0 else buffer.remaining - lazy val buffers = { - val ab = new ArrayBuffer[ByteBuffer]() - ab += header.buffer - if (buffer != null) { - ab += buffer - } - ab - } +import scala.collection.mutable.ArrayBuffer - override def toString = "" + this.getClass.getSimpleName + " (id = " + header.id + ", size = " + size + ")" -} private[spark] abstract class Message(val typ: Long, val id: Int) { var senderAddress: InetSocketAddress = null @@ -59,120 +13,16 @@ private[spark] abstract class Message(val typ: Long, val id: Int) { var finishTime = -1L def size: Int - + def getChunkForSending(maxChunkSize: Int): Option[MessageChunk] - + def getChunkForReceiving(chunkSize: Int): Option[MessageChunk] - + def timeTaken(): String = (finishTime - startTime).toString + " ms" override def toString = this.getClass.getSimpleName + "(id = " + id + ", size = " + size + ")" } -private[spark] class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: Int) -extends Message(Message.BUFFER_MESSAGE, id_) { - - val initialSize = currentSize() - var gotChunkForSendingOnce = false - - def size = initialSize - - def currentSize() = { - if (buffers == null || buffers.isEmpty) { - 0 - } else { - buffers.map(_.remaining).reduceLeft(_ + _) - } - } - - def getChunkForSending(maxChunkSize: Int): Option[MessageChunk] = { - if (maxChunkSize <= 0) { - throw new Exception("Max chunk size is " + maxChunkSize) - } - - if (size == 0 && gotChunkForSendingOnce == false) { - val newChunk = new MessageChunk(new MessageChunkHeader(typ, id, 0, 0, ackId, senderAddress), null) - gotChunkForSendingOnce = true - return Some(newChunk) - } - - while(!buffers.isEmpty) { - val buffer = buffers(0) - if (buffer.remaining == 0) { - BlockManager.dispose(buffer) - buffers -= buffer - } else { - val newBuffer = if (buffer.remaining <= maxChunkSize) { - buffer.duplicate() - } else { - buffer.slice().limit(maxChunkSize).asInstanceOf[ByteBuffer] - } - buffer.position(buffer.position + newBuffer.remaining) - val newChunk = new MessageChunk(new MessageChunkHeader( - typ, id, size, newBuffer.remaining, ackId, senderAddress), newBuffer) - gotChunkForSendingOnce = true - return Some(newChunk) - } - } - None - } - - def getChunkForReceiving(chunkSize: Int): Option[MessageChunk] = { - // STRONG ASSUMPTION: BufferMessage created when receiving data has ONLY ONE data buffer - if (buffers.size > 1) { - throw new Exception("Attempting to get chunk from message with multiple data buffers") - } - val buffer = buffers(0) - if (buffer.remaining > 0) { - if (buffer.remaining < chunkSize) { - throw new Exception("Not enough space in data buffer for receiving chunk") - } - val newBuffer = buffer.slice().limit(chunkSize).asInstanceOf[ByteBuffer] - buffer.position(buffer.position + newBuffer.remaining) - val newChunk = new MessageChunk(new MessageChunkHeader( - typ, id, size, newBuffer.remaining, ackId, senderAddress), newBuffer) - return Some(newChunk) - } - None - } - - def flip() { - buffers.foreach(_.flip) - } - - def hasAckId() = (ackId != 0) - - def isCompletelyReceived() = !buffers(0).hasRemaining - - override def toString = { - if (hasAckId) { - "BufferAckMessage(aid = " + ackId + ", id = " + id + ", size = " + size + ")" - } else { - "BufferMessage(id = " + id + ", size = " + size + ")" - } - } -} - -private[spark] object MessageChunkHeader { - val HEADER_SIZE = 40 - - def create(buffer: ByteBuffer): MessageChunkHeader = { - if (buffer.remaining != HEADER_SIZE) { - throw new IllegalArgumentException("Cannot convert buffer data to Message") - } - val typ = buffer.getLong() - val id = buffer.getInt() - val totalSize = buffer.getInt() - val chunkSize = buffer.getInt() - val other = buffer.getInt() - val ipSize = buffer.getInt() - val ipBytes = new Array[Byte](ipSize) - buffer.get(ipBytes) - val ip = InetAddress.getByAddress(ipBytes) - val port = buffer.getInt() - new MessageChunkHeader(typ, id, totalSize, chunkSize, other, new InetSocketAddress(ip, port)) - } -} private[spark] object Message { val BUFFER_MESSAGE = 1111111111L @@ -181,14 +31,16 @@ private[spark] object Message { def getNewId() = synchronized { lastId += 1 - if (lastId == 0) lastId += 1 + if (lastId == 0) { + lastId += 1 + } lastId } def createBufferMessage(dataBuffers: Seq[ByteBuffer], ackId: Int): BufferMessage = { if (dataBuffers == null) { return new BufferMessage(getNewId(), new ArrayBuffer[ByteBuffer], ackId) - } + } if (dataBuffers.exists(_ == null)) { throw new Exception("Attempting to create buffer message with null buffer") } @@ -197,7 +49,7 @@ private[spark] object Message { def createBufferMessage(dataBuffers: Seq[ByteBuffer]): BufferMessage = createBufferMessage(dataBuffers, 0) - + def createBufferMessage(dataBuffer: ByteBuffer, ackId: Int): BufferMessage = { if (dataBuffer == null) { return createBufferMessage(Array(ByteBuffer.allocate(0)), ackId) @@ -205,15 +57,18 @@ private[spark] object Message { return createBufferMessage(Array(dataBuffer), ackId) } } - - def createBufferMessage(dataBuffer: ByteBuffer): BufferMessage = + + def createBufferMessage(dataBuffer: ByteBuffer): BufferMessage = createBufferMessage(dataBuffer, 0) - - def createBufferMessage(ackId: Int): BufferMessage = createBufferMessage(new Array[ByteBuffer](0), ackId) + + def createBufferMessage(ackId: Int): BufferMessage = { + createBufferMessage(new Array[ByteBuffer](0), ackId) + } def create(header: MessageChunkHeader): Message = { val newMessage: Message = header.typ match { - case BUFFER_MESSAGE => new BufferMessage(header.id, ArrayBuffer(ByteBuffer.allocate(header.totalSize)), header.other) + case BUFFER_MESSAGE => new BufferMessage(header.id, + ArrayBuffer(ByteBuffer.allocate(header.totalSize)), header.other) } newMessage.senderAddress = header.address newMessage diff --git a/core/src/main/scala/spark/network/MessageChunk.scala b/core/src/main/scala/spark/network/MessageChunk.scala new file mode 100644 index 0000000000..aaf9204d0e --- /dev/null +++ b/core/src/main/scala/spark/network/MessageChunk.scala @@ -0,0 +1,25 @@ +package spark.network + +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer + + +private[network] +class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) { + + val size = if (buffer == null) 0 else buffer.remaining + + lazy val buffers = { + val ab = new ArrayBuffer[ByteBuffer]() + ab += header.buffer + if (buffer != null) { + ab += buffer + } + ab + } + + override def toString = { + "" + this.getClass.getSimpleName + " (id = " + header.id + ", size = " + size + ")" + } +} diff --git a/core/src/main/scala/spark/network/MessageChunkHeader.scala b/core/src/main/scala/spark/network/MessageChunkHeader.scala new file mode 100644 index 0000000000..3693d509d6 --- /dev/null +++ b/core/src/main/scala/spark/network/MessageChunkHeader.scala @@ -0,0 +1,58 @@ +package spark.network + +import java.net.InetAddress +import java.net.InetSocketAddress +import java.nio.ByteBuffer + + +private[spark] class MessageChunkHeader( + val typ: Long, + val id: Int, + val totalSize: Int, + val chunkSize: Int, + val other: Int, + val address: InetSocketAddress) { + lazy val buffer = { + // No need to change this, at 'use' time, we do a reverse lookup of the hostname. + // Refer to network.Connection + val ip = address.getAddress.getAddress() + val port = address.getPort() + ByteBuffer. + allocate(MessageChunkHeader.HEADER_SIZE). + putLong(typ). + putInt(id). + putInt(totalSize). + putInt(chunkSize). + putInt(other). + putInt(ip.size). + put(ip). + putInt(port). + position(MessageChunkHeader.HEADER_SIZE). + flip.asInstanceOf[ByteBuffer] + } + + override def toString = "" + this.getClass.getSimpleName + ":" + id + " of type " + typ + + " and sizes " + totalSize + " / " + chunkSize + " bytes" +} + + +private[spark] object MessageChunkHeader { + val HEADER_SIZE = 40 + + def create(buffer: ByteBuffer): MessageChunkHeader = { + if (buffer.remaining != HEADER_SIZE) { + throw new IllegalArgumentException("Cannot convert buffer data to Message") + } + val typ = buffer.getLong() + val id = buffer.getInt() + val totalSize = buffer.getInt() + val chunkSize = buffer.getInt() + val other = buffer.getInt() + val ipSize = buffer.getInt() + val ipBytes = new Array[Byte](ipSize) + buffer.get(ipBytes) + val ip = InetAddress.getByAddress(ipBytes) + val port = buffer.getInt() + new MessageChunkHeader(typ, id, totalSize, chunkSize, other, new InetSocketAddress(ip, port)) + } +} -- cgit v1.2.3 From 9cafacf32ddb9a3f6c5cb774e4fe527225273f16 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 7 May 2013 22:42:37 -0700 Subject: Added test for Netty suite. --- core/src/test/scala/spark/DistributedSuite.scala | 3 ++- core/src/test/scala/spark/ShuffleNettySuite.scala | 17 +++++++++++++++++ 2 files changed, 19 insertions(+), 1 deletion(-) create mode 100644 core/src/test/scala/spark/ShuffleNettySuite.scala diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index 4df3bb5b67..488c70c414 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -18,7 +18,8 @@ import scala.collection.mutable.ArrayBuffer import SparkContext._ import storage.{GetBlock, BlockManagerWorker, StorageLevel} -class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter with LocalSparkContext { +class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter + with LocalSparkContext { val clusterUrl = "local-cluster[2,1,512]" diff --git a/core/src/test/scala/spark/ShuffleNettySuite.scala b/core/src/test/scala/spark/ShuffleNettySuite.scala new file mode 100644 index 0000000000..bfaffa953e --- /dev/null +++ b/core/src/test/scala/spark/ShuffleNettySuite.scala @@ -0,0 +1,17 @@ +package spark + +import org.scalatest.BeforeAndAfterAll + + +class ShuffleNettySuite extends ShuffleSuite with BeforeAndAfterAll { + + // This test suite should run all tests in ShuffleSuite with Netty shuffle mode. + + override def beforeAll(configMap: Map[String, Any]) { + System.setProperty("spark.shuffle.use.netty", "true") + } + + override def afterAll(configMap: Map[String, Any]) { + System.setProperty("spark.shuffle.use.netty", "false") + } +} -- cgit v1.2.3 From 0ab818d50812f312596170b5e42aa76d2ff59d15 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 9 May 2013 00:38:59 -0700 Subject: fix linebreak --- .../scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 955ee5d806..170ede0f44 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -31,8 +31,8 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command("spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome().getOrElse( throw new IllegalArgumentException("must supply spark home for spark standalone")) - val appDesc = - new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, sc.ui.appUIAddress) + val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, + sc.ui.appUIAddress) client = new Client(sc.env.actorSystem, master, appDesc, this) client.start() -- cgit v1.2.3 From b05c9d22d70333924b988b2dfa359ce3e11f7c9d Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Thu, 9 May 2013 18:49:12 +0530 Subject: Remove explicit hardcoding of yarn-standalone as args(0) if it is missing. --- .../scala/spark/deploy/yarn/ApplicationMaster.scala | 19 +++---------------- .../deploy/yarn/ApplicationMasterArguments.scala | 1 - .../scala/spark/deploy/yarn/ClientArguments.scala | 1 - 3 files changed, 3 insertions(+), 18 deletions(-) diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index ae719267e8..aa72c1e5fe 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -148,22 +148,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e .getMethod("main", classOf[Array[String]]) val t = new Thread { override def run() { - var mainArgs: Array[String] = null - var startIndex = 0 - - // I am sure there is a better 'scala' way to do this .... but I am just trying to get things to work right now ! - if (args.userArgs.isEmpty || args.userArgs.get(0) != "yarn-standalone") { - // ensure that first param is ALWAYS "yarn-standalone" - mainArgs = new Array[String](args.userArgs.size() + 1) - mainArgs.update(0, "yarn-standalone") - startIndex = 1 - } - else { - mainArgs = new Array[String](args.userArgs.size()) - } - - args.userArgs.copyToArray(mainArgs, startIndex, args.userArgs.size()) - + // Copy + var mainArgs: Array[String] = new Array[String](args.userArgs.size()) + args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size()) mainMethod.invoke(null, mainArgs) } } diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala index dc89125d81..1b00208511 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -69,7 +69,6 @@ class ApplicationMasterArguments(val args: Array[String]) { " --class CLASS_NAME Name of your application's main class (required)\n" + " --args ARGS Arguments to be passed to your application's main class.\n" + " Mutliple invocations are possible, each will be passed in order.\n" + - " Note that first argument will ALWAYS be yarn-standalone : will be added if missing.\n" + " --num-workers NUM Number of workers to start (Default: 2)\n" + " --worker-cores NUM Number of cores for the workers (Default: 1)\n" + " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n") diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala index 2e69fe3fb0..24110558e7 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala @@ -92,7 +92,6 @@ class ClientArguments(val args: Array[String]) { " --class CLASS_NAME Name of your application's main class (required)\n" + " --args ARGS Arguments to be passed to your application's main class.\n" + " Mutliple invocations are possible, each will be passed in order.\n" + - " Note that first argument will ALWAYS be yarn-standalone : will be added if missing.\n" + " --num-workers NUM Number of workers to start (Default: 2)\n" + " --worker-cores NUM Number of cores for the workers (Default: 1). This is unsused right now.\n" + " --master-memory MEM Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" + -- cgit v1.2.3 From 012c9e5ab072239e07202abe4775b434be6e32b9 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 9 May 2013 14:20:01 -0700 Subject: Revert "Merge pull request #596 from esjewett/master" because the dependency on hbase introduces netty-3.2.2 which conflicts with netty-3.5.3 already in Spark. This caused multiple test failures. This reverts commit 0f1b7a06e1f6782711170234f105f1b277e3b04c, reversing changes made to aacca1b8a85bd073ce185a06d6470b070761b2f4. --- .../src/main/scala/spark/examples/HBaseTest.scala | 35 ---------------------- project/SparkBuild.scala | 6 +--- 2 files changed, 1 insertion(+), 40 deletions(-) delete mode 100644 examples/src/main/scala/spark/examples/HBaseTest.scala diff --git a/examples/src/main/scala/spark/examples/HBaseTest.scala b/examples/src/main/scala/spark/examples/HBaseTest.scala deleted file mode 100644 index 9bad876860..0000000000 --- a/examples/src/main/scala/spark/examples/HBaseTest.scala +++ /dev/null @@ -1,35 +0,0 @@ -package spark.examples - -import spark._ -import spark.rdd.NewHadoopRDD -import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor} -import org.apache.hadoop.hbase.client.HBaseAdmin -import org.apache.hadoop.hbase.mapreduce.TableInputFormat - -object HBaseTest { - def main(args: Array[String]) { - val sc = new SparkContext(args(0), "HBaseTest", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) - - val conf = HBaseConfiguration.create() - - // Other options for configuring scan behavior are available. More information available at - // http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html - conf.set(TableInputFormat.INPUT_TABLE, args(1)) - - // Initialize hBase table if necessary - val admin = new HBaseAdmin(conf) - if(!admin.isTableAvailable(args(1))) { - val tableDesc = new HTableDescriptor(args(1)) - admin.createTable(tableDesc) - } - - val hBaseRDD = sc.newAPIHadoopRDD(conf, classOf[TableInputFormat], - classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable], - classOf[org.apache.hadoop.hbase.client.Result]) - - hBaseRDD.count() - - System.exit(0) - } -} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 6f5607d31c..190d723435 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -200,11 +200,7 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", - resolvers ++= Seq("Apache HBase" at "https://repository.apache.org/content/repositories/releases"), - libraryDependencies ++= Seq( - "com.twitter" % "algebird-core_2.9.2" % "0.1.11", - "org.apache.hbase" % "hbase" % "0.94.6" - ) + libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.11") ) def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") -- cgit v1.2.3 From ee6f6aa6cd028e6a3938dcd5334661c27f493bc6 Mon Sep 17 00:00:00 2001 From: Ethan Jewett Date: Thu, 9 May 2013 18:33:38 -0500 Subject: Add hBase example --- .../src/main/scala/spark/examples/HBaseTest.scala | 35 ++++++++++++++++++++++ project/SparkBuild.scala | 6 +++- 2 files changed, 40 insertions(+), 1 deletion(-) create mode 100644 examples/src/main/scala/spark/examples/HBaseTest.scala diff --git a/examples/src/main/scala/spark/examples/HBaseTest.scala b/examples/src/main/scala/spark/examples/HBaseTest.scala new file mode 100644 index 0000000000..6e910154d4 --- /dev/null +++ b/examples/src/main/scala/spark/examples/HBaseTest.scala @@ -0,0 +1,35 @@ +package spark.examples + +import spark._ +import spark.rdd.NewHadoopRDD +import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor} +import org.apache.hadoop.hbase.client.HBaseAdmin +import org.apache.hadoop.hbase.mapreduce.TableInputFormat + +object HBaseTest { + def main(args: Array[String]) { + val sc = new SparkContext(args(0), "HBaseTest", + System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + + val conf = HBaseConfiguration.create() + + // Other options for configuring scan behavior are available. More information available at + // http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html + conf.set(TableInputFormat.INPUT_TABLE, args(1)) + + // Initialize hBase table if necessary + val admin = new HBaseAdmin(conf) + if(!admin.isTableAvailable(args(1))) { + val tableDesc = new HTableDescriptor(args(1)) + admin.createTable(tableDesc) + } + + val hBaseRDD = sc.newAPIHadoopRDD(conf, classOf[TableInputFormat], + classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable], + classOf[org.apache.hadoop.hbase.client.Result]) + + hBaseRDD.count() + + System.exit(0) + } +} \ No newline at end of file diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 190d723435..57fe04ea2d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -200,7 +200,11 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", - libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.11") + resolvers ++= Seq("Apache HBase" at "https://repository.apache.org/content/repositories/releases"), + libraryDependencies ++= Seq( + "com.twitter" % "algebird-core_2.9.2" % "0.1.11", + "org.apache.hbase" % "hbase" % "0.94.6" excludeAll(excludeNetty) + ) ) def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") -- cgit v1.2.3 From d761e7359deb7ca864d33b8f2e4380b57448630b Mon Sep 17 00:00:00 2001 From: seanm Date: Fri, 10 May 2013 12:05:10 -0600 Subject: adding kafkaStream API tests --- streaming/src/test/java/spark/streaming/JavaAPISuite.java | 4 ++-- .../src/test/scala/spark/streaming/InputStreamsSuite.scala | 11 +++++++++++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 61e4c0a207..350d0888a3 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -4,6 +4,7 @@ import com.google.common.base.Optional; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.io.Files; +import kafka.serializer.StringDecoder; import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.junit.After; import org.junit.Assert; @@ -1203,8 +1204,7 @@ public class JavaAPISuite implements Serializable { public void testKafkaStream() { HashMap topics = Maps.newHashMap(); JavaDStream test1 = ssc.kafkaStream("localhost:12345", "group", topics); - JavaDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics); - JavaDStream test3 = ssc.kafkaStream("localhost:12345", "group", topics, + JavaDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK()); } diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala index 1024d3ac97..595c766a21 100644 --- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala @@ -240,6 +240,17 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { assert(output(i) === expectedOutput(i)) } } + + test("kafka input stream") { + val ssc = new StreamingContext(master, framework, batchDuration) + val topics = Map("my-topic" -> 1) + val test1 = ssc.kafkaStream("localhost:12345", "group", topics) + val test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK) + + // Test specifying decoder + val kafkaParams = Map("zk.connect"->"localhost:12345","groupid"->"consumer-group") + val test3 = ssc.kafkaStream[String, kafka.serializer.StringDecoder](kafkaParams, topics, StorageLevel.MEMORY_AND_DISK) + } } -- cgit v1.2.3 From b95c1bdbbaeea86152e24b394a03bbbad95989d5 Mon Sep 17 00:00:00 2001 From: seanm Date: Fri, 10 May 2013 12:47:24 -0600 Subject: count() now uses a transform instead of ConstantInputDStream --- streaming/src/main/scala/spark/streaming/DStream.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index e3a9247924..e125310861 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -441,10 +441,7 @@ abstract class DStream[T: ClassManifest] ( * Return a new DStream in which each RDD has a single element generated by counting each RDD * of this DStream. */ - def count(): DStream[Long] = { - val zero = new ConstantInputDStream(context, context.sparkContext.makeRDD(Seq((null, 0L)), 1)) - this.map(_ => (null, 1L)).union(zero).reduceByKey(_ + _).map(_._2) - } + def count(): DStream[Long] = this.map(_ => (null, 1L)).transform(_.union(context.sparkContext.makeRDD(Seq((null, 0L)), 1))).reduceByKey(_ + _).map(_._2) /** * Return a new DStream in which each RDD contains the counts of each distinct value in -- cgit v1.2.3 From 6e6b3e0d7eadab97d45e975452c7e0c18246686e Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 10 May 2013 13:02:34 -0700 Subject: Actually use the cleaned closure in foreachPartition --- core/src/main/scala/spark/RDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index fd14ef17f1..dde131696f 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -489,7 +489,7 @@ abstract class RDD[T: ClassManifest]( */ def foreachPartition(f: Iterator[T] => Unit) { val cleanF = sc.clean(f) - sc.runJob(this, (iter: Iterator[T]) => f(iter)) + sc.runJob(this, (iter: Iterator[T]) => cleanF(iter)) } /** -- cgit v1.2.3 From 3632980b1b61dbb9ab9a3ab3d92fb415cb7173b9 Mon Sep 17 00:00:00 2001 From: seanm Date: Fri, 10 May 2013 15:54:26 -0600 Subject: fixing indentation --- .../src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index 13427873ff..4ad2bdf8a8 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -105,7 +105,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param storageLevel RDD storage level. Defaults to memory-only */ def kafkaStream[T, D <: kafka.serializer.Decoder[_]: Manifest]( - kafkaParams: JMap[String, String], + kafkaParams: JMap[String, String], topics: JMap[String, JInt], storageLevel: StorageLevel) : JavaDStream[T] = { -- cgit v1.2.3 From f25282def5826fab6caabff28c82c57a7f3fdcb8 Mon Sep 17 00:00:00 2001 From: seanm Date: Fri, 10 May 2013 17:34:28 -0600 Subject: fixing kafkaStream Java API and adding test --- .../scala/spark/streaming/api/java/JavaStreamingContext.scala | 10 +++++++--- streaming/src/test/java/spark/streaming/JavaAPISuite.java | 6 ++++++ 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index 4ad2bdf8a8..b35d9032f1 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -99,18 +99,22 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Create an input stream that pulls messages form a Kafka Broker. + * @param typeClass Type of RDD + * @param decoderClass Type of kafka decoder * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. * @param storageLevel RDD storage level. Defaults to memory-only */ - def kafkaStream[T, D <: kafka.serializer.Decoder[_]: Manifest]( + def kafkaStream[T, D <: kafka.serializer.Decoder[_]]( + typeClass: Class[T], + decoderClass: Class[D], kafkaParams: JMap[String, String], topics: JMap[String, JInt], storageLevel: StorageLevel) : JavaDStream[T] = { - implicit val cmt: ClassManifest[T] = - implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cmt: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cmd: Manifest[D] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[D]] ssc.kafkaStream[T, D]( kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 350d0888a3..e5fdbe1b7a 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -1206,6 +1206,12 @@ public class JavaAPISuite implements Serializable { JavaDStream test1 = ssc.kafkaStream("localhost:12345", "group", topics); JavaDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK()); + + HashMap kafkaParams = Maps.newHashMap(); + kafkaParams.put("zk.connect","localhost:12345"); + kafkaParams.put("groupid","consumer-group"); + JavaDStream test3 = ssc.kafkaStream(String.class, StringDecoder.class, kafkaParams, topics, + StorageLevel.MEMORY_AND_DISK()); } @Test -- cgit v1.2.3 From ee37612bc95e8486fa328908005293585912db71 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Sat, 11 May 2013 11:12:22 +0530 Subject: 1) Add support for HADOOP_CONF_DIR (and/or YARN_CONF_DIR - use either) : which is used to specify the client side configuration directory : which needs to be part of the CLASSPATH. 2) Move from var+=".." to var="$var.." : the former does not work on older bash shells unfortunately. --- docs/running-on-yarn.md | 3 +++ run | 65 +++++++++++++++++++++++++++++-------------------- run2.cmd | 13 ++++++++++ 3 files changed, 54 insertions(+), 27 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 26424bbe52..c8cf8ffc35 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -30,6 +30,9 @@ If you want to test out the YARN deployment mode, you can use the current Spark # Launching Spark on YARN +Ensure that HADOOP_CONF_DIR or YARN_CONF_DIR points to the directory which contains the (client side) configuration files for the hadoop cluster. +This would be used to connect to the cluster, write to the dfs and submit jobs to the resource manager. + The command to launch the YARN Client is as follows: SPARK_JAR= ./run spark.deploy.yarn.Client \ diff --git a/run b/run index 0a58ac4a36..c744bbd3dc 100755 --- a/run +++ b/run @@ -22,7 +22,7 @@ fi # values for that; it doesn't need a lot if [ "$1" = "spark.deploy.master.Master" -o "$1" = "spark.deploy.worker.Worker" ]; then SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m} - SPARK_DAEMON_JAVA_OPTS+=" -Dspark.akka.logLifecycleEvents=true" + SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" SPARK_JAVA_OPTS=$SPARK_DAEMON_JAVA_OPTS # Empty by default fi @@ -30,19 +30,19 @@ fi # Add java opts for master, worker, executor. The opts maybe null case "$1" in 'spark.deploy.master.Master') - SPARK_JAVA_OPTS+=" $SPARK_MASTER_OPTS" + SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_MASTER_OPTS" ;; 'spark.deploy.worker.Worker') - SPARK_JAVA_OPTS+=" $SPARK_WORKER_OPTS" + SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_WORKER_OPTS" ;; 'spark.executor.StandaloneExecutorBackend') - SPARK_JAVA_OPTS+=" $SPARK_EXECUTOR_OPTS" + SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_EXECUTOR_OPTS" ;; 'spark.executor.MesosExecutorBackend') - SPARK_JAVA_OPTS+=" $SPARK_EXECUTOR_OPTS" + SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_EXECUTOR_OPTS" ;; 'spark.repl.Main') - SPARK_JAVA_OPTS+=" $SPARK_REPL_OPTS" + SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_REPL_OPTS" ;; esac @@ -85,11 +85,11 @@ export SPARK_MEM # Set JAVA_OPTS to be able to load native libraries and to set heap size JAVA_OPTS="$SPARK_JAVA_OPTS" -JAVA_OPTS+=" -Djava.library.path=$SPARK_LIBRARY_PATH" -JAVA_OPTS+=" -Xms$SPARK_MEM -Xmx$SPARK_MEM" +JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH" +JAVA_OPTS="$JAVA_OPTS -Xms$SPARK_MEM -Xmx$SPARK_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e $FWDIR/conf/java-opts ] ; then - JAVA_OPTS+=" `cat $FWDIR/conf/java-opts`" + JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`" fi export JAVA_OPTS @@ -110,30 +110,30 @@ fi # Build up classpath CLASSPATH="$SPARK_CLASSPATH" -CLASSPATH+=":$FWDIR/conf" -CLASSPATH+=":$CORE_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$FWDIR/conf" +CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/classes" if [ -n "$SPARK_TESTING" ] ; then - CLASSPATH+=":$CORE_DIR/target/scala-$SCALA_VERSION/test-classes" - CLASSPATH+=":$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes" fi -CLASSPATH+=":$CORE_DIR/src/main/resources" -CLASSPATH+=":$REPL_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH+=":$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH+=":$STREAMING_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH+=":$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar +CLASSPATH="$CLASSPATH:$CORE_DIR/src/main/resources" +CLASSPATH="$CLASSPATH:$REPL_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar if [ -e "$FWDIR/lib_managed" ]; then - CLASSPATH+=":$FWDIR/lib_managed/jars/*" - CLASSPATH+=":$FWDIR/lib_managed/bundles/*" + CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/jars/*" + CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*" fi -CLASSPATH+=":$REPL_DIR/lib/*" +CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*" if [ -e $REPL_BIN_DIR/target ]; then for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do - CLASSPATH+=":$jar" + CLASSPATH="$CLASSPATH:$jar" done fi -CLASSPATH+=":$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do - CLASSPATH+=":$jar" + CLASSPATH="$CLASSPATH:$jar" done # Figure out the JAR file that our examples were packaged into. This includes a bit of a hack @@ -147,6 +147,17 @@ if [ -e "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar ]; then export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar` fi +# Add hadoop conf dir - else FileSystem.*, etc fail ! +# Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts +# the configurtion files. +if [ "x" != "x$HADOOP_CONF_DIR" ]; then + CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR" +fi +if [ "x" != "x$YARN_CONF_DIR" ]; then + CLASSPATH="$CLASSPATH:$YARN_CONF_DIR" +fi + + # Figure out whether to run our class with java or with the scala launcher. # In most cases, we'd prefer to execute our process with java because scala # creates a shell script as the parent of its Java process, which makes it @@ -156,9 +167,9 @@ fi if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then EXTRA_ARGS="" # Java options will be passed to scala as JAVA_OPTS else - CLASSPATH+=":$SCALA_LIBRARY_PATH/scala-library.jar" - CLASSPATH+=":$SCALA_LIBRARY_PATH/scala-compiler.jar" - CLASSPATH+=":$SCALA_LIBRARY_PATH/jline.jar" + CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-library.jar" + CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-compiler.jar" + CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/jline.jar" # The JVM doesn't read JAVA_OPTS by default so we need to pass it in EXTRA_ARGS="$JAVA_OPTS" fi diff --git a/run2.cmd b/run2.cmd index d2d4807971..c6f43dde5b 100644 --- a/run2.cmd +++ b/run2.cmd @@ -63,6 +63,19 @@ set CLASSPATH=%CLASSPATH%;%FWDIR%repl\lib\* set CLASSPATH=%CLASSPATH%;%FWDIR%python\lib\* set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes +rem Add hadoop conf dir - else FileSystem.*, etc fail +rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts +rem the configurtion files. +if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir + set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% +:no_hadoop_conf_dir + +if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir + set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% +:no_yarn_conf_dir + + + rem Figure out the JAR file that our examples were packaged into. rem First search in the build path from SBT: for %%d in ("examples/target/scala-%SCALA_VERSION%/spark-examples*.jar") do ( -- cgit v1.2.3 From 0345954530a445b275595962c9f949cad55a01f6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 11 May 2013 14:17:09 -0700 Subject: SPARK-738: Spark should detect and squash nonserializable exceptions --- core/src/main/scala/spark/executor/Executor.scala | 16 ++++++++++++++-- core/src/test/scala/spark/DistributedSuite.scala | 21 +++++++++++++++++++++ 2 files changed, 35 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 344face5e6..f9061b1c71 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -1,6 +1,6 @@ package spark.executor -import java.io.{File, FileOutputStream} +import java.io.{NotSerializableException, File, FileOutputStream} import java.net.{URI, URL, URLClassLoader} import java.util.concurrent._ @@ -123,7 +123,19 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert case t: Throwable => { val reason = ExceptionFailure(t) - context.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) + val serReason = + try { + ser.serialize(reason) + } + catch { + case e: NotSerializableException => { + val message = "Spark caught unserializable exn: " + t.toString + val throwable = new Exception(message) + throwable.setStackTrace(t.getStackTrace) + ser.serialize(new ExceptionFailure(throwable)) + } + } + context.statusUpdate(taskId, TaskState.FAILED, serReason) // TODO: Should we exit the whole executor here? On the one hand, the failed task may // have left some weird state around depending on when the exception was thrown, but on diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index 4df3bb5b67..8ab0f2cfa2 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -18,6 +18,9 @@ import scala.collection.mutable.ArrayBuffer import SparkContext._ import storage.{GetBlock, BlockManagerWorker, StorageLevel} +class NotSerializableClass +class NotSerializableExn(val notSer: NotSerializableClass) extends Throwable() {} + class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter with LocalSparkContext { val clusterUrl = "local-cluster[2,1,512]" @@ -27,6 +30,24 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter System.clearProperty("spark.storage.memoryFraction") } + test("task throws not serializable exception") { + // Ensures that executors do not crash when an exn is not serializable. If executors crash, + // this test will hang. Correct behavior is that executors don't crash but fail tasks + // and the scheduler throws a SparkException. + + // numSlaves must be less than numPartitions + val numSlaves = 3 + val numPartitions = 10 + + sc = new SparkContext("local-cluster[%s,1,512]".format(numSlaves), "test") + val data = sc.parallelize(1 to 100, numPartitions).map(x => (x, x)). + map(x => throw new NotSerializableExn(new NotSerializableClass)) + intercept[SparkException] { + data.count() + } + resetSparkContext() + } + test("local-cluster format") { sc = new SparkContext("local-cluster[2,1,512]", "test") assert(sc.parallelize(1 to 2, 2).count() == 2) -- cgit v1.2.3 From a5c28bb888f74d27893c198865f588ca0334a8a6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 11 May 2013 14:20:39 -0700 Subject: Removing unnecessary map --- core/src/test/scala/spark/DistributedSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index 8ab0f2cfa2..33c99471c6 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -40,7 +40,7 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter val numPartitions = 10 sc = new SparkContext("local-cluster[%s,1,512]".format(numSlaves), "test") - val data = sc.parallelize(1 to 100, numPartitions).map(x => (x, x)). + val data = sc.parallelize(1 to 100, numPartitions). map(x => throw new NotSerializableExn(new NotSerializableClass)) intercept[SparkException] { data.count() -- cgit v1.2.3 From 440719109e10ea1cc6149a8f61d42ea7cc443352 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 11 May 2013 18:27:26 -0700 Subject: Throw exception if task result exceeds Akka frame size. This partially addresses SPARK-747. --- core/src/main/scala/spark/executor/Executor.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 344face5e6..718f0ff5bc 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -72,6 +72,7 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert // Initialize Spark environment (using system properties read above) val env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, false, false) SparkEnv.set(env) + private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size") // Start worker thread pool val threadPool = new ThreadPoolExecutor( @@ -113,6 +114,9 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert val result = new TaskResult(value, accumUpdates, task.metrics.getOrElse(null)) val serializedResult = ser.serialize(result) logInfo("Serialized size of result for " + taskId + " is " + serializedResult.limit) + if (serializedResult.limit >= (akkaFrameSize - 1024)) { + throw new SparkException("Result for " + taskId + " exceeded Akka frame size") + } context.statusUpdate(taskId, TaskState.FINISHED, serializedResult) logInfo("Finished task ID " + taskId) } catch { -- cgit v1.2.3 From 3da2305ed0d4add7127953e5240632f86053b4aa Mon Sep 17 00:00:00 2001 From: Cody Koeninger Date: Sat, 11 May 2013 23:59:07 -0500 Subject: code cleanup per rxin comments --- core/src/main/scala/spark/rdd/JdbcRDD.scala | 67 ++++++++++++++++------------- 1 file changed, 37 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/spark/rdd/JdbcRDD.scala b/core/src/main/scala/spark/rdd/JdbcRDD.scala index 4c3054465c..b0f7054233 100644 --- a/core/src/main/scala/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/spark/rdd/JdbcRDD.scala @@ -5,23 +5,27 @@ import java.sql.{Connection, ResultSet} import spark.{Logging, Partition, RDD, SparkContext, TaskContext} import spark.util.NextIterator +private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { + override def index = idx +} + /** - An RDD that executes an SQL query on a JDBC connection and reads results. - @param getConnection a function that returns an open Connection. - The RDD takes care of closing the connection. - @param sql the text of the query. - The query must contain two ? placeholders for parameters used to partition the results. - E.g. "select title, author from books where ? <= id and id <= ?" - @param lowerBound the minimum value of the first placeholder - @param upperBound the maximum value of the second placeholder - The lower and upper bounds are inclusive. - @param numPartitions the number of partitions. - Given a lowerBound of 1, an upperBound of 20, and a numPartitions of 2, - the query would be executed twice, once with (1, 10) and once with (11, 20) - @param mapRow a function from a ResultSet to a single row of the desired result type(s). - This should only call getInt, getString, etc; the RDD takes care of calling next. - The default maps a ResultSet to an array of Object. -*/ + * An RDD that executes an SQL query on a JDBC connection and reads results. + * @param getConnection a function that returns an open Connection. + * The RDD takes care of closing the connection. + * @param sql the text of the query. + * The query must contain two ? placeholders for parameters used to partition the results. + * E.g. "select title, author from books where ? <= id and id <= ?" + * @param lowerBound the minimum value of the first placeholder + * @param upperBound the maximum value of the second placeholder + * The lower and upper bounds are inclusive. + * @param numPartitions the number of partitions. + * Given a lowerBound of 1, an upperBound of 20, and a numPartitions of 2, + * the query would be executed twice, once with (1, 10) and once with (11, 20) + * @param mapRow a function from a ResultSet to a single row of the desired result type(s). + * This should only call getInt, getString, etc; the RDD takes care of calling next. + * The default maps a ResultSet to an array of Object. + */ class JdbcRDD[T: ClassManifest]( sc: SparkContext, getConnection: () => Connection, @@ -29,26 +33,33 @@ class JdbcRDD[T: ClassManifest]( lowerBound: Long, upperBound: Long, numPartitions: Int, - mapRow: (ResultSet) => T = JdbcRDD.resultSetToObjectArray) + mapRow: (ResultSet) => T = JdbcRDD.resultSetToObjectArray _) extends RDD[T](sc, Nil) with Logging { - override def getPartitions: Array[Partition] = - ParallelCollectionRDD.slice(lowerBound to upperBound, numPartitions). - filter(! _.isEmpty). - zipWithIndex. - map(x => new JdbcPartition(x._2, x._1.head, x._1.last)). - toArray + override def getPartitions: Array[Partition] = { + // bounds are inclusive, hence the + 1 here and - 1 on end + val length = 1 + upperBound - lowerBound + (0 until numPartitions).map(i => { + val start = lowerBound + ((i * length) / numPartitions).toLong + val end = lowerBound + (((i + 1) * length) / numPartitions).toLong - 1 + new JdbcPartition(i, start, end) + }).toArray + } override def compute(thePart: Partition, context: TaskContext) = new NextIterator[T] { context.addOnCompleteCallback{ () => closeIfNeeded() } val part = thePart.asInstanceOf[JdbcPartition] val conn = getConnection() val stmt = conn.prepareStatement(sql, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY) - // force mysql driver to stream rather than pull entire resultset into memory + + // setFetchSize(Integer.MIN_VALUE) is a mysql driver specific way to force streaming results, + // rather than pulling entire resultset into memory. + // see http://dev.mysql.com/doc/refman/5.0/en/connector-j-reference-implementation-notes.html if (conn.getMetaData.getURL.matches("jdbc:mysql:.*")) { stmt.setFetchSize(Integer.MIN_VALUE) logInfo("statement fetch size set to: " + stmt.getFetchSize + " to force MySQL streaming ") } + stmt.setLong(1, part.lower) stmt.setLong(2, part.upper) val rs = stmt.executeQuery() @@ -81,14 +92,10 @@ class JdbcRDD[T: ClassManifest]( } } } - -} - -private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { - override def index = idx } object JdbcRDD { - val resultSetToObjectArray = (rs: ResultSet) => + def resultSetToObjectArray(rs: ResultSet) = { Array.tabulate[Object](rs.getMetaData.getColumnCount)(i => rs.getObject(i + 1)) + } } -- cgit v1.2.3 From 059ab8875463ab22fe329fb6a627cac0a7d8158c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 11 May 2013 23:39:14 -0700 Subject: Changing technique to use same code path in all cases --- core/src/main/scala/spark/TaskEndReason.scala | 13 ++++++++++--- core/src/main/scala/spark/executor/Executor.scala | 16 ++-------------- .../scala/spark/scheduler/cluster/TaskSetManager.scala | 8 ++++---- .../scala/spark/scheduler/local/LocalScheduler.scala | 3 ++- 4 files changed, 18 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/spark/TaskEndReason.scala b/core/src/main/scala/spark/TaskEndReason.scala index 420c54bc9a..ce9bb49897 100644 --- a/core/src/main/scala/spark/TaskEndReason.scala +++ b/core/src/main/scala/spark/TaskEndReason.scala @@ -14,9 +14,16 @@ private[spark] case object Success extends TaskEndReason private[spark] case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it -private[spark] -case class FetchFailed(bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int) extends TaskEndReason +private[spark] case class FetchFailed( + bmAddress: BlockManagerId, + shuffleId: Int, + mapId: Int, + reduceId: Int) + extends TaskEndReason -private[spark] case class ExceptionFailure(exception: Throwable) extends TaskEndReason +private[spark] case class ExceptionFailure( + description: String, + stackTrace: Array[StackTraceElement]) + extends TaskEndReason private[spark] case class OtherFailure(message: String) extends TaskEndReason diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index f9061b1c71..9084def9b2 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -122,20 +122,8 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert } case t: Throwable => { - val reason = ExceptionFailure(t) - val serReason = - try { - ser.serialize(reason) - } - catch { - case e: NotSerializableException => { - val message = "Spark caught unserializable exn: " + t.toString - val throwable = new Exception(message) - throwable.setStackTrace(t.getStackTrace) - ser.serialize(new ExceptionFailure(throwable)) - } - } - context.statusUpdate(taskId, TaskState.FAILED, serReason) + val reason = ExceptionFailure(t.toString, t.getStackTrace) + context.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) // TODO: Should we exit the whole executor here? On the one hand, the failed task may // have left some weird state around depending on when the exception was thrown, but on diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 27e713e2c4..6d663de2f8 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -493,7 +493,7 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe return case ef: ExceptionFailure => - val key = ef.exception.toString + val key = ef.description val now = System.currentTimeMillis val (printFull, dupCount) = { if (recentExceptions.contains(key)) { @@ -511,10 +511,10 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe } } if (printFull) { - val locs = ef.exception.getStackTrace.map(loc => "\tat %s".format(loc.toString)) - logInfo("Loss was due to %s\n%s".format(ef.exception.toString, locs.mkString("\n"))) + val locs = ef.stackTrace.map(loc => "\tat %s".format(loc.toString)) + logInfo("Loss was due to %s\n%s".format(ef.description, locs.mkString("\n"))) } else { - logInfo("Loss was due to %s [duplicate %d]".format(ef.exception.toString, dupCount)) + logInfo("Loss was due to %s [duplicate %d]".format(ef.description, dupCount)) } case _ => {} diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index f060a940a9..42d5bc4813 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -102,7 +102,8 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon } else { // TODO: Do something nicer here to return all the way to the user if (!Thread.currentThread().isInterrupted) - listener.taskEnded(task, new ExceptionFailure(t), null, null, info, null) + listener.taskEnded( + task, new ExceptionFailure(t.getMessage, t.getStackTrace), null, null, info, null) } } } -- cgit v1.2.3 From 1c15b8505124c157449b6d41e1127f3eb4081a23 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 11 May 2013 23:52:53 -0700 Subject: Removing import --- core/src/main/scala/spark/executor/Executor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 9084def9b2..1d5516966d 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -1,6 +1,6 @@ package spark.executor -import java.io.{NotSerializableException, File, FileOutputStream} +import java.io.{File, FileOutputStream} import java.net.{URI, URL, URLClassLoader} import java.util.concurrent._ -- cgit v1.2.3 From 72b9c4cb6ec4080eb8751e5e040f180272ac82a6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 11 May 2013 23:53:50 -0700 Subject: Small fix --- core/src/main/scala/spark/scheduler/local/LocalScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 42d5bc4813..a357422466 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -103,7 +103,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon // TODO: Do something nicer here to return all the way to the user if (!Thread.currentThread().isInterrupted) listener.taskEnded( - task, new ExceptionFailure(t.getMessage, t.getStackTrace), null, null, info, null) + task, new ExceptionFailure(t.toString, t.getStackTrace), null, null, info, null) } } } -- cgit v1.2.3 From 7f0833647b784c4ec7cd2f2e8e4fcd5ed6f673cd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 12 May 2013 07:54:03 -0700 Subject: Capturing class name --- core/src/main/scala/spark/TaskEndReason.scala | 1 + core/src/main/scala/spark/executor/Executor.scala | 2 +- core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala | 3 ++- core/src/main/scala/spark/scheduler/local/LocalScheduler.scala | 7 ++++--- 4 files changed, 8 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/TaskEndReason.scala b/core/src/main/scala/spark/TaskEndReason.scala index ce9bb49897..ca793eb402 100644 --- a/core/src/main/scala/spark/TaskEndReason.scala +++ b/core/src/main/scala/spark/TaskEndReason.scala @@ -22,6 +22,7 @@ private[spark] case class FetchFailed( extends TaskEndReason private[spark] case class ExceptionFailure( + className: String, description: String, stackTrace: Array[StackTraceElement]) extends TaskEndReason diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 1d5516966d..da20b84544 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -122,7 +122,7 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert } case t: Throwable => { - val reason = ExceptionFailure(t.toString, t.getStackTrace) + val reason = ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace) context.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) // TODO: Should we exit the whole executor here? On the one hand, the failed task may diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 6d663de2f8..06de3c755e 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -512,7 +512,8 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe } if (printFull) { val locs = ef.stackTrace.map(loc => "\tat %s".format(loc.toString)) - logInfo("Loss was due to %s\n%s".format(ef.description, locs.mkString("\n"))) + logInfo("Loss was due to %s\n%s\n%s".format( + ef.className, ef.description, locs.mkString("\n"))) } else { logInfo("Loss was due to %s [duplicate %d]".format(ef.description, dupCount)) } diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index a357422466..ebe42685ad 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -101,9 +101,10 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon submitTask(task, idInJob) } else { // TODO: Do something nicer here to return all the way to the user - if (!Thread.currentThread().isInterrupted) - listener.taskEnded( - task, new ExceptionFailure(t.toString, t.getStackTrace), null, null, info, null) + if (!Thread.currentThread().isInterrupted) { + val failure = new ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace) + listener.taskEnded(task, failure, null, null, info, null) + } } } } -- cgit v1.2.3 From b16c4896f617f352bb230908b7c08c7c5b028434 Mon Sep 17 00:00:00 2001 From: Cody Koeninger Date: Tue, 14 May 2013 23:44:04 -0500 Subject: add test for JdbcRDD using embedded derby, per rxin suggestion --- .gitignore | 1 + core/src/test/scala/spark/rdd/JdbcRDDSuite.scala | 56 ++++++++++++++++++++++++ project/SparkBuild.scala | 1 + 3 files changed, 58 insertions(+) create mode 100644 core/src/test/scala/spark/rdd/JdbcRDDSuite.scala diff --git a/.gitignore b/.gitignore index 155e785b01..b87fc1ee79 100644 --- a/.gitignore +++ b/.gitignore @@ -36,3 +36,4 @@ streaming-tests.log dependency-reduced-pom.xml .ensime .ensime_lucene +derby.log diff --git a/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala new file mode 100644 index 0000000000..6afb0fa9bc --- /dev/null +++ b/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala @@ -0,0 +1,56 @@ +package spark + +import org.scalatest.{ BeforeAndAfter, FunSuite } +import spark.SparkContext._ +import spark.rdd.JdbcRDD +import java.sql._ + +class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { + + before { + Class.forName("org.apache.derby.jdbc.EmbeddedDriver") + val conn = DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb;create=true") + try { + val create = conn.createStatement + create.execute(""" + CREATE TABLE FOO( + ID INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1), + DATA INTEGER + )""") + create.close + val insert = conn.prepareStatement("INSERT INTO FOO(DATA) VALUES(?)") + (1 to 100).foreach { i => + insert.setInt(1, i * 2) + insert.executeUpdate + } + insert.close + } catch { + case e: SQLException if e.getSQLState == "X0Y32" => + // table exists + } finally { + conn.close + } + } + + test("basic functionality") { + sc = new SparkContext("local", "test") + val rdd = new JdbcRDD( + sc, + () => { DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb") }, + "SELECT DATA FROM FOO WHERE ? <= ID AND ID <= ?", + 1, 100, 3, + (r: ResultSet) => { r.getInt(1) } ).cache + + assert(rdd.count === 100) + assert(rdd.reduce(_+_) === 10100) + } + + after { + try { + DriverManager.getConnection("jdbc:derby:;shutdown=true") + } catch { + case se: SQLException if se.getSQLState == "XJ015" => + // normal shutdown + } + } +} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f0b371b2cf..b11893590e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -147,6 +147,7 @@ object SparkBuild extends Build { "cc.spray" % "spray-can" % "1.0-M2.1", "cc.spray" % "spray-server" % "1.0-M2.1", "cc.spray" % "spray-json_2.9.2" % "1.1.1", + "org.apache.derby" % "derby" % "10.4.2.0" % "test", "org.apache.mesos" % "mesos" % "0.9.0-incubating" ) ++ (if (HADOOP_MAJOR_VERSION == "2") Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION) else None).toSeq, unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / ("src/hadoop" + HADOOP_MAJOR_VERSION + "/scala") } -- cgit v1.2.3 From d7d1da79d30961e461115a73bbfc9e4c4448e533 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 13 May 2013 16:37:49 -0700 Subject: when akka starts, use akkas default classloader (current thread) --- core/src/main/scala/spark/util/AkkaUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index 9fb7e001ba..cd79bd2bda 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -52,7 +52,7 @@ private[spark] object AkkaUtils { """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize, lifecycleEvents, akkaWriteTimeout)) - val actorSystem = ActorSystem(name, akkaConf, getClass.getClassLoader) + val actorSystem = ActorSystem(name, akkaConf) // Figure out the port number we bound to, in case port was passed as 0. This is a bit of a // hack because Akka doesn't let you figure out the port through the public API yet. -- cgit v1.2.3 From 38d4b97c6d47df4e1f1a3279ff786509f60e0eaf Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 13 May 2013 16:50:43 -0700 Subject: use threads classloader when deserializing task results; classnotfoundexception includes classloader --- .../main/scala/spark/scheduler/cluster/TaskSetManager.scala | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index c69f3bdb7f..b348092d89 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -503,9 +503,16 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe logInfo("Finished TID %s in %d ms (progress: %d/%d)".format( tid, info.duration, tasksFinished, numTasks)) // Deserialize task result and pass it to the scheduler - val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) - result.metrics.resultSize = serializedData.limit() - sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) + try { + val result = ser.deserialize[TaskResult[_]](serializedData) + result.metrics.resultSize = serializedData.limit() + sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) + } catch { + case cnf: ClassNotFoundException => + val loader = Thread.currentThread().getContextClassLoader + throw new SparkException("ClassNotFound with classloader: " + loader, cnf) + case ex => throw ex + } // Mark finished and stop if we've finished all the tasks finished(index) = true if (tasksFinished == numTasks) { -- cgit v1.2.3 From 404f9ff617401a2f8d12845861ce8f02cfe6442c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 14 May 2013 23:28:34 -0700 Subject: Added derby dependency to Maven pom files for the JDBC Java test. --- core/pom.xml | 5 +++++ pom.xml | 8 +++++++- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/core/pom.xml b/core/pom.xml index 9a019b5a42..57a95328c3 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -92,6 +92,11 @@ log4j + + org.apache.derby + derby + test + org.scalatest scalatest_${scala.version} diff --git a/pom.xml b/pom.xml index 3936165d78..d7cdc591cf 100644 --- a/pom.xml +++ b/pom.xml @@ -256,6 +256,12 @@ mesos ${mesos.version} + + org.apache.derby + derby + 10.4.2.0 + test + org.scala-lang @@ -565,7 +571,7 @@ 2 - 2.0.2-alpha + 2.0.2-alpha -- cgit v1.2.3 From f9d40a5848a2e1eef31ac63cd9221d5b77c1c5a7 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 14 May 2013 23:29:57 -0700 Subject: Added a comment in JdbcRDD for example usage. --- core/src/main/scala/spark/rdd/JdbcRDD.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/rdd/JdbcRDD.scala b/core/src/main/scala/spark/rdd/JdbcRDD.scala index b0f7054233..a50f407737 100644 --- a/core/src/main/scala/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/spark/rdd/JdbcRDD.scala @@ -11,11 +11,13 @@ private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) e /** * An RDD that executes an SQL query on a JDBC connection and reads results. + * For usage example, see test case JdbcRDDSuite. + * * @param getConnection a function that returns an open Connection. * The RDD takes care of closing the connection. * @param sql the text of the query. * The query must contain two ? placeholders for parameters used to partition the results. - * E.g. "select title, author from books where ? <= id and id <= ?" + * E.g. "select title, author from books where ? <= id and id <= ?" * @param lowerBound the minimum value of the first placeholder * @param upperBound the maximum value of the second placeholder * The lower and upper bounds are inclusive. -- cgit v1.2.3 From afcad7b3aa8736231a526417ede47ce6d353a70c Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Wed, 15 May 2013 14:45:14 -0300 Subject: Docs: Mention spark shell's default for MASTER --- docs/scala-programming-guide.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 2315aadbdf..b0da130fcb 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -67,6 +67,8 @@ The master URL passed to Spark can be in one of the following formats: +If no master URL is specified, the spark shell defaults to "local". + For running on YARN, Spark launches an instance of the standalone deploy cluster within YARN; see [running on YARN](running-on-yarn.html) for details. ### Deploying Code on a Cluster -- cgit v1.2.3 From b8e46b6074e5ecc1ae4ed22ea32983597c14b683 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 16 May 2013 01:52:40 -0700 Subject: Abort job if result exceeds Akka frame size; add test. --- core/src/main/scala/spark/TaskEndReason.scala | 2 ++ core/src/main/scala/spark/executor/Executor.scala | 3 ++- .../main/scala/spark/scheduler/cluster/TaskSetManager.scala | 6 ++++++ core/src/test/scala/spark/DistributedSuite.scala | 13 +++++++++++++ 4 files changed, 23 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/TaskEndReason.scala b/core/src/main/scala/spark/TaskEndReason.scala index 420c54bc9a..c5da453562 100644 --- a/core/src/main/scala/spark/TaskEndReason.scala +++ b/core/src/main/scala/spark/TaskEndReason.scala @@ -20,3 +20,5 @@ case class FetchFailed(bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, re private[spark] case class ExceptionFailure(exception: Throwable) extends TaskEndReason private[spark] case class OtherFailure(message: String) extends TaskEndReason + +private[spark] case class TaskResultTooBigFailure() extends TaskEndReason \ No newline at end of file diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 718f0ff5bc..9ec4eb6e88 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -115,7 +115,8 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert val serializedResult = ser.serialize(result) logInfo("Serialized size of result for " + taskId + " is " + serializedResult.limit) if (serializedResult.limit >= (akkaFrameSize - 1024)) { - throw new SparkException("Result for " + taskId + " exceeded Akka frame size") + context.statusUpdate(taskId, TaskState.FAILED, ser.serialize(TaskResultTooBigFailure())) + return } context.statusUpdate(taskId, TaskState.FINISHED, serializedResult) logInfo("Finished task ID " + taskId) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 27e713e2c4..df7f0eafff 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -492,6 +492,12 @@ private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSe sched.taskSetFinished(this) return + case taskResultTooBig: TaskResultTooBigFailure => + logInfo("Loss was due to task %s result exceeding Akka frame size;" + + "aborting job".format(tid)) + abort("Task %s result exceeded Akka frame size".format(tid)) + return + case ef: ExceptionFailure => val key = ef.exception.toString val now = System.currentTimeMillis diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index 4df3bb5b67..9f58999cbe 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -277,6 +277,19 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } } } + + test("job should fail if TaskResult exceeds Akka frame size") { + // We must use local-cluster mode since results are returned differently + // when running under LocalScheduler: + sc = new SparkContext("local-cluster[1,1,512]", "test") + val akkaFrameSize = + sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt + val rdd = sc.parallelize(Seq(1)).map{x => new Array[Byte](akkaFrameSize)} + val exception = intercept[SparkException] { + rdd.reduce((x, y) => x) + } + exception.getMessage should endWith("result exceeded Akka frame size") + } } object DistributedSuite { -- cgit v1.2.3 From 87540a7b386837d177a6d356ad1f5ef2c1ad6ea5 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Thu, 16 May 2013 15:27:58 +0530 Subject: Fix running on yarn documentation --- docs/running-on-yarn.md | 26 ++++++++++++++++++++++---- 1 file changed, 22 insertions(+), 4 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index c8cf8ffc35..41c0b235dd 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -11,14 +11,32 @@ Ex: mvn -Phadoop2-yarn clean install # Building spark core consolidated jar. -Currently, only sbt can buid a consolidated jar which contains the entire spark code - which is required for launching jars on yarn. -To do this via sbt - though (right now) is a manual process of enabling it in project/SparkBuild.scala. +We need a consolidated spark core jar (which bundles all the required dependencies) to run Spark jobs on a yarn cluster. +This can be built either through sbt or via maven. + +- Building spark assembled jar via sbt. +It is a manual process of enabling it in project/SparkBuild.scala. Please comment out the HADOOP_VERSION, HADOOP_MAJOR_VERSION and HADOOP_YARN variables before the line 'For Hadoop 2 YARN support' Next, uncomment the subsequent 3 variable declaration lines (for these three variables) which enable hadoop yarn support. -Currnetly, it is a TODO to add support for maven assembly. +Assembly of the jar Ex: +./sbt/sbt clean assembly + +The assembled jar would typically be something like : +./streaming/target/spark-streaming-.jar + + +- Building spark assembled jar via sbt. +Use the hadoop2-yarn profile and execute the package target. + +Something like this. Ex: +$ mvn -Phadoop2-yarn clean package -DskipTests=true + + +This will build the shaded (consolidated) jar. Typically something like : +./repl-bin/target/spark-repl-bin--shaded-hadoop2-yarn.jar # Preparations @@ -62,6 +80,6 @@ The above starts a YARN Client programs which periodically polls the Application # Important Notes - When your application instantiates a Spark context it must use a special "standalone" master url. This starts the scheduler without forcing it to connect to a cluster. A good way to handle this is to pass "standalone" as an argument to your program, as shown in the example above. -- YARN does not support requesting container resources based on the number of cores. Thus the numbers of cores given via command line arguments cannot be guaranteed. +- We do not requesting container resources based on the number of cores. Thus the numbers of cores given via command line arguments cannot be guaranteed. - Currently, we have not yet integrated with hadoop security. If --user is present, the hadoop_user specified will be used to run the tasks on the cluster. If unspecified, current user will be used (which should be valid in cluster). Once hadoop security support is added, and if hadoop cluster is enabled with security, additional restrictions would apply via delegation tokens passed. -- cgit v1.2.3 From feddd2530ddfac7a01b03c9113b29945ec0e9a82 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Thu, 16 May 2013 17:49:14 +0530 Subject: Filter out nulls - prevent NPE --- core/src/main/scala/spark/SparkContext.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 2ae4ad8659..15a75c7e93 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -537,6 +537,8 @@ class SparkContext( * filesystems), or an HTTP, HTTPS or FTP URI. */ def addJar(path: String) { + // weird - debug why this is happening. + if (null == path) return val uri = new URI(path) val key = uri.getScheme match { case null | "file" => env.httpFileServer.addJar(new File(uri.getPath)) -- cgit v1.2.3 From f16c781709f9e108d9fe8ac052fb55146ce8a14f Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Thu, 16 May 2013 17:50:22 +0530 Subject: Fix documentation to use yarn-standalone as master --- docs/running-on-yarn.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 41c0b235dd..2e46ff0ed1 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -69,7 +69,7 @@ For example: SPARK_JAR=./core/target/spark-core-assembly-{{site.SPARK_VERSION}}.jar ./run spark.deploy.yarn.Client \ --jar examples/target/scala-{{site.SCALA_VERSION}}/spark-examples_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}.jar \ --class spark.examples.SparkPi \ - --args standalone \ + --args yarn-standalone \ --num-workers 3 \ --master-memory 4g \ --worker-memory 2g \ @@ -79,7 +79,7 @@ The above starts a YARN Client programs which periodically polls the Application # Important Notes -- When your application instantiates a Spark context it must use a special "standalone" master url. This starts the scheduler without forcing it to connect to a cluster. A good way to handle this is to pass "standalone" as an argument to your program, as shown in the example above. +- When your application instantiates a Spark context it must use a special "yarn-standalone" master url. This starts the scheduler without forcing it to connect to a cluster. A good way to handle this is to pass "yarn-standalone" as an argument to your program, as shown in the example above. - We do not requesting container resources based on the number of cores. Thus the numbers of cores given via command line arguments cannot be guaranteed. - Currently, we have not yet integrated with hadoop security. If --user is present, the hadoop_user specified will be used to run the tasks on the cluster. If unspecified, current user will be used (which should be valid in cluster). Once hadoop security support is added, and if hadoop cluster is enabled with security, additional restrictions would apply via delegation tokens passed. -- cgit v1.2.3 From f0881f8d4812dcee955aa303d7a4b76c58b75a61 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Fri, 17 May 2013 01:58:50 +0530 Subject: Hope this does not turn into a bike shed change --- core/src/main/scala/spark/SparkContext.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 15a75c7e93..736b5485b7 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -537,8 +537,10 @@ class SparkContext( * filesystems), or an HTTP, HTTPS or FTP URI. */ def addJar(path: String) { - // weird - debug why this is happening. - if (null == path) return + if (null == path) { + logInfo("null specified as parameter to addJar") + return + } val uri = new URI(path) val key = uri.getScheme match { case null | "file" => env.httpFileServer.addJar(new File(uri.getPath)) -- cgit v1.2.3 From c6e2770bfe940a4f4f26f75c9ba228faea7316f0 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Fri, 17 May 2013 05:10:38 +0800 Subject: Fix ClusterScheduler bug to avoid allocating tasks to same slave --- .../spark/scheduler/cluster/ClusterScheduler.scala | 48 +++++++++++++--------- .../main/scala/spark/scheduler/cluster/Pool.scala | 20 ++++----- .../spark/scheduler/cluster/Schedulable.scala | 3 +- .../spark/scheduler/cluster/TaskSetManager.scala | 8 +++- .../spark/scheduler/ClusterSchedulerSuite.scala | 46 +++++++++++++-------- 5 files changed, 75 insertions(+), 50 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 1a300c9e8c..4caafcc1d3 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -164,27 +164,35 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // Build a list of tasks to assign to each slave val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) val availableCpus = offers.map(o => o.cores).toArray - for (i <- 0 until offers.size) { - var launchedTask = true - val execId = offers(i).executorId - val host = offers(i).hostname - while (availableCpus(i) > 0 && launchedTask) { + var launchedTask = false + val sortedLeafSchedulable = rootPool.getSortedLeafSchedulable() + for (schedulable <- sortedLeafSchedulable) + { + logDebug("parentName:%s,name:%s,runningTasks:%s".format(schedulable.parent.name,schedulable.name,schedulable.runningTasks)) + } + for (schedulable <- sortedLeafSchedulable) { + do { launchedTask = false - rootPool.receiveOffer(execId,host,availableCpus(i)) match { - case Some(task) => - tasks(i) += task - val tid = task.taskId - taskIdToTaskSetId(tid) = task.taskSetId - taskSetTaskIds(task.taskSetId) += tid - taskIdToExecutorId(tid) = execId - activeExecutorIds += execId - executorsByHost(host) += execId - availableCpus(i) -= 1 - launchedTask = true - - case None => {} - } - } + for (i <- 0 until offers.size) { + var launchedTask = true + val execId = offers(i).executorId + val host = offers(i).hostname + schedulable.slaveOffer(execId,host,availableCpus(i)) match { + case Some(task) => + tasks(i) += task + val tid = task.taskId + taskIdToTaskSetId(tid) = task.taskSetId + taskSetTaskIds(task.taskSetId) += tid + taskIdToExecutorId(tid) = execId + activeExecutorIds += execId + executorsByHost(host) += execId + availableCpus(i) -= 1 + launchedTask = true + + case None => {} + } + } + } while(launchedTask) } if (tasks.size > 0) { hasLaunchedTask = true diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/spark/scheduler/cluster/Pool.scala index d5482f71ad..ae603e7dd9 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Pool.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Pool.scala @@ -75,19 +75,17 @@ private[spark] class Pool( return shouldRevive } - override def receiveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { + override def slaveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { + return None + } + + override def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] = { + var leafSchedulableQueue = new ArrayBuffer[Schedulable] val sortedSchedulableQueue = schedulableQueue.sortWith(taskSetSchedulingAlgorithm.comparator) - for (manager <- sortedSchedulableQueue) { - logInfo("parentName:%s,schedulableName:%s,minShares:%d,weight:%d,runningTasks:%d".format( - manager.parent.name, manager.name, manager.minShare, manager.weight, manager.runningTasks)) + for (schedulable <- sortedSchedulableQueue) { + leafSchedulableQueue ++= schedulable.getSortedLeafSchedulable() } - for (manager <- sortedSchedulableQueue) { - val task = manager.receiveOffer(execId, host, availableCpus) - if (task != None) { - return task - } - } - return None + return leafSchedulableQueue } override def increaseRunningTasks(taskNum: Int) { diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala index 54e8ae95f9..c620588e14 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala @@ -21,6 +21,7 @@ private[spark] trait Schedulable { def removeSchedulable(schedulable: Schedulable): Unit def getSchedulableByName(name: String): Schedulable def executorLost(executorId: String, host: String): Unit - def receiveOffer(execId: String, host: String, avaiableCpus: Double): Option[TaskDescription] + def slaveOffer(execId: String, host: String, avaiableCpus: Double): Option[TaskDescription] def checkSpeculatableTasks(): Boolean + def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index baaaa41a37..80edbe77a1 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -198,7 +198,7 @@ private[spark] class TaskSetManager( } // Respond to an offer of a single slave from the scheduler by finding a task - override def receiveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { + override def slaveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { val time = System.currentTimeMillis val localOnly = (time - lastPreferredLaunchTime < LOCALITY_WAIT) @@ -398,6 +398,12 @@ private[spark] class TaskSetManager( //nothing } + override def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] = { + var leafSchedulableQueue = new ArrayBuffer[Schedulable] + leafSchedulableQueue += this + return leafSchedulableQueue + } + override def executorLost(execId: String, hostname: String) { logInfo("Re-queueing tasks for " + execId + " from TaskSet " + taskSet.id) val newHostsAlive = sched.hostsAlive diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala index 2eda48196b..8426be7575 100644 --- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala @@ -6,6 +6,7 @@ import org.scalatest.BeforeAndAfter import spark._ import spark.scheduler._ import spark.scheduler.cluster._ +import scala.collection.mutable.ArrayBuffer import java.util.Properties @@ -25,34 +26,34 @@ class DummyTaskSetManager( var numTasks = initNumTasks var tasksFinished = 0 - def increaseRunningTasks(taskNum: Int) { + override def increaseRunningTasks(taskNum: Int) { runningTasks += taskNum if (parent != null) { parent.increaseRunningTasks(taskNum) } } - def decreaseRunningTasks(taskNum: Int) { + override def decreaseRunningTasks(taskNum: Int) { runningTasks -= taskNum if (parent != null) { parent.decreaseRunningTasks(taskNum) } } - def addSchedulable(schedulable: Schedulable) { + override def addSchedulable(schedulable: Schedulable) { } - def removeSchedulable(schedulable: Schedulable) { + override def removeSchedulable(schedulable: Schedulable) { } - def getSchedulableByName(name: String): Schedulable = { + override def getSchedulableByName(name: String): Schedulable = { return null } - def executorLost(executorId: String, host: String): Unit = { + override def executorLost(executorId: String, host: String): Unit = { } - def receiveOffer(execId: String, host: String, avaiableCpus: Double): Option[TaskDescription] = { + override def slaveOffer(execId: String, host: String, avaiableCpus: Double): Option[TaskDescription] = { if (tasksFinished + runningTasks < numTasks) { increaseRunningTasks(1) return Some(new TaskDescription(0, stageId.toString, execId, "task 0:0", null)) @@ -60,10 +61,16 @@ class DummyTaskSetManager( return None } - def checkSpeculatableTasks(): Boolean = { + override def checkSpeculatableTasks(): Boolean = { return true } + override def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] = { + var leafSchedulableQueue = new ArrayBuffer[Schedulable] + leafSchedulableQueue += this + return leafSchedulableQueue + } + def taskFinished() { decreaseRunningTasks(1) tasksFinished +=1 @@ -80,16 +87,21 @@ class DummyTaskSetManager( class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { - def receiveOffer(rootPool: Pool) : Option[TaskDescription] = { - rootPool.receiveOffer("execId_1", "hostname_1", 1) + def resourceOffer(rootPool: Pool): Int = { + val taskSetQueue = rootPool.getSortedLeafSchedulable() + for (taskSet <- taskSetQueue) + { + taskSet.slaveOffer("execId_1", "hostname_1", 1) match { + case Some(task) => + return task.taskSetId.toInt + case None => {} + } + } + -1 } def checkTaskSetId(rootPool: Pool, expectedTaskSetId: Int) { - receiveOffer(rootPool) match { - case Some(task) => - assert(task.taskSetId.toInt === expectedTaskSetId) - case _ => - } + assert(resourceOffer(rootPool) === expectedTaskSetId) } test("FIFO Scheduler Test") { @@ -105,9 +117,9 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { schedulableBuilder.addTaskSetManager(taskSetManager2, null) checkTaskSetId(rootPool, 0) - receiveOffer(rootPool) + resourceOffer(rootPool) checkTaskSetId(rootPool, 1) - receiveOffer(rootPool) + resourceOffer(rootPool) taskSetManager1.abort() checkTaskSetId(rootPool, 2) } -- cgit v1.2.3 From f742435f18f65e1dbf6235dd49f93b10f22cfe4b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 16 May 2013 14:31:03 -0700 Subject: Removed the duplicated netty dependency in SBT build file. --- project/SparkBuild.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 234b021c93..0ea23b446f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -142,7 +142,6 @@ object SparkBuild extends Build { ), libraryDependencies ++= Seq( - "io.netty" % "netty" % "3.5.3.Final", "com.google.guava" % "guava" % "11.0.1", "log4j" % "log4j" % "1.2.16", "org.slf4j" % "slf4j-api" % slf4jVersion, -- cgit v1.2.3 From 61cf17623835007114ee69394999faaba8a46206 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 16 May 2013 14:31:26 -0700 Subject: Added dependency on netty-all in Maven. --- core/pom.xml | 4 ++++ pom.xml | 5 +++++ 2 files changed, 9 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index 57a95328c3..d8687bf991 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -87,6 +87,10 @@ org.apache.mesos mesos + + io.netty + netty-all + log4j log4j diff --git a/pom.xml b/pom.xml index d7cdc591cf..eda18fdd12 100644 --- a/pom.xml +++ b/pom.xml @@ -256,6 +256,11 @@ mesos ${mesos.version} + + io.netty + netty-all + 4.0.0.Beta2 + org.apache.derby derby -- cgit v1.2.3 From 43644a293f5faec088530cf3a84d3680f2a103af Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 16 May 2013 14:31:38 -0700 Subject: Only check for repl classes if the user is running the repl. Otherwise, check for core classes in run. This fixed the problem that core tests depend on whether repl module is compiled or not. --- run | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/run b/run index c744bbd3dc..c0065c53f1 100755 --- a/run +++ b/run @@ -102,12 +102,18 @@ STREAMING_DIR="$FWDIR/streaming" PYSPARK_DIR="$FWDIR/python" # Exit if the user hasn't compiled Spark -if [ ! -e "$REPL_DIR/target" ]; then - echo "Failed to find Spark classes in $REPL_DIR/target" >&2 +if [ ! -e "$CORE_DIR/target" ]; then + echo "Failed to find Spark classes in $CORE_DIR/target" >&2 echo "You need to compile Spark before running this program" >&2 exit 1 fi +if [[ "$@" = *repl* && ! -e "$REPL_DIR/target" ]]; then + echo "Failed to find Spark classes in $REPL_DIR/target" >&2 + echo "You need to compile Spark repl module before running this program" >&2 + exit 1 +fi + # Build up classpath CLASSPATH="$SPARK_CLASSPATH" CLASSPATH="$CLASSPATH:$FWDIR/conf" -- cgit v1.2.3 From 3b3300383a6ccb9d8b62243c7814eb6c2e1ab313 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 16 May 2013 16:51:28 -0700 Subject: Updated Scala version in docs generation ruby script. --- docs/_plugins/copy_api_dirs.rb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index d77e53963c..c10ae595de 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -18,7 +18,7 @@ if ENV['SKIP_API'] != '1' # Copy over the scaladoc from each project into the docs directory. # This directory will be copied over to _site when `jekyll` command is run. projects.each do |project_name| - source = "../" + project_name + "/target/scala-2.9.2/api" + source = "../" + project_name + "/target/scala-2.9.3/api" dest = "api/" + project_name puts "echo making directory " + dest -- cgit v1.2.3 From dc146406aefa4285d2a2a5d9d45f2ef883e9ef73 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 16 May 2013 17:07:14 -0700 Subject: Updated Scala version in docs generation ruby script. --- docs/_plugins/copy_api_dirs.rb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index d77e53963c..c10ae595de 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -18,7 +18,7 @@ if ENV['SKIP_API'] != '1' # Copy over the scaladoc from each project into the docs directory. # This directory will be copied over to _site when `jekyll` command is run. projects.each do |project_name| - source = "../" + project_name + "/target/scala-2.9.2/api" + source = "../" + project_name + "/target/scala-2.9.3/api" dest = "api/" + project_name puts "echo making directory " + dest -- cgit v1.2.3 From da2642bead2eaf15bfcc28520858cf212d5975a4 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Fri, 17 May 2013 06:58:46 +0530 Subject: Fix example jar name --- docs/running-on-yarn.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 2e46ff0ed1..3946100247 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -25,7 +25,7 @@ Assembly of the jar Ex: ./sbt/sbt clean assembly The assembled jar would typically be something like : -./streaming/target/spark-streaming-.jar +./core/target/spark-core-assembly-0.8.0-SNAPSHOT.jar - Building spark assembled jar via sbt. -- cgit v1.2.3 From d19753b9c78857acae441dce3133fbb6c5855f95 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Sat, 18 May 2013 06:45:19 +0800 Subject: expose TaskSetManager type to resourceOffer function in ClusterScheduler --- .../spark/scheduler/cluster/ClusterScheduler.scala | 14 +-- .../main/scala/spark/scheduler/cluster/Pool.scala | 12 +-- .../spark/scheduler/cluster/Schedulable.scala | 3 +- .../spark/scheduler/cluster/TaskDescription.scala | 1 - .../spark/scheduler/cluster/TaskSetManager.scala | 12 +-- .../spark/scheduler/ClusterSchedulerSuite.scala | 112 ++++++++++++--------- 6 files changed, 84 insertions(+), 70 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 4caafcc1d3..e6399a3547 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -165,24 +165,24 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) val availableCpus = offers.map(o => o.cores).toArray var launchedTask = false - val sortedLeafSchedulable = rootPool.getSortedLeafSchedulable() - for (schedulable <- sortedLeafSchedulable) + val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() + for (manager <- sortedTaskSetQueue) { - logDebug("parentName:%s,name:%s,runningTasks:%s".format(schedulable.parent.name,schedulable.name,schedulable.runningTasks)) + logInfo("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks)) } - for (schedulable <- sortedLeafSchedulable) { + for (manager <- sortedTaskSetQueue) { do { launchedTask = false for (i <- 0 until offers.size) { var launchedTask = true val execId = offers(i).executorId val host = offers(i).hostname - schedulable.slaveOffer(execId,host,availableCpus(i)) match { + manager.slaveOffer(execId,host,availableCpus(i)) match { case Some(task) => tasks(i) += task val tid = task.taskId - taskIdToTaskSetId(tid) = task.taskSetId - taskSetTaskIds(task.taskSetId) += tid + taskIdToTaskSetId(tid) = manager.taskSet.id + taskSetTaskIds(manager.taskSet.id) += tid taskIdToExecutorId(tid) = execId activeExecutorIds += execId executorsByHost(host) += execId diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/spark/scheduler/cluster/Pool.scala index ae603e7dd9..4dc15f413c 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Pool.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Pool.scala @@ -75,17 +75,13 @@ private[spark] class Pool( return shouldRevive } - override def slaveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { - return None - } - - override def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] = { - var leafSchedulableQueue = new ArrayBuffer[Schedulable] + override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { + var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] val sortedSchedulableQueue = schedulableQueue.sortWith(taskSetSchedulingAlgorithm.comparator) for (schedulable <- sortedSchedulableQueue) { - leafSchedulableQueue ++= schedulable.getSortedLeafSchedulable() + sortedTaskSetQueue ++= schedulable.getSortedTaskSetQueue() } - return leafSchedulableQueue + return sortedTaskSetQueue } override def increaseRunningTasks(taskNum: Int) { diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala index c620588e14..6bb7525b49 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala @@ -21,7 +21,6 @@ private[spark] trait Schedulable { def removeSchedulable(schedulable: Schedulable): Unit def getSchedulableByName(name: String): Schedulable def executorLost(executorId: String, host: String): Unit - def slaveOffer(execId: String, host: String, avaiableCpus: Double): Option[TaskDescription] def checkSpeculatableTasks(): Boolean - def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] + def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala index cdd004c94b..b41e951be9 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala @@ -5,7 +5,6 @@ import spark.util.SerializableBuffer private[spark] class TaskDescription( val taskId: Long, - val taskSetId: String, val executorId: String, val name: String, _serializedTask: ByteBuffer) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 80edbe77a1..b9d2dbf487 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -198,7 +198,7 @@ private[spark] class TaskSetManager( } // Respond to an offer of a single slave from the scheduler by finding a task - override def slaveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { + def slaveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = { if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { val time = System.currentTimeMillis val localOnly = (time - lastPreferredLaunchTime < LOCALITY_WAIT) @@ -234,7 +234,7 @@ private[spark] class TaskSetManager( logInfo("Serialized task %s:%d as %d bytes in %d ms".format( taskSet.id, index, serializedTask.limit, timeTaken)) val taskName = "task %s:%d".format(taskSet.id, index) - return Some(new TaskDescription(taskId, taskSet.id, execId, taskName, serializedTask)) + return Some(new TaskDescription(taskId, execId, taskName, serializedTask)) } case _ => } @@ -398,10 +398,10 @@ private[spark] class TaskSetManager( //nothing } - override def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] = { - var leafSchedulableQueue = new ArrayBuffer[Schedulable] - leafSchedulableQueue += this - return leafSchedulableQueue + override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { + var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] + sortedTaskSetQueue += this + return sortedTaskSetQueue } override def executorLost(execId: String, hostname: String) { diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala index 8426be7575..956cc7421c 100644 --- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala @@ -13,18 +13,20 @@ import java.util.Properties class DummyTaskSetManager( initPriority: Int, initStageId: Int, - initNumTasks: Int) - extends Schedulable { - - var parent: Schedulable = null - var weight = 1 - var minShare = 2 - var runningTasks = 0 - var priority = initPriority - var stageId = initStageId - var name = "TaskSet_"+stageId - var numTasks = initNumTasks - var tasksFinished = 0 + initNumTasks: Int, + clusterScheduler: ClusterScheduler, + taskSet: TaskSet) + extends TaskSetManager(clusterScheduler,taskSet) { + + parent = null + weight = 1 + minShare = 2 + runningTasks = 0 + priority = initPriority + stageId = initStageId + name = "TaskSet_"+stageId + override val numTasks = initNumTasks + tasksFinished = 0 override def increaseRunningTasks(taskNum: Int) { runningTasks += taskNum @@ -41,11 +43,11 @@ class DummyTaskSetManager( } override def addSchedulable(schedulable: Schedulable) { - } - + } + override def removeSchedulable(schedulable: Schedulable) { } - + override def getSchedulableByName(name: String): Schedulable = { return null } @@ -65,11 +67,11 @@ class DummyTaskSetManager( return true } - override def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] = { - var leafSchedulableQueue = new ArrayBuffer[Schedulable] - leafSchedulableQueue += this - return leafSchedulableQueue - } +// override def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] = { +// var leafSchedulableQueue = new ArrayBuffer[Schedulable] +// leafSchedulableQueue += this +// return leafSchedulableQueue +// } def taskFinished() { decreaseRunningTasks(1) @@ -85,10 +87,28 @@ class DummyTaskSetManager( } } +class DummyTask(stageId: Int) extends Task[Int](stageId) +{ + def run(attemptId: Long): Int = { + return 0 + } +} + class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { - + + val sc = new SparkContext("local", "ClusterSchedulerSuite") + val clusterScheduler = new ClusterScheduler(sc) + var tasks = ArrayBuffer[Task[_]]() + val task = new DummyTask(0) + val taskSet = new TaskSet(tasks.toArray,0,0,0,null) + tasks += task + + def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int): DummyTaskSetManager = { + new DummyTaskSetManager(priority, stage, numTasks, clusterScheduler, taskSet) + } + def resourceOffer(rootPool: Pool): Int = { - val taskSetQueue = rootPool.getSortedLeafSchedulable() + val taskSetQueue = rootPool.getSortedTaskSetQueue() for (taskSet <- taskSetQueue) { taskSet.slaveOffer("execId_1", "hostname_1", 1) match { @@ -109,13 +129,13 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { val schedulableBuilder = new FIFOSchedulableBuilder(rootPool) schedulableBuilder.buildPools() - val taskSetManager0 = new DummyTaskSetManager(0, 0, 2) - val taskSetManager1 = new DummyTaskSetManager(0, 1, 2) - val taskSetManager2 = new DummyTaskSetManager(0, 2, 2) + val taskSetManager0 = createDummyTaskSetManager(0, 0, 2) + val taskSetManager1 = createDummyTaskSetManager(0, 1, 2) + val taskSetManager2 = createDummyTaskSetManager(0, 2, 2) schedulableBuilder.addTaskSetManager(taskSetManager0, null) schedulableBuilder.addTaskSetManager(taskSetManager1, null) schedulableBuilder.addTaskSetManager(taskSetManager2, null) - + checkTaskSetId(rootPool, 0) resourceOffer(rootPool) checkTaskSetId(rootPool, 1) @@ -130,7 +150,7 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) val schedulableBuilder = new FairSchedulableBuilder(rootPool) schedulableBuilder.buildPools() - + assert(rootPool.getSchedulableByName("default") != null) assert(rootPool.getSchedulableByName("1") != null) assert(rootPool.getSchedulableByName("2") != null) @@ -146,16 +166,16 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { properties1.setProperty("spark.scheduler.cluster.fair.pool","1") val properties2 = new Properties() properties2.setProperty("spark.scheduler.cluster.fair.pool","2") - - val taskSetManager10 = new DummyTaskSetManager(1, 0, 1) - val taskSetManager11 = new DummyTaskSetManager(1, 1, 1) - val taskSetManager12 = new DummyTaskSetManager(1, 2, 2) + + val taskSetManager10 = createDummyTaskSetManager(1, 0, 1) + val taskSetManager11 = createDummyTaskSetManager(1, 1, 1) + val taskSetManager12 = createDummyTaskSetManager(1, 2, 2) schedulableBuilder.addTaskSetManager(taskSetManager10, properties1) schedulableBuilder.addTaskSetManager(taskSetManager11, properties1) schedulableBuilder.addTaskSetManager(taskSetManager12, properties1) - - val taskSetManager23 = new DummyTaskSetManager(2, 3, 2) - val taskSetManager24 = new DummyTaskSetManager(2, 4, 2) + + val taskSetManager23 = createDummyTaskSetManager(2, 3, 2) + val taskSetManager24 = createDummyTaskSetManager(2, 4, 2) schedulableBuilder.addTaskSetManager(taskSetManager23, properties2) schedulableBuilder.addTaskSetManager(taskSetManager24, properties2) @@ -190,27 +210,27 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { val pool11 = new Pool("11", SchedulingMode.FAIR, 2, 1) pool1.addSchedulable(pool10) pool1.addSchedulable(pool11) - - val taskSetManager000 = new DummyTaskSetManager(0, 0, 5) - val taskSetManager001 = new DummyTaskSetManager(0, 1, 5) + + val taskSetManager000 = createDummyTaskSetManager(0, 0, 5) + val taskSetManager001 = createDummyTaskSetManager(0, 1, 5) pool00.addSchedulable(taskSetManager000) pool00.addSchedulable(taskSetManager001) - - val taskSetManager010 = new DummyTaskSetManager(1, 2, 5) - val taskSetManager011 = new DummyTaskSetManager(1, 3, 5) + + val taskSetManager010 = createDummyTaskSetManager(1, 2, 5) + val taskSetManager011 = createDummyTaskSetManager(1, 3, 5) pool01.addSchedulable(taskSetManager010) pool01.addSchedulable(taskSetManager011) - - val taskSetManager100 = new DummyTaskSetManager(2, 4, 5) - val taskSetManager101 = new DummyTaskSetManager(2, 5, 5) + + val taskSetManager100 = createDummyTaskSetManager(2, 4, 5) + val taskSetManager101 = createDummyTaskSetManager(2, 5, 5) pool10.addSchedulable(taskSetManager100) pool10.addSchedulable(taskSetManager101) - val taskSetManager110 = new DummyTaskSetManager(3, 6, 5) - val taskSetManager111 = new DummyTaskSetManager(3, 7, 5) + val taskSetManager110 = createDummyTaskSetManager(3, 6, 5) + val taskSetManager111 = createDummyTaskSetManager(3, 7, 5) pool11.addSchedulable(taskSetManager110) pool11.addSchedulable(taskSetManager111) - + checkTaskSetId(rootPool, 0) checkTaskSetId(rootPool, 4) checkTaskSetId(rootPool, 6) -- cgit v1.2.3 From 0eab7a78b90e2593075c479282f631a5a20e77a9 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 17 May 2013 18:05:46 -0700 Subject: Fixed a couple typos and formating problems in the YARN documentation. --- docs/running-on-yarn.md | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 3946100247..66fb8d73e8 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -14,29 +14,31 @@ Ex: mvn -Phadoop2-yarn clean install We need a consolidated spark core jar (which bundles all the required dependencies) to run Spark jobs on a yarn cluster. This can be built either through sbt or via maven. -- Building spark assembled jar via sbt. -It is a manual process of enabling it in project/SparkBuild.scala. +- Building spark assembled jar via sbt. + It is a manual process of enabling it in project/SparkBuild.scala. Please comment out the HADOOP_VERSION, HADOOP_MAJOR_VERSION and HADOOP_YARN variables before the line 'For Hadoop 2 YARN support' Next, uncomment the subsequent 3 variable declaration lines (for these three variables) which enable hadoop yarn support. -Assembly of the jar Ex: -./sbt/sbt clean assembly +Assembly of the jar Ex: + + ./sbt/sbt clean assembly The assembled jar would typically be something like : -./core/target/spark-core-assembly-0.8.0-SNAPSHOT.jar +`./core/target/spark-core-assembly-0.8.0-SNAPSHOT.jar` -- Building spark assembled jar via sbt. -Use the hadoop2-yarn profile and execute the package target. +- Building spark assembled jar via Maven. + Use the hadoop2-yarn profile and execute the package target. Something like this. Ex: -$ mvn -Phadoop2-yarn clean package -DskipTests=true + + mvn -Phadoop2-yarn clean package -DskipTests=true This will build the shaded (consolidated) jar. Typically something like : -./repl-bin/target/spark-repl-bin--shaded-hadoop2-yarn.jar +`./repl-bin/target/spark-repl-bin--shaded-hadoop2-yarn.jar` # Preparations -- cgit v1.2.3 From 8d78c5f89f25d013c997c03587193f3d87a268b0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 17 May 2013 18:51:35 -0700 Subject: Changed the logging level from info to warning when addJar(null) is called. --- core/src/main/scala/spark/SparkContext.scala | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 736b5485b7..69b4c5d20d 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -538,16 +538,17 @@ class SparkContext( */ def addJar(path: String) { if (null == path) { - logInfo("null specified as parameter to addJar") - return - } - val uri = new URI(path) - val key = uri.getScheme match { - case null | "file" => env.httpFileServer.addJar(new File(uri.getPath)) - case _ => path + logWarning("null specified as parameter to addJar", + new SparkException("null specified as parameter to addJar")) + } else { + val uri = new URI(path) + val key = uri.getScheme match { + case null | "file" => env.httpFileServer.addJar(new File(uri.getPath)) + case _ => path + } + 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 e7982c798efccd523165d0e347c7912ba14fcdd7 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Sat, 18 May 2013 16:11:29 -0700 Subject: Exclude old versions of Netty from Maven-based build --- pom.xml | 6 ++++++ streaming/pom.xml | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/pom.xml b/pom.xml index eda18fdd12..6ee64d07c2 100644 --- a/pom.xml +++ b/pom.xml @@ -565,6 +565,12 @@ org.apache.avro avro-ipc 1.7.1.cloudera.2 + + + org.jboss.netty + netty + + diff --git a/streaming/pom.xml b/streaming/pom.xml index 08ff3e2ae1..4dc9a19d51 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -41,6 +41,12 @@ org.apache.flume flume-ng-sdk 1.2.0 + + + org.jboss.netty + netty + + com.github.sgroschupf -- cgit v1.2.3 From ecd6d75c6a88232c40070baed3dd67bdf77f0c69 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Tue, 21 May 2013 06:49:23 +0800 Subject: fix bug of unit tests --- .../spark/scheduler/cluster/ClusterScheduler.scala | 6 +- .../spark/scheduler/ClusterSchedulerSuite.scala | 72 ++++++++++++---------- 2 files changed, 43 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 9547f4f6dd..053d4b8e4a 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -352,7 +352,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) executorsByHostPort(hostPort) += execId availableCpus(i) -= 1 launchedTask = true - + case None => {} } } @@ -373,7 +373,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } } while (launchedTask) } - + if (tasks.size > 0) { hasLaunchedTask = true } @@ -522,7 +522,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) hostPortsAlive -= hostPort hostToAliveHostPorts.getOrElseUpdate(Utils.parseHostPort(hostPort)._1, new HashSet[String]).remove(hostPort) } - + val execs = executorsByHostPort.getOrElse(hostPort, new HashSet) execs -= executorId if (execs.isEmpty) { diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala index 7af749fb5c..a39418b716 100644 --- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala @@ -67,12 +67,6 @@ class DummyTaskSetManager( return true } -// override def getSortedLeafSchedulable(): ArrayBuffer[Schedulable] = { -// var leafSchedulableQueue = new ArrayBuffer[Schedulable] -// leafSchedulableQueue += this -// return leafSchedulableQueue -// } - def taskFinished() { decreaseRunningTasks(1) tasksFinished +=1 @@ -94,17 +88,10 @@ class DummyTask(stageId: Int) extends Task[Int](stageId) } } -class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { - - val sc = new SparkContext("local", "ClusterSchedulerSuite") - val clusterScheduler = new ClusterScheduler(sc) - var tasks = ArrayBuffer[Task[_]]() - val task = new DummyTask(0) - val taskSet = new TaskSet(tasks.toArray,0,0,0,null) - tasks += task +class ClusterSchedulerSuite extends FunSuite with LocalSparkContext { - def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int): DummyTaskSetManager = { - new DummyTaskSetManager(priority, stage, numTasks, clusterScheduler, taskSet) + def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: ClusterScheduler, taskSet: TaskSet): DummyTaskSetManager = { + new DummyTaskSetManager(priority, stage, numTasks, cs , taskSet) } def resourceOffer(rootPool: Pool): Int = { @@ -125,13 +112,20 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { } test("FIFO Scheduler Test") { + sc = new SparkContext("local", "ClusterSchedulerSuite") + val clusterScheduler = new ClusterScheduler(sc) + var tasks = ArrayBuffer[Task[_]]() + val task = new DummyTask(0) + tasks += task + val taskSet = new TaskSet(tasks.toArray,0,0,0,null) + val rootPool = new Pool("", SchedulingMode.FIFO, 0, 0) val schedulableBuilder = new FIFOSchedulableBuilder(rootPool) schedulableBuilder.buildPools() - val taskSetManager0 = createDummyTaskSetManager(0, 0, 2) - val taskSetManager1 = createDummyTaskSetManager(0, 1, 2) - val taskSetManager2 = createDummyTaskSetManager(0, 2, 2) + val taskSetManager0 = createDummyTaskSetManager(0, 0, 2, clusterScheduler, taskSet) + val taskSetManager1 = createDummyTaskSetManager(0, 1, 2, clusterScheduler, taskSet) + val taskSetManager2 = createDummyTaskSetManager(0, 2, 2, clusterScheduler, taskSet) schedulableBuilder.addTaskSetManager(taskSetManager0, null) schedulableBuilder.addTaskSetManager(taskSetManager1, null) schedulableBuilder.addTaskSetManager(taskSetManager2, null) @@ -145,6 +139,13 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { } test("Fair Scheduler Test") { + sc = new SparkContext("local", "ClusterSchedulerSuite") + val clusterScheduler = new ClusterScheduler(sc) + var tasks = ArrayBuffer[Task[_]]() + val task = new DummyTask(0) + tasks += task + val taskSet = new TaskSet(tasks.toArray,0,0,0,null) + val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() System.setProperty("spark.fairscheduler.allocation.file", xmlPath) val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) @@ -167,15 +168,15 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { val properties2 = new Properties() properties2.setProperty("spark.scheduler.cluster.fair.pool","2") - val taskSetManager10 = createDummyTaskSetManager(1, 0, 1) - val taskSetManager11 = createDummyTaskSetManager(1, 1, 1) - val taskSetManager12 = createDummyTaskSetManager(1, 2, 2) + val taskSetManager10 = createDummyTaskSetManager(1, 0, 1, clusterScheduler, taskSet) + val taskSetManager11 = createDummyTaskSetManager(1, 1, 1, clusterScheduler, taskSet) + val taskSetManager12 = createDummyTaskSetManager(1, 2, 2, clusterScheduler, taskSet) schedulableBuilder.addTaskSetManager(taskSetManager10, properties1) schedulableBuilder.addTaskSetManager(taskSetManager11, properties1) schedulableBuilder.addTaskSetManager(taskSetManager12, properties1) - val taskSetManager23 = createDummyTaskSetManager(2, 3, 2) - val taskSetManager24 = createDummyTaskSetManager(2, 4, 2) + val taskSetManager23 = createDummyTaskSetManager(2, 3, 2, clusterScheduler, taskSet) + val taskSetManager24 = createDummyTaskSetManager(2, 4, 2, clusterScheduler, taskSet) schedulableBuilder.addTaskSetManager(taskSetManager23, properties2) schedulableBuilder.addTaskSetManager(taskSetManager24, properties2) @@ -195,6 +196,13 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { } test("Nested Pool Test") { + sc = new SparkContext("local", "ClusterSchedulerSuite") + val clusterScheduler = new ClusterScheduler(sc) + var tasks = ArrayBuffer[Task[_]]() + val task = new DummyTask(0) + tasks += task + val taskSet = new TaskSet(tasks.toArray,0,0,0,null) + val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) val pool0 = new Pool("0", SchedulingMode.FAIR, 3, 1) val pool1 = new Pool("1", SchedulingMode.FAIR, 4, 1) @@ -211,23 +219,23 @@ class ClusterSchedulerSuite extends FunSuite with BeforeAndAfter { pool1.addSchedulable(pool10) pool1.addSchedulable(pool11) - val taskSetManager000 = createDummyTaskSetManager(0, 0, 5) - val taskSetManager001 = createDummyTaskSetManager(0, 1, 5) + val taskSetManager000 = createDummyTaskSetManager(0, 0, 5, clusterScheduler, taskSet) + val taskSetManager001 = createDummyTaskSetManager(0, 1, 5, clusterScheduler, taskSet) pool00.addSchedulable(taskSetManager000) pool00.addSchedulable(taskSetManager001) - val taskSetManager010 = createDummyTaskSetManager(1, 2, 5) - val taskSetManager011 = createDummyTaskSetManager(1, 3, 5) + val taskSetManager010 = createDummyTaskSetManager(1, 2, 5, clusterScheduler, taskSet) + val taskSetManager011 = createDummyTaskSetManager(1, 3, 5, clusterScheduler, taskSet) pool01.addSchedulable(taskSetManager010) pool01.addSchedulable(taskSetManager011) - val taskSetManager100 = createDummyTaskSetManager(2, 4, 5) - val taskSetManager101 = createDummyTaskSetManager(2, 5, 5) + val taskSetManager100 = createDummyTaskSetManager(2, 4, 5, clusterScheduler, taskSet) + val taskSetManager101 = createDummyTaskSetManager(2, 5, 5, clusterScheduler, taskSet) pool10.addSchedulable(taskSetManager100) pool10.addSchedulable(taskSetManager101) - val taskSetManager110 = createDummyTaskSetManager(3, 6, 5) - val taskSetManager111 = createDummyTaskSetManager(3, 7, 5) + val taskSetManager110 = createDummyTaskSetManager(3, 6, 5, clusterScheduler, taskSet) + val taskSetManager111 = createDummyTaskSetManager(3, 7, 5, clusterScheduler, taskSet) pool11.addSchedulable(taskSetManager110) pool11.addSchedulable(taskSetManager111) -- cgit v1.2.3 From 3217d486f7fdd590250f2efee567e4779e130d34 Mon Sep 17 00:00:00 2001 From: Ethan Jewett Date: Mon, 20 May 2013 19:41:38 -0500 Subject: Add hBase dependency to examples POM --- examples/pom.xml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/examples/pom.xml b/examples/pom.xml index c42d2bcdb9..0fbb5a3d5d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -67,6 +67,11 @@ hadoop-core provided + + org.apache.hbase + hbase + 0.94.6 + @@ -105,6 +110,11 @@ hadoop-client provided + + org.apache.hbase + hbase + 0.94.6 + -- cgit v1.2.3 From 786c97b87c9d3074796e2d931635d9c6f72b9704 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Wed, 27 Feb 2013 10:06:05 -0800 Subject: DistributedSuite: remove dead test code --- core/src/test/scala/spark/DistributedSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index 068bb6ca4f..0866fb47b3 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -222,7 +222,6 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter sc = new SparkContext(clusterUrl, "test") val data = sc.parallelize(Seq(true, true), 2) assert(data.count === 2) // force executors to start - val masterId = SparkEnv.get.blockManager.blockManagerId assert(data.map(markNodeIfIdentity).collect.size === 2) assert(data.map(failOnMarkedIdentity).collect.size === 2) } -- cgit v1.2.3 From f350f14084dd04a2ea77e92e35b3cf415ef72202 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Tue, 5 Feb 2013 17:08:55 -0800 Subject: Use ARRAY_SAMPLE_SIZE constant instead of 100.0 --- core/src/main/scala/spark/SizeEstimator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SizeEstimator.scala b/core/src/main/scala/spark/SizeEstimator.scala index d4e1157250..f8a4c4e489 100644 --- a/core/src/main/scala/spark/SizeEstimator.scala +++ b/core/src/main/scala/spark/SizeEstimator.scala @@ -198,7 +198,7 @@ private[spark] object SizeEstimator extends Logging { val elem = JArray.get(array, index) size += SizeEstimator.estimate(elem, state.visited) } - state.size += ((length / 100.0) * size).toLong + state.size += ((length / (ARRAY_SAMPLE_SIZE * 1.0)) * size).toLong } } } -- cgit v1.2.3 From bd3ea8f2a66de5ddc12dc1b2273e675d0abb8393 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 24 May 2013 14:26:19 +0800 Subject: fix CheckpointRDD getPreferredLocations java.io.FileNotFoundException --- core/src/main/scala/spark/rdd/CheckpointRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 43ee39c993..377b1bdbe0 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -43,7 +43,7 @@ class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: Stri checkpointData.get.cpFile = Some(checkpointPath) override def getPreferredLocations(split: Partition): Seq[String] = { - val status = fs.getFileStatus(new Path(checkpointPath)) + val status = fs.getFileStatus(new Path(checkpointPath, CheckpointRDD.splitIdToFile(split.index))) val locations = fs.getFileBlockLocations(status, 0, status.getLen) locations.headOption.toList.flatMap(_.getHosts).filter(_ != "localhost") } -- cgit v1.2.3 From cda2b150412314c47c2c24883111bfc441c3a3a2 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 24 May 2013 13:05:06 -0700 Subject: Use ec2-metadata in start-slave.sh. PR #419 applied the same change, but only to start-master.sh, so some workers were still starting their web UI's using internal addresses. This should finally fix SPARK-613. --- bin/start-slave.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/bin/start-slave.sh b/bin/start-slave.sh index 616c76e4ee..26b5b9d462 100755 --- a/bin/start-slave.sh +++ b/bin/start-slave.sh @@ -6,7 +6,8 @@ bin=`cd "$bin"; pwd` # Set SPARK_PUBLIC_DNS so slaves can be linked in master web UI if [ "$SPARK_PUBLIC_DNS" = "" ]; then # If we appear to be running on EC2, use the public address by default: - if [[ `hostname` == *ec2.internal ]]; then + # NOTE: ec2-metadata is installed on Amazon Linux AMI. Check based on that and hostname + if command -v ec2-metadata > /dev/null || [[ `hostname` == *ec2.internal ]]; then export SPARK_PUBLIC_DNS=`wget -q -O - http://instance-data.ec2.internal/latest/meta-data/public-hostname` fi fi -- cgit v1.2.3 From 6ea085169d8ba2d09ca9236273d65238b8411f04 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 24 May 2013 14:08:37 -0700 Subject: Fixed the bug that shuffle serializer is ignored by the new shuffle block iterators for local blocks. Also added a unit test for that. --- .../scala/spark/storage/BlockFetcherIterator.scala | 2 +- core/src/test/scala/spark/ShuffleSuite.scala | 21 ++++++++++++++++++--- 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index 43f835237c..88eed0d8c8 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -163,7 +163,7 @@ object BlockFetcherIterator { // these all at once because they will just memory-map some files, so they won't consume // any memory that might exceed our maxBytesInFlight for (id <- localBlockIds) { - getLocal(id) match { + getLocalFromDisk(id, serializer) match { case Some(iter) => { // Pass 0 as size since it's not in flight results.put(new FetchResult(id, 0, () => iter)) diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 2b2a90defa..fdee7ca384 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -99,7 +99,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { val sums = pairs.reduceByKey(_+_, 10).collect() assert(sums.toSet === Set((1, 7), (2, 1))) } - + test("reduceByKey with partitioner") { sc = new SparkContext("local", "test") val p = new Partitioner() { @@ -272,7 +272,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { } // partitionBy so we have a narrow dependency val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).partitionBy(p) - // more partitions/no partitioner so a shuffle dependency + // more partitions/no partitioner so a shuffle dependency val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) val c = a.subtract(b) assert(c.collect().toSet === Set((1, "a"), (3, "c"))) @@ -298,18 +298,33 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { } // partitionBy so we have a narrow dependency val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).partitionBy(p) - // more partitions/no partitioner so a shuffle dependency + // more partitions/no partitioner so a shuffle dependency val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) val c = a.subtractByKey(b) assert(c.collect().toSet === Set((1, "a"), (1, "a"))) assert(c.partitioner.get === p) } + test("shuffle serializer") { + // Use a local cluster with 2 processes to make sure there are both local and remote blocks + sc = new SparkContext("local-cluster[1,2,512]", "test") + val a = sc.parallelize(1 to 10, 2) + val b = a.map { x => + (x, new ShuffleSuite.NonJavaSerializableClass(x * 2)) + } + // If the Kryo serializer is not used correctly, the shuffle would fail because the + // default Java serializer cannot handle the non serializable class. + val c = new ShuffledRDD(b, new HashPartitioner(3), classOf[spark.KryoSerializer].getName) + assert(c.count === 10) + } } object ShuffleSuite { + def mergeCombineException(x: Int, y: Int): Int = { throw new SparkException("Exception for map-side combine.") x + y } + + class NonJavaSerializableClass(val value: Int) } -- cgit v1.2.3 From 26962c9340ac92b11d43e87200e699471d0b6330 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 24 May 2013 16:39:33 -0700 Subject: Automatically configure Netty port. This makes unit tests using local-cluster pass. Previously they were failing because Netty was trying to bind to the same port for all processes. Pair programmed with @shivaram. --- .../main/java/spark/network/netty/FileServer.java | 68 ++++++++++++++++------ .../scala/spark/network/netty/ShuffleSender.scala | 23 ++++---- .../scala/spark/storage/BlockFetcherIterator.scala | 3 +- .../main/scala/spark/storage/BlockManager.scala | 13 +++-- .../main/scala/spark/storage/BlockManagerId.scala | 32 +++++++--- core/src/main/scala/spark/storage/DiskStore.scala | 52 ++++------------- .../test/scala/spark/MapOutputTrackerSuite.scala | 28 ++++----- core/src/test/scala/spark/ShuffleSuite.scala | 2 +- .../scala/spark/scheduler/DAGSchedulerSuite.scala | 14 ++--- .../scala/spark/storage/BlockManagerSuite.scala | 6 +- 10 files changed, 129 insertions(+), 112 deletions(-) diff --git a/core/src/main/java/spark/network/netty/FileServer.java b/core/src/main/java/spark/network/netty/FileServer.java index 647b26bf8a..dd3f12561c 100644 --- a/core/src/main/java/spark/network/netty/FileServer.java +++ b/core/src/main/java/spark/network/netty/FileServer.java @@ -1,51 +1,83 @@ package spark.network.netty; +import java.net.InetSocketAddress; + import io.netty.bootstrap.ServerBootstrap; -import io.netty.channel.ChannelOption; import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelOption; import io.netty.channel.oio.OioEventLoopGroup; import io.netty.channel.socket.oio.OioServerSocketChannel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * Server that accept the path of a file an echo back its content. */ class FileServer { + private Logger LOG = LoggerFactory.getLogger(this.getClass().getName()); + private ServerBootstrap bootstrap = null; - private Channel channel = null; - private PathResolver pResolver; + private ChannelFuture channelFuture = null; + private int port = 0; + private Thread blockingThread = null; - public FileServer(PathResolver pResolver) { - this.pResolver = pResolver; - } + public FileServer(PathResolver pResolver, int port) { + InetSocketAddress addr = new InetSocketAddress(port); - public void run(int port) { // Configure the server. bootstrap = new ServerBootstrap(); - try { - bootstrap.group(new OioEventLoopGroup(), new OioEventLoopGroup()) + bootstrap.group(new OioEventLoopGroup(), new OioEventLoopGroup()) .channel(OioServerSocketChannel.class) .option(ChannelOption.SO_BACKLOG, 100) .option(ChannelOption.SO_RCVBUF, 1500) .childHandler(new FileServerChannelInitializer(pResolver)); - // Start the server. - channel = bootstrap.bind(port).sync().channel(); - channel.closeFuture().sync(); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } finally{ + // Start the server. + channelFuture = bootstrap.bind(addr); + this.port = addr.getPort(); + } + + /** + * Start the file server asynchronously in a new thread. + */ + public void start() { + try { + blockingThread = new Thread() { + public void run() { + try { + Channel channel = channelFuture.sync().channel(); + channel.closeFuture().sync(); + } catch (InterruptedException e) { + LOG.error("File server start got interrupted", e); + } + } + }; + blockingThread.setDaemon(true); + blockingThread.start(); + } finally { bootstrap.shutdown(); } } + public int getPort() { + return port; + } + public void stop() { - if (channel!=null) { - channel.close(); + if (blockingThread != null) { + blockingThread.stop(); + blockingThread = null; + } + if (channelFuture != null) { + channelFuture.channel().closeFuture(); + channelFuture = null; } if (bootstrap != null) { bootstrap.shutdown(); + bootstrap = null; } } } diff --git a/core/src/main/scala/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/spark/network/netty/ShuffleSender.scala index dc87fefc56..d6fa4b1e80 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleSender.scala +++ b/core/src/main/scala/spark/network/netty/ShuffleSender.scala @@ -5,23 +5,22 @@ import java.io.File import spark.Logging -private[spark] class ShuffleSender(val port: Int, val pResolver: PathResolver) extends Logging { - val server = new FileServer(pResolver) +private[spark] class ShuffleSender(portIn: Int, val pResolver: PathResolver) extends Logging { - Runtime.getRuntime().addShutdownHook( - new Thread() { - override def run() { - server.stop() - } - } - ) + val server = new FileServer(pResolver, portIn) + server.start() - def start() { - server.run(port) + def stop() { + server.stop() } + + def port: Int = server.getPort() } +/** + * An application for testing the shuffle sender as a standalone program. + */ private[spark] object ShuffleSender { def main(args: Array[String]) { @@ -50,7 +49,5 @@ private[spark] object ShuffleSender { } } val sender = new ShuffleSender(port, pResovler) - - sender.start() } } diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index 88eed0d8c8..95308c7282 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -272,8 +272,7 @@ object BlockFetcherIterator { logDebug("Sending request for %d blocks (%s) from %s".format( req.blocks.size, Utils.memoryBytesToString(req.size), req.address.host)) - val cmId = new ConnectionManagerId( - req.address.host, System.getProperty("spark.shuffle.sender.port", "6653").toInt) + val cmId = new ConnectionManagerId(req.address.host, req.address.nettyPort) val cpier = new ShuffleCopier cpier.getBlocks(cmId, req.blocks, putResult) logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.host ) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 40d608628e..d35c43f194 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -94,11 +94,16 @@ private[spark] class BlockManager( private[storage] val diskStore: DiskStore = new DiskStore(this, System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir"))) + // If we use Netty for shuffle, start a new Netty-based shuffle sender service. + private val useNetty = System.getProperty("spark.shuffle.use.netty", "false").toBoolean + private val nettyPortConfig = System.getProperty("spark.shuffle.sender.port", "0").toInt + private val nettyPort = if (useNetty) diskStore.startShuffleBlockSender(nettyPortConfig) else 0 + val connectionManager = new ConnectionManager(0) implicit val futureExecContext = connectionManager.futureExecContext val blockManagerId = BlockManagerId( - executorId, connectionManager.id.host, connectionManager.id.port) + executorId, connectionManager.id.host, connectionManager.id.port, nettyPort) // Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory // for receiving shuffle outputs) @@ -266,7 +271,6 @@ private[spark] class BlockManager( } } - /** * Get locations of an array of blocks. */ @@ -274,7 +278,7 @@ private[spark] class BlockManager( val startTimeMs = System.currentTimeMillis val locations = master.getLocations(blockIds).toArray logDebug("Got multiple block location in " + Utils.getUsedTimeMs(startTimeMs)) - return locations + locations } /** @@ -971,8 +975,7 @@ private[spark] object BlockManager extends Logging { assert (env != null || blockManagerMaster != null) val locationBlockIds: Seq[Seq[BlockManagerId]] = if (env != null) { - val blockManager = env.blockManager - blockManager.getLocationBlockIds(blockIds) + env.blockManager.getLocationBlockIds(blockIds) } else { blockManagerMaster.getLocations(blockIds) } diff --git a/core/src/main/scala/spark/storage/BlockManagerId.scala b/core/src/main/scala/spark/storage/BlockManagerId.scala index f4a2181490..1e557d6148 100644 --- a/core/src/main/scala/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/spark/storage/BlockManagerId.scala @@ -7,18 +7,19 @@ import spark.Utils /** * This class represent an unique identifier for a BlockManager. * The first 2 constructors of this class is made private to ensure that - * BlockManagerId objects can be created only using the factory method in - * [[spark.storage.BlockManager$]]. This allows de-duplication of ID objects. + * BlockManagerId objects can be created only using the apply method in + * the companion object. This allows de-duplication of ID objects. * Also, constructor parameters are private to ensure that parameters cannot * be modified from outside this class. */ private[spark] class BlockManagerId private ( private var executorId_ : String, private var host_ : String, - private var port_ : Int + private var port_ : Int, + private var nettyPort_ : Int ) extends Externalizable { - private def this() = this(null, null, 0) // For deserialization only + private def this() = this(null, null, 0, 0) // For deserialization only def executorId: String = executorId_ @@ -39,28 +40,32 @@ private[spark] class BlockManagerId private ( def port: Int = port_ + def nettyPort: Int = nettyPort_ + override def writeExternal(out: ObjectOutput) { out.writeUTF(executorId_) out.writeUTF(host_) out.writeInt(port_) + out.writeInt(nettyPort_) } override def readExternal(in: ObjectInput) { executorId_ = in.readUTF() host_ = in.readUTF() port_ = in.readInt() + nettyPort_ = in.readInt() } @throws(classOf[IOException]) private def readResolve(): Object = BlockManagerId.getCachedBlockManagerId(this) - override def toString = "BlockManagerId(%s, %s, %d)".format(executorId, host, port) + override def toString = "BlockManagerId(%s, %s, %d, %d)".format(executorId, host, port, nettyPort) - override def hashCode: Int = (executorId.hashCode * 41 + host.hashCode) * 41 + port + override def hashCode: Int = (executorId.hashCode * 41 + host.hashCode) * 41 + port + nettyPort override def equals(that: Any) = that match { case id: BlockManagerId => - executorId == id.executorId && port == id.port && host == id.host + executorId == id.executorId && port == id.port && host == id.host && nettyPort == id.nettyPort case _ => false } @@ -69,8 +74,17 @@ private[spark] class BlockManagerId private ( private[spark] object BlockManagerId { - def apply(execId: String, host: String, port: Int) = - getCachedBlockManagerId(new BlockManagerId(execId, host, port)) + /** + * Returns a [[spark.storage.BlockManagerId]] for the given configuraiton. + * + * @param execId ID of the executor. + * @param host Host name of the block manager. + * @param port Port of the block manager. + * @param nettyPort Optional port for the Netty-based shuffle sender. + * @return A new [[spark.storage.BlockManagerId]]. + */ + def apply(execId: String, host: String, port: Int, nettyPort: Int) = + getCachedBlockManagerId(new BlockManagerId(execId, host, port, nettyPort)) def apply(in: ObjectInput) = { val obj = new BlockManagerId() diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 933eeaa216..57d4dafefc 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -82,22 +82,15 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) val MAX_DIR_CREATION_ATTEMPTS: Int = 10 val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt - var shuffleSender : Thread = null - val thisInstance = this + var shuffleSender : ShuffleSender = null // Create one local directory for each path mentioned in spark.local.dir; then, inside this // directory, create multiple subdirectories that we will hash files into, in order to avoid // having really large inodes at the top level. val localDirs = createLocalDirs() val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) - val useNetty = System.getProperty("spark.shuffle.use.netty", "false").toBoolean - addShutdownHook() - if(useNetty){ - startShuffleBlockSender() - } - def getBlockWriter(blockId: String, serializer: Serializer, bufferSize: Int) : BlockObjectWriter = { new DiskBlockObjectWriter(blockId, serializer, bufferSize) @@ -274,8 +267,9 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) localDirs.foreach { localDir => if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) } - if (useNetty && shuffleSender != null) + if (shuffleSender != null) { shuffleSender.stop + } } catch { case t: Throwable => logError("Exception while deleting local spark dirs", t) } @@ -283,39 +277,17 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) }) } - private def startShuffleBlockSender() { - try { - val port = System.getProperty("spark.shuffle.sender.port", "6653").toInt - - val pResolver = new PathResolver { - override def getAbsolutePath(blockId: String): String = { - if (!blockId.startsWith("shuffle_")) { - return null - } - thisInstance.getFile(blockId).getAbsolutePath() - } - } - shuffleSender = new Thread { - override def run() = { - val sender = new ShuffleSender(port, pResolver) - logInfo("Created ShuffleSender binding to port : "+ port) - sender.start - } - } - shuffleSender.setDaemon(true) - shuffleSender.start - - } catch { - case interrupted: InterruptedException => - logInfo("Runner thread for ShuffleBlockSender interrupted") - - case e: Exception => { - logError("Error running ShuffleBlockSender ", e) - if (shuffleSender != null) { - shuffleSender.stop - shuffleSender = null + private[storage] def startShuffleBlockSender(port: Int): Int = { + val pResolver = new PathResolver { + override def getAbsolutePath(blockId: String): String = { + if (!blockId.startsWith("shuffle_")) { + return null } + DiskStore.this.getFile(blockId).getAbsolutePath() } } + shuffleSender = new ShuffleSender(port, pResolver) + logInfo("Created ShuffleSender binding to port : "+ shuffleSender.port) + shuffleSender.port } } diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index b5cedc0b68..6e585e1c3a 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -8,7 +8,7 @@ import spark.storage.BlockManagerId import spark.util.AkkaUtils class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { - + test("compressSize") { assert(MapOutputTracker.compressSize(0L) === 0) assert(MapOutputTracker.compressSize(1L) === 1) @@ -45,13 +45,13 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val compressedSize10000 = MapOutputTracker.compressSize(10000L) val size1000 = MapOutputTracker.decompressSize(compressedSize1000) val size10000 = MapOutputTracker.decompressSize(compressedSize10000) - tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000), + tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000, 0), Array(compressedSize1000, compressedSize10000))) - tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000), + tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000, 0), Array(compressedSize10000, compressedSize1000))) val statuses = tracker.getServerStatuses(10, 0) - assert(statuses.toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000), - (BlockManagerId("b", "hostB", 1000), size10000))) + assert(statuses.toSeq === Seq((BlockManagerId("a", "hostA", 1000, 0), size1000), + (BlockManagerId("b", "hostB", 1000, 0), size10000))) tracker.stop() } @@ -64,14 +64,14 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val compressedSize10000 = MapOutputTracker.compressSize(10000L) val size1000 = MapOutputTracker.decompressSize(compressedSize1000) val size10000 = MapOutputTracker.decompressSize(compressedSize10000) - tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000), + tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000, 0), Array(compressedSize1000, compressedSize1000, compressedSize1000))) - tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000), + tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000, 0), Array(compressedSize10000, compressedSize1000, compressedSize1000))) // As if we had two simulatenous fetch failures - tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) - tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) + tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000, 0)) + tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000, 0)) // The remaining reduce task might try to grab the output despite the shuffle failure; // this should cause it to fail, and the scheduler will ignore the failure due to the @@ -88,12 +88,12 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val masterTracker = new MapOutputTracker() masterTracker.trackerActor = actorSystem.actorOf( Props(new MapOutputTrackerActor(masterTracker)), "MapOutputTracker") - + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0) val slaveTracker = new MapOutputTracker() slaveTracker.trackerActor = slaveSystem.actorFor( "akka://spark@localhost:" + boundPort + "/user/MapOutputTracker") - + masterTracker.registerShuffle(10, 1) masterTracker.incrementGeneration() slaveTracker.updateGeneration(masterTracker.getGeneration) @@ -102,13 +102,13 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val compressedSize1000 = MapOutputTracker.compressSize(1000L) val size1000 = MapOutputTracker.decompressSize(compressedSize1000) masterTracker.registerMapOutput(10, 0, new MapStatus( - BlockManagerId("a", "hostA", 1000), Array(compressedSize1000))) + BlockManagerId("a", "hostA", 1000, 0), Array(compressedSize1000))) masterTracker.incrementGeneration() slaveTracker.updateGeneration(masterTracker.getGeneration) assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq((BlockManagerId("a", "hostA", 1000), size1000))) + Seq((BlockManagerId("a", "hostA", 1000, 0), size1000))) - masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) + masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000, 0)) masterTracker.incrementGeneration() slaveTracker.updateGeneration(masterTracker.getGeneration) intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index fdee7ca384..58c834c735 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -326,5 +326,5 @@ object ShuffleSuite { x + y } - class NonJavaSerializableClass(val value: Int) + class NonJavaSerializableClass(val value: Int) extends Serializable } diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 16554eac6e..30e6fef950 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -44,7 +44,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont override def submitTasks(taskSet: TaskSet) = { // normally done by TaskSetManager taskSet.tasks.foreach(_.generation = mapOutputTracker.getGeneration) - taskSets += taskSet + taskSets += taskSet } override def setListener(listener: TaskSchedulerListener) = {} override def defaultParallelism() = 2 @@ -164,7 +164,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont } } } - + /** Sends the rdd to the scheduler for scheduling. */ private def submit( rdd: RDD[_], @@ -174,7 +174,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont listener: JobListener = listener) { runEvent(JobSubmitted(rdd, func, partitions, allowLocal, null, listener)) } - + /** Sends TaskSetFailed to the scheduler. */ private def failed(taskSet: TaskSet, message: String) { runEvent(TaskSetFailed(taskSet, message)) @@ -209,11 +209,11 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont runEvent(JobSubmitted(rdd, jobComputeFunc, Array(0), true, null, listener)) assert(results === Map(0 -> 42)) } - + test("run trivial job w/ dependency") { val baseRdd = makeRdd(1, Nil) val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) - submit(finalRdd, Array(0)) + submit(finalRdd, Array(0)) complete(taskSets(0), Seq((Success, 42))) assert(results === Map(0 -> 42)) } @@ -250,7 +250,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont complete(taskSets(1), Seq((Success, 42))) assert(results === Map(0 -> 42)) } - + test("run trivial shuffle with fetch failure") { val shuffleMapRdd = makeRdd(2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) @@ -398,6 +398,6 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont new MapStatus(makeBlockManagerId(host), Array.fill[Byte](reduces)(2)) private def makeBlockManagerId(host: String): BlockManagerId = - BlockManagerId("exec-" + host, host, 12345) + BlockManagerId("exec-" + host, host, 12345, 0) } diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index 71d1f0bcc8..bff2475686 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -99,9 +99,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("BlockManagerId object caching") { - val id1 = BlockManagerId("e1", "XXX", 1) - val id2 = BlockManagerId("e1", "XXX", 1) // this should return the same object as id1 - val id3 = BlockManagerId("e1", "XXX", 2) // this should return a different object + val id1 = BlockManagerId("e1", "XXX", 1, 0) + val id2 = BlockManagerId("e1", "XXX", 1, 0) // this should return the same object as id1 + val id3 = BlockManagerId("e1", "XXX", 2, 0) // this should return a different object assert(id2 === id1, "id2 is not same as id1") assert(id2.eq(id1), "id2 is not the same object as id1") assert(id3 != id1, "id3 is same as id1") -- cgit v1.2.3 From 6bbbe012877115eab084fea09baf677abaf52f2b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 24 May 2013 16:51:45 -0700 Subject: Fixed a stupid mistake that NonJavaSerializableClass was made Java serializable. --- core/src/test/scala/spark/ShuffleSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 58c834c735..fdee7ca384 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -326,5 +326,5 @@ object ShuffleSuite { x + y } - class NonJavaSerializableClass(val value: Int) extends Serializable + class NonJavaSerializableClass(val value: Int) } -- cgit v1.2.3 From a674d67c0aebb940e3b816e2307206115baec175 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 28 May 2013 16:24:05 -0500 Subject: Fix start-slave not passing instance number to spark-daemon. --- bin/start-slave.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/start-slave.sh b/bin/start-slave.sh index 26b5b9d462..dfcbc6981b 100755 --- a/bin/start-slave.sh +++ b/bin/start-slave.sh @@ -12,4 +12,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then fi fi -"$bin"/spark-daemon.sh start spark.deploy.worker.Worker "$@" +"$bin"/spark-daemon.sh start spark.deploy.worker.Worker 1 "$@" -- cgit v1.2.3 From 4fe1fbdd51f781157138ffd35da5834366379688 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 28 May 2013 16:26:32 -0500 Subject: Remove unused addIfNoPort. --- core/src/main/scala/spark/Utils.scala | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index c1495d5317..84626df553 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -403,17 +403,6 @@ private object Utils extends Logging { hostPortParseResults.get(hostPort) } - def addIfNoPort(hostPort: String, port: Int): String = { - if (port <= 0) throw new IllegalArgumentException("Invalid port specified " + port) - - // This is potentially broken - when dealing with ipv6 addresses for example, sigh ... but then hadoop does not support ipv6 right now. - // For now, we assume that if port exists, then it is valid - not check if it is an int > 0 - val indx: Int = hostPort.lastIndexOf(':') - if (-1 != indx) return hostPort - - hostPort + ":" + port - } - private[spark] val daemonThreadFactory: ThreadFactory = new ThreadFactoryBuilder().setDaemon(true).build() -- cgit v1.2.3 From fbc1ab346867d5c81dc59e4c8d85aeda2f516ce2 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Tue, 28 May 2013 16:27:16 -0700 Subject: Couple of Netty fixes a. Fix the port number by reading it from the bound channel b. Fix the shutdown sequence to make sure we actually block on the channel c. Fix the unit test to use two JVMs. --- .../main/java/spark/network/netty/FileServer.java | 45 ++++++++++++---------- core/src/test/scala/spark/ShuffleSuite.scala | 14 ++++++- 2 files changed, 37 insertions(+), 22 deletions(-) diff --git a/core/src/main/java/spark/network/netty/FileServer.java b/core/src/main/java/spark/network/netty/FileServer.java index dd3f12561c..dd3a557ae5 100644 --- a/core/src/main/java/spark/network/netty/FileServer.java +++ b/core/src/main/java/spark/network/netty/FileServer.java @@ -37,29 +37,33 @@ class FileServer { .childHandler(new FileServerChannelInitializer(pResolver)); // Start the server. channelFuture = bootstrap.bind(addr); - this.port = addr.getPort(); + try { + // Get the address we bound to. + InetSocketAddress boundAddress = + ((InetSocketAddress) channelFuture.sync().channel().localAddress()); + this.port = boundAddress.getPort(); + } catch (InterruptedException ie) { + this.port = 0; + } } /** * Start the file server asynchronously in a new thread. */ public void start() { - try { - blockingThread = new Thread() { - public void run() { - try { - Channel channel = channelFuture.sync().channel(); - channel.closeFuture().sync(); - } catch (InterruptedException e) { - LOG.error("File server start got interrupted", e); - } + blockingThread = new Thread() { + public void run() { + try { + channelFuture.channel().closeFuture().sync(); + LOG.info("FileServer exiting"); + } catch (InterruptedException e) { + LOG.error("File server start got interrupted", e); } - }; - blockingThread.setDaemon(true); - blockingThread.start(); - } finally { - bootstrap.shutdown(); - } + // NOTE: bootstrap is shutdown in stop() + } + }; + blockingThread.setDaemon(true); + blockingThread.start(); } public int getPort() { @@ -67,17 +71,16 @@ class FileServer { } public void stop() { - if (blockingThread != null) { - blockingThread.stop(); - blockingThread = null; - } + // Close the bound channel. if (channelFuture != null) { - channelFuture.channel().closeFuture(); + channelFuture.channel().close(); channelFuture = null; } + // Shutdown bootstrap. if (bootstrap != null) { bootstrap.shutdown(); bootstrap = null; } + // TODO: Shutdown all accepted channels as well ? } } diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index fdee7ca384..a4fe14b9ae 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -305,9 +305,20 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { assert(c.partitioner.get === p) } + test("shuffle local cluster") { + // Use a local cluster with 2 processes to make sure there are both local and remote blocks + sc = new SparkContext("local-cluster[2,1,512]", "test") + val a = sc.parallelize(1 to 10, 2) + val b = a.map { + x => (x, x * 2) + } + val c = new ShuffledRDD(b, new HashPartitioner(3)) + assert(c.count === 10) + } + test("shuffle serializer") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks - sc = new SparkContext("local-cluster[1,2,512]", "test") + sc = new SparkContext("local-cluster[2,1,512]", "test") val a = sc.parallelize(1 to 10, 2) val b = a.map { x => (x, new ShuffleSuite.NonJavaSerializableClass(x * 2)) @@ -317,6 +328,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { val c = new ShuffledRDD(b, new HashPartitioner(3), classOf[spark.KryoSerializer].getName) assert(c.count === 10) } + } object ShuffleSuite { -- cgit v1.2.3 From b79b10a6d60a7f1f199e6bddd1243a05c57526ad Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 29 May 2013 00:52:55 -0700 Subject: Flush serializer to fix zero-size kryo blocks bug. Also convert the local-cluster test case to check for non-zero block sizes --- core/src/main/scala/spark/storage/DiskStore.scala | 2 ++ core/src/test/scala/spark/ShuffleSuite.scala | 22 +++++++++++++++++----- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 57d4dafefc..1829c2f92e 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -59,6 +59,8 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) // Flush the partial writes, and set valid length to be the length of the entire file. // Return the number of bytes written for this commit. override def commit(): Long = { + // NOTE: Flush the serializer first and then the compressed/buffered output stream + objOut.flush() bs.flush() val prevPos = lastValidPosition lastValidPosition = channel.position() diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index a4fe14b9ae..271f4a4e44 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -305,15 +305,27 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { assert(c.partitioner.get === p) } - test("shuffle local cluster") { - // Use a local cluster with 2 processes to make sure there are both local and remote blocks + test("shuffle non-zero block size") { sc = new SparkContext("local-cluster[2,1,512]", "test") + val NUM_BLOCKS = 3 + val a = sc.parallelize(1 to 10, 2) - val b = a.map { - x => (x, x * 2) + val b = a.map { x => + (x, new ShuffleSuite.NonJavaSerializableClass(x * 2)) } - val c = new ShuffledRDD(b, new HashPartitioner(3)) + // If the Kryo serializer is not used correctly, the shuffle would fail because the + // default Java serializer cannot handle the non serializable class. + val c = new ShuffledRDD(b, new HashPartitioner(NUM_BLOCKS), + classOf[spark.KryoSerializer].getName) + val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId + assert(c.count === 10) + + // All blocks must have non-zero size + (0 until NUM_BLOCKS).foreach { id => + val statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, id) + assert(statuses.forall(s => s._2 > 0)) + } } test("shuffle serializer") { -- cgit v1.2.3 From 618c8cae1ee5dede98824823e00f7863571c0e57 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 29 May 2013 13:09:58 -0700 Subject: Skip fetching zero-sized blocks in OIO. Also unify splitLocalRemoteBlocks for netty/nio and add a test case --- .../scala/spark/storage/BlockFetcherIterator.scala | 61 +++++----------------- core/src/test/scala/spark/ShuffleSuite.scala | 27 ++++++++++ 2 files changed, 39 insertions(+), 49 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index 95308c7282..1d69d658f7 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -124,6 +124,7 @@ object BlockFetcherIterator { protected def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { // Split local and remote blocks. Remote blocks are further split into FetchRequests of size // at most maxBytesInFlight in order to limit the amount of data in flight. + val originalTotalBlocks = _totalBlocks val remoteRequests = new ArrayBuffer[FetchRequest] for ((address, blockInfos) <- blocksByAddress) { if (address == blockManagerId) { @@ -140,8 +141,15 @@ object BlockFetcherIterator { var curBlocks = new ArrayBuffer[(String, Long)] while (iterator.hasNext) { val (blockId, size) = iterator.next() - curBlocks += ((blockId, size)) - curRequestSize += size + // Skip empty blocks + if (size > 0) { + curBlocks += ((blockId, size)) + curRequestSize += size + } else if (size == 0) { + _totalBlocks -= 1 + } else { + throw new BlockException(blockId, "Negative block size " + size) + } if (curRequestSize >= minRequestSize) { // Add this FetchRequest remoteRequests += new FetchRequest(address, curBlocks) @@ -155,6 +163,8 @@ object BlockFetcherIterator { } } } + logInfo("Getting " + _totalBlocks + " non-zero-bytes blocks out of " + + originalTotalBlocks + " blocks") remoteRequests } @@ -278,53 +288,6 @@ object BlockFetcherIterator { logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.host ) } - override protected def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { - // Split local and remote blocks. Remote blocks are further split into FetchRequests of size - // at most maxBytesInFlight in order to limit the amount of data in flight. - val originalTotalBlocks = _totalBlocks; - val remoteRequests = new ArrayBuffer[FetchRequest] - for ((address, blockInfos) <- blocksByAddress) { - if (address == blockManagerId) { - localBlockIds ++= blockInfos.map(_._1) - } else { - remoteBlockIds ++= blockInfos.map(_._1) - // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them - // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 - // nodes, rather than blocking on reading output from one node. - val minRequestSize = math.max(maxBytesInFlight / 5, 1L) - logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize) - val iterator = blockInfos.iterator - var curRequestSize = 0L - var curBlocks = new ArrayBuffer[(String, Long)] - while (iterator.hasNext) { - val (blockId, size) = iterator.next() - if (size > 0) { - curBlocks += ((blockId, size)) - curRequestSize += size - } else if (size == 0) { - //here we changes the totalBlocks - _totalBlocks -= 1 - } else { - throw new BlockException(blockId, "Negative block size " + size) - } - if (curRequestSize >= minRequestSize) { - // Add this FetchRequest - remoteRequests += new FetchRequest(address, curBlocks) - curRequestSize = 0 - curBlocks = new ArrayBuffer[(String, Long)] - } - } - // Add in the final request - if (!curBlocks.isEmpty) { - remoteRequests += new FetchRequest(address, curBlocks) - } - } - } - logInfo("Getting " + _totalBlocks + " non-zero-bytes blocks out of " + - originalTotalBlocks + " blocks") - remoteRequests - } - private var copiers: List[_ <: Thread] = null override def initialize() { diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index fdee7ca384..4e50ae2ca9 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -317,6 +317,33 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { val c = new ShuffledRDD(b, new HashPartitioner(3), classOf[spark.KryoSerializer].getName) assert(c.count === 10) } + + test("zero sized blocks") { + // Use a local cluster with 2 processes to make sure there are both local and remote blocks + sc = new SparkContext("local-cluster[2,1,512]", "test") + + // 10 partitions from 4 keys + val NUM_BLOCKS = 10 + val a = sc.parallelize(1 to 4, NUM_BLOCKS) + val b = a.map(x => (x, x*2)) + + // NOTE: The default Java serializer doesn't create zero-sized blocks. + // So, use Kryo + val c = new ShuffledRDD(b, new HashPartitioner(10), classOf[spark.KryoSerializer].getName) + + val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId + assert(c.count === 4) + + val blockSizes = (0 until NUM_BLOCKS).flatMap { id => + val statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, id) + statuses.map(x => x._2) + } + val nonEmptyBlocks = blockSizes.filter(x => x > 0) + + // We should have at most 4 non-zero sized partitions + assert(nonEmptyBlocks.size <= 4) + } + } object ShuffleSuite { -- cgit v1.2.3 From 19fd6d54c012bd9f73620e9b817f4975de162277 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 29 May 2013 17:29:34 -0700 Subject: Also flush serializer in revertPartialWrites --- core/src/main/scala/spark/storage/DiskStore.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 1829c2f92e..c7281200e7 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -70,6 +70,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) override def revertPartialWrites() { // Discard current writes. We do this by flushing the outstanding writes and // truncate the file to the last valid position. + objOut.flush() bs.flush() channel.truncate(lastValidPosition) } -- cgit v1.2.3 From ecceb101d3019ef511c42a8a8a3bb0e46520ffef Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Thu, 30 May 2013 10:43:01 +0800 Subject: implement FIFO and fair scheduler for spark local mode --- .../spark/scheduler/cluster/ClusterScheduler.scala | 2 +- .../scheduler/cluster/ClusterTaskSetManager.scala | 734 +++++++++++++++++++++ .../spark/scheduler/cluster/TaskSetManager.scala | 733 +------------------- .../spark/scheduler/local/LocalScheduler.scala | 386 +++++++++-- .../spark/scheduler/ClusterSchedulerSuite.scala | 2 +- 5 files changed, 1057 insertions(+), 800 deletions(-) create mode 100644 core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 053d4b8e4a..3a0c29b27f 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -177,7 +177,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) val tasks = taskSet.tasks logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks") this.synchronized { - val manager = new TaskSetManager(this, taskSet) + val manager = new ClusterTaskSetManager(this, taskSet) activeTaskSets(taskSet.id) = manager schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) taskSetTaskIds(taskSet.id) = new HashSet[Long]() diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala new file mode 100644 index 0000000000..ec4041ab86 --- /dev/null +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -0,0 +1,734 @@ +package spark.scheduler.cluster + +import java.util.{HashMap => JHashMap, NoSuchElementException, Arrays} + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet +import scala.math.max +import scala.math.min + +import spark._ +import spark.scheduler._ +import spark.TaskState.TaskState +import java.nio.ByteBuffer + +private[spark] object TaskLocality extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY") with Logging { + + // process local is expected to be used ONLY within tasksetmanager for now. + val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value + + type TaskLocality = Value + + def isAllowed(constraint: TaskLocality, condition: TaskLocality): Boolean = { + + // Must not be the constraint. + assert (constraint != TaskLocality.PROCESS_LOCAL) + + constraint match { + case TaskLocality.NODE_LOCAL => condition == TaskLocality.NODE_LOCAL + case TaskLocality.RACK_LOCAL => condition == TaskLocality.NODE_LOCAL || condition == TaskLocality.RACK_LOCAL + // For anything else, allow + case _ => true + } + } + + def parse(str: String): TaskLocality = { + // better way to do this ? + try { + val retval = TaskLocality.withName(str) + // Must not specify PROCESS_LOCAL ! + assert (retval != TaskLocality.PROCESS_LOCAL) + + retval + } catch { + case nEx: NoSuchElementException => { + logWarning("Invalid task locality specified '" + str + "', defaulting to NODE_LOCAL"); + // default to preserve earlier behavior + NODE_LOCAL + } + } + } +} + +/** + * Schedules the tasks within a single TaskSet in the ClusterScheduler. + */ +private[spark] class ClusterTaskSetManager( + sched: ClusterScheduler, + val taskSet: TaskSet) + extends TaskSetManager + with Logging { + + // Maximum time to wait to run a task in a preferred location (in ms) + val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong + + // CPUs to request per task + val CPUS_PER_TASK = System.getProperty("spark.task.cpus", "1").toDouble + + // Maximum times a task is allowed to fail before failing the job + val MAX_TASK_FAILURES = 4 + + // Quantile of tasks at which to start speculation + val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble + val SPECULATION_MULTIPLIER = System.getProperty("spark.speculation.multiplier", "1.5").toDouble + + // Serializer for closures and tasks. + val ser = SparkEnv.get.closureSerializer.newInstance() + + val tasks = taskSet.tasks + val numTasks = tasks.length + val copiesRunning = new Array[Int](numTasks) + val finished = new Array[Boolean](numTasks) + val numFailures = new Array[Int](numTasks) + val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) + var tasksFinished = 0 + + var weight = 1 + var minShare = 0 + var runningTasks = 0 + var priority = taskSet.priority + var stageId = taskSet.stageId + var name = "TaskSet_"+taskSet.stageId.toString + var parent:Schedulable = null + + // Last time when we launched a preferred task (for delay scheduling) + var lastPreferredLaunchTime = System.currentTimeMillis + + // List of pending tasks for each node (process local to container). These collections are actually + // treated as stacks, in which new tasks are added to the end of the + // ArrayBuffer and removed from the end. This makes it faster to detect + // tasks that repeatedly fail because whenever a task failed, it is put + // back at the head of the stack. They are also only cleaned up lazily; + // when a task is launched, it remains in all the pending lists except + // the one that it was launched from, but gets removed from them later. + private val pendingTasksForHostPort = new HashMap[String, ArrayBuffer[Int]] + + // List of pending tasks for each node. + // Essentially, similar to pendingTasksForHostPort, except at host level + private val pendingTasksForHost = new HashMap[String, ArrayBuffer[Int]] + + // List of pending tasks for each node based on rack locality. + // Essentially, similar to pendingTasksForHost, except at rack level + private val pendingRackLocalTasksForHost = new HashMap[String, ArrayBuffer[Int]] + + // List containing pending tasks with no locality preferences + val pendingTasksWithNoPrefs = new ArrayBuffer[Int] + + // List containing all pending tasks (also used as a stack, as above) + val allPendingTasks = new ArrayBuffer[Int] + + // Tasks that can be speculated. Since these will be a small fraction of total + // tasks, we'll just hold them in a HashSet. + val speculatableTasks = new HashSet[Int] + + // Task index, start and finish time for each task attempt (indexed by task ID) + val taskInfos = new HashMap[Long, TaskInfo] + + // Did the job fail? + var failed = false + var causeOfFailure = "" + + // How frequently to reprint duplicate exceptions in full, in milliseconds + val EXCEPTION_PRINT_INTERVAL = + System.getProperty("spark.logging.exceptionPrintInterval", "10000").toLong + // Map of recent exceptions (identified by string representation and + // top stack frame) to duplicate count (how many times the same + // exception has appeared) and time the full exception was + // printed. This should ideally be an LRU map that can drop old + // exceptions automatically. + val recentExceptions = HashMap[String, (Int, Long)]() + + // Figure out the current map output tracker generation and set it on all tasks + val generation = sched.mapOutputTracker.getGeneration + logDebug("Generation for " + taskSet.id + ": " + generation) + for (t <- tasks) { + t.generation = generation + } + + // Add all our tasks to the pending lists. We do this in reverse order + // of task index so that tasks with low indices get launched first. + for (i <- (0 until numTasks).reverse) { + addPendingTask(i) + } + + // Note that it follows the hierarchy. + // if we search for NODE_LOCAL, the output will include PROCESS_LOCAL and + // if we search for RACK_LOCAL, it will include PROCESS_LOCAL & NODE_LOCAL + private def findPreferredLocations(_taskPreferredLocations: Seq[String], scheduler: ClusterScheduler, + taskLocality: TaskLocality.TaskLocality): HashSet[String] = { + + if (TaskLocality.PROCESS_LOCAL == taskLocality) { + // straight forward comparison ! Special case it. + val retval = new HashSet[String]() + scheduler.synchronized { + for (location <- _taskPreferredLocations) { + if (scheduler.isExecutorAliveOnHostPort(location)) { + retval += location + } + } + } + + return retval + } + + val taskPreferredLocations = + if (TaskLocality.NODE_LOCAL == taskLocality) { + _taskPreferredLocations + } else { + assert (TaskLocality.RACK_LOCAL == taskLocality) + // Expand set to include all 'seen' rack local hosts. + // This works since container allocation/management happens within master - so any rack locality information is updated in msater. + // Best case effort, and maybe sort of kludge for now ... rework it later ? + val hosts = new HashSet[String] + _taskPreferredLocations.foreach(h => { + val rackOpt = scheduler.getRackForHost(h) + if (rackOpt.isDefined) { + val hostsOpt = scheduler.getCachedHostsForRack(rackOpt.get) + if (hostsOpt.isDefined) { + hosts ++= hostsOpt.get + } + } + + // Ensure that irrespective of what scheduler says, host is always added ! + hosts += h + }) + + hosts + } + + val retval = new HashSet[String] + scheduler.synchronized { + for (prefLocation <- taskPreferredLocations) { + val aliveLocationsOpt = scheduler.getExecutorsAliveOnHost(Utils.parseHostPort(prefLocation)._1) + if (aliveLocationsOpt.isDefined) { + retval ++= aliveLocationsOpt.get + } + } + } + + retval + } + + // Add a task to all the pending-task lists that it should be on. + private def addPendingTask(index: Int) { + // We can infer hostLocalLocations from rackLocalLocations by joining it against tasks(index).preferredLocations (with appropriate + // hostPort <-> host conversion). But not doing it for simplicity sake. If this becomes a performance issue, modify it. + val processLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.PROCESS_LOCAL) + val hostLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) + val rackLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) + + if (rackLocalLocations.size == 0) { + // Current impl ensures this. + assert (processLocalLocations.size == 0) + assert (hostLocalLocations.size == 0) + pendingTasksWithNoPrefs += index + } else { + + // process local locality + for (hostPort <- processLocalLocations) { + // DEBUG Code + Utils.checkHostPort(hostPort) + + val hostPortList = pendingTasksForHostPort.getOrElseUpdate(hostPort, ArrayBuffer()) + hostPortList += index + } + + // host locality (includes process local) + for (hostPort <- hostLocalLocations) { + // DEBUG Code + Utils.checkHostPort(hostPort) + + val host = Utils.parseHostPort(hostPort)._1 + val hostList = pendingTasksForHost.getOrElseUpdate(host, ArrayBuffer()) + hostList += index + } + + // rack locality (includes process local and host local) + for (rackLocalHostPort <- rackLocalLocations) { + // DEBUG Code + Utils.checkHostPort(rackLocalHostPort) + + val rackLocalHost = Utils.parseHostPort(rackLocalHostPort)._1 + val list = pendingRackLocalTasksForHost.getOrElseUpdate(rackLocalHost, ArrayBuffer()) + list += index + } + } + + allPendingTasks += index + } + + // Return the pending tasks list for a given host port (process local), or an empty list if + // there is no map entry for that host + private def getPendingTasksForHostPort(hostPort: String): ArrayBuffer[Int] = { + // DEBUG Code + Utils.checkHostPort(hostPort) + pendingTasksForHostPort.getOrElse(hostPort, ArrayBuffer()) + } + + // Return the pending tasks list for a given host, or an empty list if + // there is no map entry for that host + private def getPendingTasksForHost(hostPort: String): ArrayBuffer[Int] = { + val host = Utils.parseHostPort(hostPort)._1 + pendingTasksForHost.getOrElse(host, ArrayBuffer()) + } + + // Return the pending tasks (rack level) list for a given host, or an empty list if + // there is no map entry for that host + private def getRackLocalPendingTasksForHost(hostPort: String): ArrayBuffer[Int] = { + val host = Utils.parseHostPort(hostPort)._1 + pendingRackLocalTasksForHost.getOrElse(host, ArrayBuffer()) + } + + // Number of pending tasks for a given host Port (which would be process local) + def numPendingTasksForHostPort(hostPort: String): Int = { + getPendingTasksForHostPort(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) + } + + // Number of pending tasks for a given host (which would be data local) + def numPendingTasksForHost(hostPort: String): Int = { + getPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) + } + + // Number of pending rack local tasks for a given host + def numRackLocalPendingTasksForHost(hostPort: String): Int = { + getRackLocalPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) + } + + + // Dequeue a pending task from the given list and return its index. + // Return None if the list is empty. + // This method also cleans up any tasks in the list that have already + // been launched, since we want that to happen lazily. + private def findTaskFromList(list: ArrayBuffer[Int]): Option[Int] = { + while (!list.isEmpty) { + val index = list.last + list.trimEnd(1) + if (copiesRunning(index) == 0 && !finished(index)) { + return Some(index) + } + } + return None + } + + // Return a speculative task for a given host if any are available. The task should not have an + // attempt running on this host, in case the host is slow. In addition, if locality is set, the + // task must have a preference for this host/rack/no preferred locations at all. + private def findSpeculativeTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = { + + assert (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) + speculatableTasks.retain(index => !finished(index)) // Remove finished tasks from set + + if (speculatableTasks.size > 0) { + val localTask = speculatableTasks.find { + index => + val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) + val attemptLocs = taskAttempts(index).map(_.hostPort) + (locations.size == 0 || locations.contains(hostPort)) && !attemptLocs.contains(hostPort) + } + + if (localTask != None) { + speculatableTasks -= localTask.get + return localTask + } + + // check for rack locality + if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { + val rackTask = speculatableTasks.find { + index => + val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) + val attemptLocs = taskAttempts(index).map(_.hostPort) + locations.contains(hostPort) && !attemptLocs.contains(hostPort) + } + + if (rackTask != None) { + speculatableTasks -= rackTask.get + return rackTask + } + } + + // Any task ... + if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { + // Check for attemptLocs also ? + val nonLocalTask = speculatableTasks.find(i => !taskAttempts(i).map(_.hostPort).contains(hostPort)) + if (nonLocalTask != None) { + speculatableTasks -= nonLocalTask.get + return nonLocalTask + } + } + } + return None + } + + // Dequeue a pending task for a given node and return its index. + // If localOnly is set to false, allow non-local tasks as well. + private def findTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = { + val processLocalTask = findTaskFromList(getPendingTasksForHostPort(hostPort)) + if (processLocalTask != None) { + return processLocalTask + } + + val localTask = findTaskFromList(getPendingTasksForHost(hostPort)) + if (localTask != None) { + return localTask + } + + if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { + val rackLocalTask = findTaskFromList(getRackLocalPendingTasksForHost(hostPort)) + if (rackLocalTask != None) { + return rackLocalTask + } + } + + // Look for no pref tasks AFTER rack local tasks - this has side effect that we will get to failed tasks later rather than sooner. + // TODO: That code path needs to be revisited (adding to no prefs list when host:port goes down). + val noPrefTask = findTaskFromList(pendingTasksWithNoPrefs) + if (noPrefTask != None) { + return noPrefTask + } + + if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { + val nonLocalTask = findTaskFromList(allPendingTasks) + if (nonLocalTask != None) { + return nonLocalTask + } + } + + // Finally, if all else has failed, find a speculative task + return findSpeculativeTask(hostPort, locality) + } + + private def isProcessLocalLocation(task: Task[_], hostPort: String): Boolean = { + Utils.checkHostPort(hostPort) + + val locs = task.preferredLocations + + locs.contains(hostPort) + } + + private def isHostLocalLocation(task: Task[_], hostPort: String): Boolean = { + val locs = task.preferredLocations + + // If no preference, consider it as host local + if (locs.isEmpty) return true + + val host = Utils.parseHostPort(hostPort)._1 + locs.find(h => Utils.parseHostPort(h)._1 == host).isDefined + } + + // Does a host count as a rack local preferred location for a task? (assumes host is NOT preferred location). + // This is true if either the task has preferred locations and this host is one, or it has + // no preferred locations (in which we still count the launch as preferred). + private def isRackLocalLocation(task: Task[_], hostPort: String): Boolean = { + + val locs = task.preferredLocations + + val preferredRacks = new HashSet[String]() + for (preferredHost <- locs) { + val rack = sched.getRackForHost(preferredHost) + if (None != rack) preferredRacks += rack.get + } + + if (preferredRacks.isEmpty) return false + + val hostRack = sched.getRackForHost(hostPort) + + return None != hostRack && preferredRacks.contains(hostRack.get) + } + + // Respond to an offer of a single slave from the scheduler by finding a task + def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = { + + if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { + // If explicitly specified, use that + val locality = if (overrideLocality != null) overrideLocality else { + // expand only if we have waited for more than LOCALITY_WAIT for a host local task ... + val time = System.currentTimeMillis + if (time - lastPreferredLaunchTime < LOCALITY_WAIT) TaskLocality.NODE_LOCAL else TaskLocality.ANY + } + + findTask(hostPort, locality) match { + case Some(index) => { + // Found a task; do some bookkeeping and return a Mesos task for it + val task = tasks(index) + val taskId = sched.newTaskId() + // Figure out whether this should count as a preferred launch + val taskLocality = + if (isProcessLocalLocation(task, hostPort)) TaskLocality.PROCESS_LOCAL else + if (isHostLocalLocation(task, hostPort)) TaskLocality.NODE_LOCAL else + if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL else + TaskLocality.ANY + val prefStr = taskLocality.toString + logInfo("Starting task %s:%d as TID %s on slave %s: %s (%s)".format( + taskSet.id, index, taskId, execId, hostPort, prefStr)) + // Do various bookkeeping + copiesRunning(index) += 1 + val time = System.currentTimeMillis + val info = new TaskInfo(taskId, index, time, execId, hostPort, taskLocality) + taskInfos(taskId) = info + taskAttempts(index) = info :: taskAttempts(index) + if (TaskLocality.NODE_LOCAL == taskLocality) { + lastPreferredLaunchTime = time + } + // Serialize and return the task + val startTime = System.currentTimeMillis + val serializedTask = Task.serializeWithDependencies( + task, sched.sc.addedFiles, sched.sc.addedJars, ser) + val timeTaken = System.currentTimeMillis - startTime + increaseRunningTasks(1) + logInfo("Serialized task %s:%d as %d bytes in %d ms".format( + taskSet.id, index, serializedTask.limit, timeTaken)) + val taskName = "task %s:%d".format(taskSet.id, index) + return Some(new TaskDescription(taskId, execId, taskName, serializedTask)) + } + case _ => + } + } + return None + } + + def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { + state match { + case TaskState.FINISHED => + taskFinished(tid, state, serializedData) + case TaskState.LOST => + taskLost(tid, state, serializedData) + case TaskState.FAILED => + taskLost(tid, state, serializedData) + case TaskState.KILLED => + taskLost(tid, state, serializedData) + case _ => + } + } + + def taskFinished(tid: Long, state: TaskState, serializedData: ByteBuffer) { + val info = taskInfos(tid) + if (info.failed) { + // We might get two task-lost messages for the same task in coarse-grained Mesos mode, + // or even from Mesos itself when acks get delayed. + return + } + val index = info.index + info.markSuccessful() + decreaseRunningTasks(1) + if (!finished(index)) { + tasksFinished += 1 + logInfo("Finished TID %s in %d ms (progress: %d/%d)".format( + tid, info.duration, tasksFinished, numTasks)) + // Deserialize task result and pass it to the scheduler + val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) + result.metrics.resultSize = serializedData.limit() + sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) + // Mark finished and stop if we've finished all the tasks + finished(index) = true + if (tasksFinished == numTasks) { + sched.taskSetFinished(this) + } + } else { + logInfo("Ignoring task-finished event for TID " + tid + + " because task " + index + " is already finished") + } + } + + def taskLost(tid: Long, state: TaskState, serializedData: ByteBuffer) { + val info = taskInfos(tid) + if (info.failed) { + // We might get two task-lost messages for the same task in coarse-grained Mesos mode, + // or even from Mesos itself when acks get delayed. + return + } + val index = info.index + info.markFailed() + decreaseRunningTasks(1) + if (!finished(index)) { + logInfo("Lost TID %s (task %s:%d)".format(tid, taskSet.id, index)) + copiesRunning(index) -= 1 + // Check if the problem is a map output fetch failure. In that case, this + // task will never succeed on any node, so tell the scheduler about it. + if (serializedData != null && serializedData.limit() > 0) { + val reason = ser.deserialize[TaskEndReason](serializedData, getClass.getClassLoader) + reason match { + case fetchFailed: FetchFailed => + logInfo("Loss was due to fetch failure from " + fetchFailed.bmAddress) + sched.listener.taskEnded(tasks(index), fetchFailed, null, null, info, null) + finished(index) = true + tasksFinished += 1 + sched.taskSetFinished(this) + decreaseRunningTasks(runningTasks) + return + + case ef: ExceptionFailure => + val key = ef.description + val now = System.currentTimeMillis + val (printFull, dupCount) = { + if (recentExceptions.contains(key)) { + val (dupCount, printTime) = recentExceptions(key) + if (now - printTime > EXCEPTION_PRINT_INTERVAL) { + recentExceptions(key) = (0, now) + (true, 0) + } else { + recentExceptions(key) = (dupCount + 1, printTime) + (false, dupCount + 1) + } + } else { + recentExceptions(key) = (0, now) + (true, 0) + } + } + if (printFull) { + val locs = ef.stackTrace.map(loc => "\tat %s".format(loc.toString)) + logInfo("Loss was due to %s\n%s\n%s".format( + ef.className, ef.description, locs.mkString("\n"))) + } else { + logInfo("Loss was due to %s [duplicate %d]".format(ef.description, dupCount)) + } + + case _ => {} + } + } + // On non-fetch failures, re-enqueue the task as pending for a max number of retries + addPendingTask(index) + // Count failed attempts only on FAILED and LOST state (not on KILLED) + if (state == TaskState.FAILED || state == TaskState.LOST) { + numFailures(index) += 1 + if (numFailures(index) > MAX_TASK_FAILURES) { + logError("Task %s:%d failed more than %d times; aborting job".format( + taskSet.id, index, MAX_TASK_FAILURES)) + abort("Task %s:%d failed more than %d times".format(taskSet.id, index, MAX_TASK_FAILURES)) + } + } + } else { + logInfo("Ignoring task-lost event for TID " + tid + + " because task " + index + " is already finished") + } + } + + def error(message: String) { + // Save the error message + abort("Error: " + message) + } + + def abort(message: String) { + failed = true + causeOfFailure = message + // TODO: Kill running tasks if we were not terminated due to a Mesos error + sched.listener.taskSetFailed(taskSet, message) + decreaseRunningTasks(runningTasks) + sched.taskSetFinished(this) + } + + override def increaseRunningTasks(taskNum: Int) { + runningTasks += taskNum + if (parent != null) { + parent.increaseRunningTasks(taskNum) + } + } + + override def decreaseRunningTasks(taskNum: Int) { + runningTasks -= taskNum + if (parent != null) { + parent.decreaseRunningTasks(taskNum) + } + } + + //TODO: for now we just find Pool not TaskSetManager, we can extend this function in future if needed + override def getSchedulableByName(name: String): Schedulable = { + return null + } + + override def addSchedulable(schedulable:Schedulable) { + //nothing + } + + override def removeSchedulable(schedulable:Schedulable) { + //nothing + } + + override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { + var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] + sortedTaskSetQueue += this + return sortedTaskSetQueue + } + + override def executorLost(execId: String, hostPort: String) { + logInfo("Re-queueing tasks for " + execId + " from TaskSet " + taskSet.id) + + // If some task has preferred locations only on hostname, and there are no more executors there, + // put it in the no-prefs list to avoid the wait from delay scheduling + + // host local tasks - should we push this to rack local or no pref list ? For now, preserving behavior and moving to + // no prefs list. Note, this was done due to impliations related to 'waiting' for data local tasks, etc. + // Note: NOT checking process local list - since host local list is super set of that. We need to ad to no prefs only if + // there is no host local node for the task (not if there is no process local node for the task) + for (index <- getPendingTasksForHost(Utils.parseHostPort(hostPort)._1)) { + // val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) + val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) + if (newLocs.isEmpty) { + pendingTasksWithNoPrefs += index + } + } + + // Re-enqueue any tasks that ran on the failed executor if this is a shuffle map stage + if (tasks(0).isInstanceOf[ShuffleMapTask]) { + for ((tid, info) <- taskInfos if info.executorId == execId) { + val index = taskInfos(tid).index + if (finished(index)) { + finished(index) = false + copiesRunning(index) -= 1 + tasksFinished -= 1 + addPendingTask(index) + // Tell the DAGScheduler that this task was resubmitted so that it doesn't think our + // stage finishes when a total of tasks.size tasks finish. + sched.listener.taskEnded(tasks(index), Resubmitted, null, null, info, null) + } + } + } + // Also re-enqueue any tasks that were running on the node + for ((tid, info) <- taskInfos if info.running && info.executorId == execId) { + taskLost(tid, TaskState.KILLED, null) + } + } + + /** + * Check for tasks to be speculated and return true if there are any. This is called periodically + * by the ClusterScheduler. + * + * TODO: To make this scale to large jobs, we need to maintain a list of running tasks, so that + * we don't scan the whole task set. It might also help to make this sorted by launch time. + */ + override def checkSpeculatableTasks(): Boolean = { + // Can't speculate if we only have one task, or if all tasks have finished. + if (numTasks == 1 || tasksFinished == numTasks) { + return false + } + var foundTasks = false + val minFinishedForSpeculation = (SPECULATION_QUANTILE * numTasks).floor.toInt + logDebug("Checking for speculative tasks: minFinished = " + minFinishedForSpeculation) + if (tasksFinished >= minFinishedForSpeculation) { + val time = System.currentTimeMillis() + val durations = taskInfos.values.filter(_.successful).map(_.duration).toArray + Arrays.sort(durations) + val medianDuration = durations(min((0.5 * numTasks).round.toInt, durations.size - 1)) + val threshold = max(SPECULATION_MULTIPLIER * medianDuration, 100) + // TODO: Threshold should also look at standard deviation of task durations and have a lower + // bound based on that. + logDebug("Task length threshold for speculation: " + threshold) + for ((tid, info) <- taskInfos) { + val index = info.index + if (!finished(index) && copiesRunning(index) == 1 && info.timeRunning(time) > threshold && + !speculatableTasks.contains(index)) { + logInfo( + "Marking task %s:%d (on %s) as speculatable because it ran more than %.0f ms".format( + taskSet.id, index, info.hostPort, threshold)) + speculatableTasks += index + foundTasks = true + } + } + } + return foundTasks + } + + override def hasPendingTasks(): Boolean = { + numTasks > 0 && tasksFinished < numTasks + } +} diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 1c403ef323..2b5a74d4e5 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -1,734 +1,17 @@ package spark.scheduler.cluster -import java.util.{HashMap => JHashMap, NoSuchElementException, Arrays} - import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet -import scala.math.max -import scala.math.min - -import spark._ import spark.scheduler._ import spark.TaskState.TaskState import java.nio.ByteBuffer - -private[spark] object TaskLocality extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY") with Logging { - - // process local is expected to be used ONLY within tasksetmanager for now. - val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value - - type TaskLocality = Value - - def isAllowed(constraint: TaskLocality, condition: TaskLocality): Boolean = { - - // Must not be the constraint. - assert (constraint != TaskLocality.PROCESS_LOCAL) - - constraint match { - case TaskLocality.NODE_LOCAL => condition == TaskLocality.NODE_LOCAL - case TaskLocality.RACK_LOCAL => condition == TaskLocality.NODE_LOCAL || condition == TaskLocality.RACK_LOCAL - // For anything else, allow - case _ => true - } - } - - def parse(str: String): TaskLocality = { - // better way to do this ? - try { - val retval = TaskLocality.withName(str) - // Must not specify PROCESS_LOCAL ! - assert (retval != TaskLocality.PROCESS_LOCAL) - - retval - } catch { - case nEx: NoSuchElementException => { - logWarning("Invalid task locality specified '" + str + "', defaulting to NODE_LOCAL"); - // default to preserve earlier behavior - NODE_LOCAL - } - } - } -} - /** - * Schedules the tasks within a single TaskSet in the ClusterScheduler. */ -private[spark] class TaskSetManager( - sched: ClusterScheduler, - val taskSet: TaskSet) - extends Schedulable - with Logging { - - // Maximum time to wait to run a task in a preferred location (in ms) - val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong - - // CPUs to request per task - val CPUS_PER_TASK = System.getProperty("spark.task.cpus", "1").toDouble - - // Maximum times a task is allowed to fail before failing the job - val MAX_TASK_FAILURES = 4 - - // Quantile of tasks at which to start speculation - val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble - val SPECULATION_MULTIPLIER = System.getProperty("spark.speculation.multiplier", "1.5").toDouble - - // Serializer for closures and tasks. - val ser = SparkEnv.get.closureSerializer.newInstance() - - val tasks = taskSet.tasks - val numTasks = tasks.length - val copiesRunning = new Array[Int](numTasks) - val finished = new Array[Boolean](numTasks) - val numFailures = new Array[Int](numTasks) - val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) - var tasksFinished = 0 - - var weight = 1 - var minShare = 0 - var runningTasks = 0 - var priority = taskSet.priority - var stageId = taskSet.stageId - var name = "TaskSet_"+taskSet.stageId.toString - var parent:Schedulable = null - - // Last time when we launched a preferred task (for delay scheduling) - var lastPreferredLaunchTime = System.currentTimeMillis - - // List of pending tasks for each node (process local to container). These collections are actually - // treated as stacks, in which new tasks are added to the end of the - // ArrayBuffer and removed from the end. This makes it faster to detect - // tasks that repeatedly fail because whenever a task failed, it is put - // back at the head of the stack. They are also only cleaned up lazily; - // when a task is launched, it remains in all the pending lists except - // the one that it was launched from, but gets removed from them later. - private val pendingTasksForHostPort = new HashMap[String, ArrayBuffer[Int]] - - // List of pending tasks for each node. - // Essentially, similar to pendingTasksForHostPort, except at host level - private val pendingTasksForHost = new HashMap[String, ArrayBuffer[Int]] - - // List of pending tasks for each node based on rack locality. - // Essentially, similar to pendingTasksForHost, except at rack level - private val pendingRackLocalTasksForHost = new HashMap[String, ArrayBuffer[Int]] - - // List containing pending tasks with no locality preferences - val pendingTasksWithNoPrefs = new ArrayBuffer[Int] - - // List containing all pending tasks (also used as a stack, as above) - val allPendingTasks = new ArrayBuffer[Int] - - // Tasks that can be speculated. Since these will be a small fraction of total - // tasks, we'll just hold them in a HashSet. - val speculatableTasks = new HashSet[Int] - - // Task index, start and finish time for each task attempt (indexed by task ID) - val taskInfos = new HashMap[Long, TaskInfo] - - // Did the job fail? - var failed = false - var causeOfFailure = "" - - // How frequently to reprint duplicate exceptions in full, in milliseconds - val EXCEPTION_PRINT_INTERVAL = - System.getProperty("spark.logging.exceptionPrintInterval", "10000").toLong - // Map of recent exceptions (identified by string representation and - // top stack frame) to duplicate count (how many times the same - // exception has appeared) and time the full exception was - // printed. This should ideally be an LRU map that can drop old - // exceptions automatically. - val recentExceptions = HashMap[String, (Int, Long)]() - - // Figure out the current map output tracker generation and set it on all tasks - val generation = sched.mapOutputTracker.getGeneration - logDebug("Generation for " + taskSet.id + ": " + generation) - for (t <- tasks) { - t.generation = generation - } - - // Add all our tasks to the pending lists. We do this in reverse order - // of task index so that tasks with low indices get launched first. - for (i <- (0 until numTasks).reverse) { - addPendingTask(i) - } - - // Note that it follows the hierarchy. - // if we search for NODE_LOCAL, the output will include PROCESS_LOCAL and - // if we search for RACK_LOCAL, it will include PROCESS_LOCAL & NODE_LOCAL - private def findPreferredLocations(_taskPreferredLocations: Seq[String], scheduler: ClusterScheduler, - taskLocality: TaskLocality.TaskLocality): HashSet[String] = { - - if (TaskLocality.PROCESS_LOCAL == taskLocality) { - // straight forward comparison ! Special case it. - val retval = new HashSet[String]() - scheduler.synchronized { - for (location <- _taskPreferredLocations) { - if (scheduler.isExecutorAliveOnHostPort(location)) { - retval += location - } - } - } - - return retval - } - - val taskPreferredLocations = - if (TaskLocality.NODE_LOCAL == taskLocality) { - _taskPreferredLocations - } else { - assert (TaskLocality.RACK_LOCAL == taskLocality) - // Expand set to include all 'seen' rack local hosts. - // This works since container allocation/management happens within master - so any rack locality information is updated in msater. - // Best case effort, and maybe sort of kludge for now ... rework it later ? - val hosts = new HashSet[String] - _taskPreferredLocations.foreach(h => { - val rackOpt = scheduler.getRackForHost(h) - if (rackOpt.isDefined) { - val hostsOpt = scheduler.getCachedHostsForRack(rackOpt.get) - if (hostsOpt.isDefined) { - hosts ++= hostsOpt.get - } - } - - // Ensure that irrespective of what scheduler says, host is always added ! - hosts += h - }) - - hosts - } - - val retval = new HashSet[String] - scheduler.synchronized { - for (prefLocation <- taskPreferredLocations) { - val aliveLocationsOpt = scheduler.getExecutorsAliveOnHost(Utils.parseHostPort(prefLocation)._1) - if (aliveLocationsOpt.isDefined) { - retval ++= aliveLocationsOpt.get - } - } - } - - retval - } - - // Add a task to all the pending-task lists that it should be on. - private def addPendingTask(index: Int) { - // We can infer hostLocalLocations from rackLocalLocations by joining it against tasks(index).preferredLocations (with appropriate - // hostPort <-> host conversion). But not doing it for simplicity sake. If this becomes a performance issue, modify it. - val processLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.PROCESS_LOCAL) - val hostLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) - val rackLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) - - if (rackLocalLocations.size == 0) { - // Current impl ensures this. - assert (processLocalLocations.size == 0) - assert (hostLocalLocations.size == 0) - pendingTasksWithNoPrefs += index - } else { - - // process local locality - for (hostPort <- processLocalLocations) { - // DEBUG Code - Utils.checkHostPort(hostPort) - - val hostPortList = pendingTasksForHostPort.getOrElseUpdate(hostPort, ArrayBuffer()) - hostPortList += index - } - - // host locality (includes process local) - for (hostPort <- hostLocalLocations) { - // DEBUG Code - Utils.checkHostPort(hostPort) - - val host = Utils.parseHostPort(hostPort)._1 - val hostList = pendingTasksForHost.getOrElseUpdate(host, ArrayBuffer()) - hostList += index - } - - // rack locality (includes process local and host local) - for (rackLocalHostPort <- rackLocalLocations) { - // DEBUG Code - Utils.checkHostPort(rackLocalHostPort) - - val rackLocalHost = Utils.parseHostPort(rackLocalHostPort)._1 - val list = pendingRackLocalTasksForHost.getOrElseUpdate(rackLocalHost, ArrayBuffer()) - list += index - } - } - - allPendingTasks += index - } - - // Return the pending tasks list for a given host port (process local), or an empty list if - // there is no map entry for that host - private def getPendingTasksForHostPort(hostPort: String): ArrayBuffer[Int] = { - // DEBUG Code - Utils.checkHostPort(hostPort) - pendingTasksForHostPort.getOrElse(hostPort, ArrayBuffer()) - } - - // Return the pending tasks list for a given host, or an empty list if - // there is no map entry for that host - private def getPendingTasksForHost(hostPort: String): ArrayBuffer[Int] = { - val host = Utils.parseHostPort(hostPort)._1 - pendingTasksForHost.getOrElse(host, ArrayBuffer()) - } - - // Return the pending tasks (rack level) list for a given host, or an empty list if - // there is no map entry for that host - private def getRackLocalPendingTasksForHost(hostPort: String): ArrayBuffer[Int] = { - val host = Utils.parseHostPort(hostPort)._1 - pendingRackLocalTasksForHost.getOrElse(host, ArrayBuffer()) - } - - // Number of pending tasks for a given host Port (which would be process local) - def numPendingTasksForHostPort(hostPort: String): Int = { - getPendingTasksForHostPort(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) - } - - // Number of pending tasks for a given host (which would be data local) - def numPendingTasksForHost(hostPort: String): Int = { - getPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) - } - - // Number of pending rack local tasks for a given host - def numRackLocalPendingTasksForHost(hostPort: String): Int = { - getRackLocalPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) - } - - - // Dequeue a pending task from the given list and return its index. - // Return None if the list is empty. - // This method also cleans up any tasks in the list that have already - // been launched, since we want that to happen lazily. - private def findTaskFromList(list: ArrayBuffer[Int]): Option[Int] = { - while (!list.isEmpty) { - val index = list.last - list.trimEnd(1) - if (copiesRunning(index) == 0 && !finished(index)) { - return Some(index) - } - } - return None - } - - // Return a speculative task for a given host if any are available. The task should not have an - // attempt running on this host, in case the host is slow. In addition, if locality is set, the - // task must have a preference for this host/rack/no preferred locations at all. - private def findSpeculativeTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = { - - assert (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) - speculatableTasks.retain(index => !finished(index)) // Remove finished tasks from set - - if (speculatableTasks.size > 0) { - val localTask = speculatableTasks.find { - index => - val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) - val attemptLocs = taskAttempts(index).map(_.hostPort) - (locations.size == 0 || locations.contains(hostPort)) && !attemptLocs.contains(hostPort) - } - - if (localTask != None) { - speculatableTasks -= localTask.get - return localTask - } - - // check for rack locality - if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { - val rackTask = speculatableTasks.find { - index => - val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) - val attemptLocs = taskAttempts(index).map(_.hostPort) - locations.contains(hostPort) && !attemptLocs.contains(hostPort) - } - - if (rackTask != None) { - speculatableTasks -= rackTask.get - return rackTask - } - } - - // Any task ... - if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { - // Check for attemptLocs also ? - val nonLocalTask = speculatableTasks.find(i => !taskAttempts(i).map(_.hostPort).contains(hostPort)) - if (nonLocalTask != None) { - speculatableTasks -= nonLocalTask.get - return nonLocalTask - } - } - } - return None - } - - // Dequeue a pending task for a given node and return its index. - // If localOnly is set to false, allow non-local tasks as well. - private def findTask(hostPort: String, locality: TaskLocality.TaskLocality): Option[Int] = { - val processLocalTask = findTaskFromList(getPendingTasksForHostPort(hostPort)) - if (processLocalTask != None) { - return processLocalTask - } - - val localTask = findTaskFromList(getPendingTasksForHost(hostPort)) - if (localTask != None) { - return localTask - } - - if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { - val rackLocalTask = findTaskFromList(getRackLocalPendingTasksForHost(hostPort)) - if (rackLocalTask != None) { - return rackLocalTask - } - } - - // Look for no pref tasks AFTER rack local tasks - this has side effect that we will get to failed tasks later rather than sooner. - // TODO: That code path needs to be revisited (adding to no prefs list when host:port goes down). - val noPrefTask = findTaskFromList(pendingTasksWithNoPrefs) - if (noPrefTask != None) { - return noPrefTask - } - - if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { - val nonLocalTask = findTaskFromList(allPendingTasks) - if (nonLocalTask != None) { - return nonLocalTask - } - } - - // Finally, if all else has failed, find a speculative task - return findSpeculativeTask(hostPort, locality) - } - - private def isProcessLocalLocation(task: Task[_], hostPort: String): Boolean = { - Utils.checkHostPort(hostPort) - - val locs = task.preferredLocations - - locs.contains(hostPort) - } - - private def isHostLocalLocation(task: Task[_], hostPort: String): Boolean = { - val locs = task.preferredLocations - - // If no preference, consider it as host local - if (locs.isEmpty) return true - - val host = Utils.parseHostPort(hostPort)._1 - locs.find(h => Utils.parseHostPort(h)._1 == host).isDefined - } - - // Does a host count as a rack local preferred location for a task? (assumes host is NOT preferred location). - // This is true if either the task has preferred locations and this host is one, or it has - // no preferred locations (in which we still count the launch as preferred). - private def isRackLocalLocation(task: Task[_], hostPort: String): Boolean = { - - val locs = task.preferredLocations - - val preferredRacks = new HashSet[String]() - for (preferredHost <- locs) { - val rack = sched.getRackForHost(preferredHost) - if (None != rack) preferredRacks += rack.get - } - - if (preferredRacks.isEmpty) return false - - val hostRack = sched.getRackForHost(hostPort) - - return None != hostRack && preferredRacks.contains(hostRack.get) - } - - // Respond to an offer of a single slave from the scheduler by finding a task - def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = { - - if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { - // If explicitly specified, use that - val locality = if (overrideLocality != null) overrideLocality else { - // expand only if we have waited for more than LOCALITY_WAIT for a host local task ... - val time = System.currentTimeMillis - if (time - lastPreferredLaunchTime < LOCALITY_WAIT) TaskLocality.NODE_LOCAL else TaskLocality.ANY - } - - findTask(hostPort, locality) match { - case Some(index) => { - // Found a task; do some bookkeeping and return a Mesos task for it - val task = tasks(index) - val taskId = sched.newTaskId() - // Figure out whether this should count as a preferred launch - val taskLocality = - if (isProcessLocalLocation(task, hostPort)) TaskLocality.PROCESS_LOCAL else - if (isHostLocalLocation(task, hostPort)) TaskLocality.NODE_LOCAL else - if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL else - TaskLocality.ANY - val prefStr = taskLocality.toString - logInfo("Starting task %s:%d as TID %s on slave %s: %s (%s)".format( - taskSet.id, index, taskId, execId, hostPort, prefStr)) - // Do various bookkeeping - copiesRunning(index) += 1 - val time = System.currentTimeMillis - val info = new TaskInfo(taskId, index, time, execId, hostPort, taskLocality) - taskInfos(taskId) = info - taskAttempts(index) = info :: taskAttempts(index) - if (TaskLocality.NODE_LOCAL == taskLocality) { - lastPreferredLaunchTime = time - } - // Serialize and return the task - val startTime = System.currentTimeMillis - val serializedTask = Task.serializeWithDependencies( - task, sched.sc.addedFiles, sched.sc.addedJars, ser) - val timeTaken = System.currentTimeMillis - startTime - increaseRunningTasks(1) - logInfo("Serialized task %s:%d as %d bytes in %d ms".format( - taskSet.id, index, serializedTask.limit, timeTaken)) - val taskName = "task %s:%d".format(taskSet.id, index) - return Some(new TaskDescription(taskId, execId, taskName, serializedTask)) - } - case _ => - } - } - return None - } - - def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { - state match { - case TaskState.FINISHED => - taskFinished(tid, state, serializedData) - case TaskState.LOST => - taskLost(tid, state, serializedData) - case TaskState.FAILED => - taskLost(tid, state, serializedData) - case TaskState.KILLED => - taskLost(tid, state, serializedData) - case _ => - } - } - - def taskFinished(tid: Long, state: TaskState, serializedData: ByteBuffer) { - val info = taskInfos(tid) - if (info.failed) { - // We might get two task-lost messages for the same task in coarse-grained Mesos mode, - // or even from Mesos itself when acks get delayed. - return - } - val index = info.index - info.markSuccessful() - decreaseRunningTasks(1) - if (!finished(index)) { - tasksFinished += 1 - logInfo("Finished TID %s in %d ms (progress: %d/%d)".format( - tid, info.duration, tasksFinished, numTasks)) - // Deserialize task result and pass it to the scheduler - val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) - result.metrics.resultSize = serializedData.limit() - sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) - // Mark finished and stop if we've finished all the tasks - finished(index) = true - if (tasksFinished == numTasks) { - sched.taskSetFinished(this) - } - } else { - logInfo("Ignoring task-finished event for TID " + tid + - " because task " + index + " is already finished") - } - } - - def taskLost(tid: Long, state: TaskState, serializedData: ByteBuffer) { - val info = taskInfos(tid) - if (info.failed) { - // We might get two task-lost messages for the same task in coarse-grained Mesos mode, - // or even from Mesos itself when acks get delayed. - return - } - val index = info.index - info.markFailed() - decreaseRunningTasks(1) - if (!finished(index)) { - logInfo("Lost TID %s (task %s:%d)".format(tid, taskSet.id, index)) - copiesRunning(index) -= 1 - // Check if the problem is a map output fetch failure. In that case, this - // task will never succeed on any node, so tell the scheduler about it. - if (serializedData != null && serializedData.limit() > 0) { - val reason = ser.deserialize[TaskEndReason](serializedData, getClass.getClassLoader) - reason match { - case fetchFailed: FetchFailed => - logInfo("Loss was due to fetch failure from " + fetchFailed.bmAddress) - sched.listener.taskEnded(tasks(index), fetchFailed, null, null, info, null) - finished(index) = true - tasksFinished += 1 - sched.taskSetFinished(this) - decreaseRunningTasks(runningTasks) - return - - case ef: ExceptionFailure => - val key = ef.description - val now = System.currentTimeMillis - val (printFull, dupCount) = { - if (recentExceptions.contains(key)) { - val (dupCount, printTime) = recentExceptions(key) - if (now - printTime > EXCEPTION_PRINT_INTERVAL) { - recentExceptions(key) = (0, now) - (true, 0) - } else { - recentExceptions(key) = (dupCount + 1, printTime) - (false, dupCount + 1) - } - } else { - recentExceptions(key) = (0, now) - (true, 0) - } - } - if (printFull) { - val locs = ef.stackTrace.map(loc => "\tat %s".format(loc.toString)) - logInfo("Loss was due to %s\n%s\n%s".format( - ef.className, ef.description, locs.mkString("\n"))) - } else { - logInfo("Loss was due to %s [duplicate %d]".format(ef.description, dupCount)) - } - - case _ => {} - } - } - // On non-fetch failures, re-enqueue the task as pending for a max number of retries - addPendingTask(index) - // Count failed attempts only on FAILED and LOST state (not on KILLED) - if (state == TaskState.FAILED || state == TaskState.LOST) { - numFailures(index) += 1 - if (numFailures(index) > MAX_TASK_FAILURES) { - logError("Task %s:%d failed more than %d times; aborting job".format( - taskSet.id, index, MAX_TASK_FAILURES)) - abort("Task %s:%d failed more than %d times".format(taskSet.id, index, MAX_TASK_FAILURES)) - } - } - } else { - logInfo("Ignoring task-lost event for TID " + tid + - " because task " + index + " is already finished") - } - } - - def error(message: String) { - // Save the error message - abort("Error: " + message) - } - - def abort(message: String) { - failed = true - causeOfFailure = message - // TODO: Kill running tasks if we were not terminated due to a Mesos error - sched.listener.taskSetFailed(taskSet, message) - decreaseRunningTasks(runningTasks) - sched.taskSetFinished(this) - } - - override def increaseRunningTasks(taskNum: Int) { - runningTasks += taskNum - if (parent != null) { - parent.increaseRunningTasks(taskNum) - } - } - - override def decreaseRunningTasks(taskNum: Int) { - runningTasks -= taskNum - if (parent != null) { - parent.decreaseRunningTasks(taskNum) - } - } - - //TODO: for now we just find Pool not TaskSetManager, we can extend this function in future if needed - override def getSchedulableByName(name: String): Schedulable = { - return null - } - - override def addSchedulable(schedulable:Schedulable) { - //nothing - } - - override def removeSchedulable(schedulable:Schedulable) { - //nothing - } - - override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { - var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] - sortedTaskSetQueue += this - return sortedTaskSetQueue - } - - override def executorLost(execId: String, hostPort: String) { - logInfo("Re-queueing tasks for " + execId + " from TaskSet " + taskSet.id) - - // If some task has preferred locations only on hostname, and there are no more executors there, - // put it in the no-prefs list to avoid the wait from delay scheduling - - // host local tasks - should we push this to rack local or no pref list ? For now, preserving behavior and moving to - // no prefs list. Note, this was done due to impliations related to 'waiting' for data local tasks, etc. - // Note: NOT checking process local list - since host local list is super set of that. We need to ad to no prefs only if - // there is no host local node for the task (not if there is no process local node for the task) - for (index <- getPendingTasksForHost(Utils.parseHostPort(hostPort)._1)) { - // val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) - val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) - if (newLocs.isEmpty) { - pendingTasksWithNoPrefs += index - } - } - - // Re-enqueue any tasks that ran on the failed executor if this is a shuffle map stage - if (tasks(0).isInstanceOf[ShuffleMapTask]) { - for ((tid, info) <- taskInfos if info.executorId == execId) { - val index = taskInfos(tid).index - if (finished(index)) { - finished(index) = false - copiesRunning(index) -= 1 - tasksFinished -= 1 - addPendingTask(index) - // Tell the DAGScheduler that this task was resubmitted so that it doesn't think our - // stage finishes when a total of tasks.size tasks finish. - sched.listener.taskEnded(tasks(index), Resubmitted, null, null, info, null) - } - } - } - // Also re-enqueue any tasks that were running on the node - for ((tid, info) <- taskInfos if info.running && info.executorId == execId) { - taskLost(tid, TaskState.KILLED, null) - } - } - - /** - * Check for tasks to be speculated and return true if there are any. This is called periodically - * by the ClusterScheduler. - * - * TODO: To make this scale to large jobs, we need to maintain a list of running tasks, so that - * we don't scan the whole task set. It might also help to make this sorted by launch time. - */ - override def checkSpeculatableTasks(): Boolean = { - // Can't speculate if we only have one task, or if all tasks have finished. - if (numTasks == 1 || tasksFinished == numTasks) { - return false - } - var foundTasks = false - val minFinishedForSpeculation = (SPECULATION_QUANTILE * numTasks).floor.toInt - logDebug("Checking for speculative tasks: minFinished = " + minFinishedForSpeculation) - if (tasksFinished >= minFinishedForSpeculation) { - val time = System.currentTimeMillis() - val durations = taskInfos.values.filter(_.successful).map(_.duration).toArray - Arrays.sort(durations) - val medianDuration = durations(min((0.5 * numTasks).round.toInt, durations.size - 1)) - val threshold = max(SPECULATION_MULTIPLIER * medianDuration, 100) - // TODO: Threshold should also look at standard deviation of task durations and have a lower - // bound based on that. - logDebug("Task length threshold for speculation: " + threshold) - for ((tid, info) <- taskInfos) { - val index = info.index - if (!finished(index) && copiesRunning(index) == 1 && info.timeRunning(time) > threshold && - !speculatableTasks.contains(index)) { - logInfo( - "Marking task %s:%d (on %s) as speculatable because it ran more than %.0f ms".format( - taskSet.id, index, info.hostPort, threshold)) - speculatableTasks += index - foundTasks = true - } - } - } - return foundTasks - } - - override def hasPendingTasks(): Boolean = { - numTasks > 0 && tasksFinished < numTasks - } +private[spark] trait TaskSetManager extends Schedulable { + def taskSet: TaskSet + def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] + def numPendingTasksForHostPort(hostPort: String): Int + def numRackLocalPendingTasksForHost(hostPort :String): Int + def numPendingTasksForHost(hostPort: String): Int + def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) + def error(message: String) } diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 37a67f9b1b..664dc9e886 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -2,19 +2,215 @@ package spark.scheduler.local import java.io.File import java.util.concurrent.atomic.AtomicInteger +import java.nio.ByteBuffer +import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet import spark._ +import spark.TaskState.TaskState import spark.executor.ExecutorURLClassLoader import spark.scheduler._ -import spark.scheduler.cluster.{TaskLocality, TaskInfo} +import spark.scheduler.cluster._ +import akka.actor._ /** * A simple TaskScheduler implementation that runs tasks locally in a thread pool. Optionally * the scheduler also allows each task to fail up to maxFailures times, which is useful for * testing fault recovery. */ -private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkContext) + +private[spark] case class LocalReviveOffers() +private[spark] case class LocalStatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) + +private[spark] class LocalActor(localScheduler: LocalScheduler, var freeCores: Int) extends Actor with Logging { + def receive = { + case LocalReviveOffers => + logInfo("LocalReviveOffers") + launchTask(localScheduler.resourceOffer(freeCores)) + case LocalStatusUpdate(taskId, state, serializeData) => + logInfo("LocalStatusUpdate") + freeCores += 1 + localScheduler.statusUpdate(taskId, state, serializeData) + launchTask(localScheduler.resourceOffer(freeCores)) + } + + def launchTask(tasks : Seq[TaskDescription]) { + for (task <- tasks) + { + freeCores -= 1 + localScheduler.threadPool.submit(new Runnable { + def run() { + localScheduler.runTask(task.taskId,task.serializedTask) + } + }) + } + } +} + +private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) extends TaskSetManager with Logging { + var parent: Schedulable = null + var weight: Int = 1 + var minShare: Int = 0 + var runningTasks: Int = 0 + var priority: Int = taskSet.priority + var stageId: Int = taskSet.stageId + var name: String = "TaskSet_"+taskSet.stageId.toString + + + var failCount = new Array[Int](taskSet.tasks.size) + val taskInfos = new HashMap[Long, TaskInfo] + val numTasks = taskSet.tasks.size + var numFinished = 0 + val ser = SparkEnv.get.closureSerializer.newInstance() + val copiesRunning = new Array[Int](numTasks) + val finished = new Array[Boolean](numTasks) + val numFailures = new Array[Int](numTasks) + + def increaseRunningTasks(taskNum: Int): Unit = { + runningTasks += taskNum + if (parent != null) { + parent.increaseRunningTasks(taskNum) + } + } + + def decreaseRunningTasks(taskNum: Int): Unit = { + runningTasks -= taskNum + if (parent != null) { + parent.decreaseRunningTasks(taskNum) + } + } + + def addSchedulable(schedulable: Schedulable): Unit = { + //nothing + } + + def removeSchedulable(schedulable: Schedulable): Unit = { + //nothing + } + + def getSchedulableByName(name: String): Schedulable = { + return null + } + + def executorLost(executorId: String, host: String): Unit = { + //nothing + } + + def checkSpeculatableTasks(): Boolean = { + return true + } + + def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { + var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] + sortedTaskSetQueue += this + return sortedTaskSetQueue + } + + def hasPendingTasks(): Boolean = { + return true + } + + def findTask(): Option[Int] = { + for (i <- 0 to numTasks-1) { + if (copiesRunning(i) == 0 && !finished(i)) { + return Some(i) + } + } + return None + } + + def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = { + Thread.currentThread().setContextClassLoader(sched.classLoader) + SparkEnv.set(sched.env) + logInfo("availableCpus:%d,numFinished:%d,numTasks:%d".format(availableCpus.toInt, numFinished, numTasks)) + if (availableCpus > 0 && numFinished < numTasks) { + findTask() match { + case Some(index) => + logInfo(taskSet.tasks(index).toString) + val taskId = sched.attemptId.getAndIncrement() + val task = taskSet.tasks(index) + logInfo("taskId:%d,task:%s".format(index,task)) + val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) + taskInfos(taskId) = info + val bytes = Task.serializeWithDependencies(task, sched.sc.addedFiles, sched.sc.addedJars, ser) + logInfo("Size of task " + taskId + " is " + bytes.limit + " bytes") + val taskName = "task %s:%d".format(taskSet.id, index) + copiesRunning(index) += 1 + increaseRunningTasks(1) + return Some(new TaskDescription(taskId, null, taskName, bytes)) + case None => {} + } + } + return None + } + + def numPendingTasksForHostPort(hostPort: String): Int = { + return 0 + } + + def numRackLocalPendingTasksForHost(hostPort :String): Int = { + return 0 + } + + def numPendingTasksForHost(hostPort: String): Int = { + return 0 + } + + def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { + state match { + case TaskState.FINISHED => + taskEnded(tid, state, serializedData) + case TaskState.FAILED => + taskFailed(tid, state, serializedData) + case _ => {} + } + } + + def taskEnded(tid: Long, state: TaskState, serializedData: ByteBuffer) { + val info = taskInfos(tid) + val index = info.index + val task = taskSet.tasks(index) + info.markSuccessful() + val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) + result.metrics.resultSize = serializedData.limit() + sched.listener.taskEnded(task, Success, result.value, result.accumUpdates, info, result.metrics) + numFinished += 1 + decreaseRunningTasks(1) + finished(index) = true + if (numFinished == numTasks) { + sched.taskSetFinished(this) + } + } + + def taskFailed(tid: Long, state: TaskState, serializedData: ByteBuffer) { + val info = taskInfos(tid) + val index = info.index + val task = taskSet.tasks(index) + info.markFailed() + decreaseRunningTasks(1) + val reason: ExceptionFailure = ser.deserialize[ExceptionFailure](serializedData, getClass.getClassLoader) + if (!finished(index)) { + copiesRunning(index) -= 1 + numFailures(index) += 1 + val locs = reason.stackTrace.map(loc => "\tat %s".format(loc.toString)) + logInfo("Loss was due to %s\n%s\n%s".format(reason.className, reason.description, locs.mkString("\n"))) + if (numFailures(index) > 4) { + val errorMessage = "Task %s:%d failed more than %d times; aborting job %s".format(taskSet.id, index, 4, reason.description) + //logError(errorMessage) + //sched.listener.taskEnded(task, reason, null, null, info, null) + sched.listener.taskSetFailed(taskSet, errorMessage) + sched.taskSetFinished(this) + decreaseRunningTasks(runningTasks) + } + } + } + + def error(message: String) { + } +} + +private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: SparkContext) extends TaskScheduler with Logging { @@ -30,90 +226,126 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon val classLoader = new ExecutorURLClassLoader(Array(), Thread.currentThread.getContextClassLoader) + var schedulableBuilder: SchedulableBuilder = null + var rootPool: Pool = null + val activeTaskSets = new HashMap[String, TaskSetManager] + val taskIdToTaskSetId = new HashMap[Long, String] + val taskSetTaskIds = new HashMap[String, HashSet[Long]] + + var localActor: ActorRef = null // TODO: Need to take into account stage priority in scheduling - override def start() { } + override def start() { + //default scheduler is FIFO + val schedulingMode = System.getProperty("spark.cluster.schedulingmode", "FIFO") + //temporarily set rootPool name to empty + rootPool = new Pool("", SchedulingMode.withName(schedulingMode), 0, 0) + schedulableBuilder = { + schedulingMode match { + case "FIFO" => + new FIFOSchedulableBuilder(rootPool) + case "FAIR" => + new FairSchedulableBuilder(rootPool) + } + } + schedulableBuilder.buildPools() + + //val properties = new ArrayBuffer[(String, String)] + localActor = env.actorSystem.actorOf( + Props(new LocalActor(this, threads)), "Test") + } override def setListener(listener: TaskSchedulerListener) { this.listener = listener } override def submitTasks(taskSet: TaskSet) { - val tasks = taskSet.tasks - val failCount = new Array[Int](tasks.size) + var manager = new LocalTaskSetManager(this, taskSet) + schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) + activeTaskSets(taskSet.id) = manager + taskSetTaskIds(taskSet.id) = new HashSet[Long]() + localActor ! LocalReviveOffers + } - def submitTask(task: Task[_], idInJob: Int) { - val myAttemptId = attemptId.getAndIncrement() - threadPool.submit(new Runnable { - def run() { - runTask(task, idInJob, myAttemptId) - } - }) + def resourceOffer(freeCores: Int): Seq[TaskDescription] = { + var freeCpuCores = freeCores + val tasks = new ArrayBuffer[TaskDescription](freeCores) + val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() + for (manager <- sortedTaskSetQueue) { + logInfo("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks)) } - def runTask(task: Task[_], idInJob: Int, attemptId: Int) { - logInfo("Running " + task) - val info = new TaskInfo(attemptId, idInJob, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) - // Set the Spark execution environment for the worker thread - SparkEnv.set(env) - try { - Accumulators.clear() - Thread.currentThread().setContextClassLoader(classLoader) - - // Serialize and deserialize the task so that accumulators are changed to thread-local ones; - // this adds a bit of unnecessary overhead but matches how the Mesos Executor works. - val ser = SparkEnv.get.closureSerializer.newInstance() - val bytes = Task.serializeWithDependencies(task, sc.addedFiles, sc.addedJars, ser) - logInfo("Size of task " + idInJob + " is " + bytes.limit + " bytes") - val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(bytes) - updateDependencies(taskFiles, taskJars) // Download any files added with addFile - val deserStart = System.currentTimeMillis() - val deserializedTask = ser.deserialize[Task[_]]( - taskBytes, Thread.currentThread.getContextClassLoader) - val deserTime = System.currentTimeMillis() - deserStart - - // Run it - val result: Any = deserializedTask.run(attemptId) - - // Serialize and deserialize the result to emulate what the Mesos - // executor does. This is useful to catch serialization errors early - // on in development (so when users move their local Spark programs - // to the cluster, they don't get surprised by serialization errors). - val serResult = ser.serialize(result) - deserializedTask.metrics.get.resultSize = serResult.limit() - val resultToReturn = ser.deserialize[Any](serResult) - val accumUpdates = ser.deserialize[collection.mutable.Map[Long, Any]]( - ser.serialize(Accumulators.values)) - logInfo("Finished " + task) - info.markSuccessful() - deserializedTask.metrics.get.executorRunTime = info.duration.toInt //close enough - deserializedTask.metrics.get.executorDeserializeTime = deserTime.toInt - - // If the threadpool has not already been shutdown, notify DAGScheduler - if (!Thread.currentThread().isInterrupted) - listener.taskEnded(task, Success, resultToReturn, accumUpdates, info, deserializedTask.metrics.getOrElse(null)) - } catch { - case t: Throwable => { - logError("Exception in task " + idInJob, t) - failCount.synchronized { - failCount(idInJob) += 1 - if (failCount(idInJob) <= maxFailures) { - submitTask(task, idInJob) - } else { - // TODO: Do something nicer here to return all the way to the user - if (!Thread.currentThread().isInterrupted) { - val failure = new ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace) - listener.taskEnded(task, failure, null, null, info, null) - } - } + var launchTask = false + for (manager <- sortedTaskSetQueue) { + do { + launchTask = false + logInfo("freeCores is" + freeCpuCores) + manager.slaveOffer(null,null,freeCpuCores) match { + case Some(task) => + tasks += task + taskIdToTaskSetId(task.taskId) = manager.taskSet.id + taskSetTaskIds(manager.taskSet.id) += task.taskId + freeCpuCores -= 1 + launchTask = true + case None => {} } - } - } + } while(launchTask) } + return tasks + } - for ((task, i) <- tasks.zipWithIndex) { - submitTask(task, i) - } + def taskSetFinished(manager: TaskSetManager) { + activeTaskSets -= manager.taskSet.id + manager.parent.removeSchedulable(manager) + logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id, manager.parent.name)) + taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id) + taskSetTaskIds -= manager.taskSet.id + } + + def runTask(taskId: Long, bytes: ByteBuffer) { + logInfo("Running " + taskId) + val info = new TaskInfo(taskId, 0 , System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) + // Set the Spark execution environment for the worker thread + SparkEnv.set(env) + val ser = SparkEnv.get.closureSerializer.newInstance() + try { + Accumulators.clear() + Thread.currentThread().setContextClassLoader(classLoader) + + // Serialize and deserialize the task so that accumulators are changed to thread-local ones; + // this adds a bit of unnecessary overhead but matches how the Mesos Executor works. + val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(bytes) + updateDependencies(taskFiles, taskJars) // Download any files added with addFile + val deserStart = System.currentTimeMillis() + val deserializedTask = ser.deserialize[Task[_]]( + taskBytes, Thread.currentThread.getContextClassLoader) + val deserTime = System.currentTimeMillis() - deserStart + + // Run it + val result: Any = deserializedTask.run(taskId) + + // Serialize and deserialize the result to emulate what the Mesos + // executor does. This is useful to catch serialization errors early + // on in development (so when users move their local Spark programs + // to the cluster, they don't get surprised by serialization errors). + val serResult = ser.serialize(result) + deserializedTask.metrics.get.resultSize = serResult.limit() + val resultToReturn = ser.deserialize[Any](serResult) + val accumUpdates = ser.deserialize[collection.mutable.Map[Long, Any]]( + ser.serialize(Accumulators.values)) + logInfo("Finished " + taskId) + deserializedTask.metrics.get.executorRunTime = deserTime.toInt//info.duration.toInt //close enough + deserializedTask.metrics.get.executorDeserializeTime = deserTime.toInt + + val taskResult = new TaskResult(result, accumUpdates, deserializedTask.metrics.getOrElse(null)) + val serializedResult = ser.serialize(taskResult) + localActor ! LocalStatusUpdate(taskId, TaskState.FINISHED, serializedResult) + } catch { + case t: Throwable => { + val failure = new ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace) + localActor ! LocalStatusUpdate(taskId, TaskState.FAILED, ser.serialize(failure)) + } + } } /** @@ -128,6 +360,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon Utils.fetchFile(name, new File(SparkFiles.getRootDirectory)) currentFiles(name) = timestamp } + for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) Utils.fetchFile(name, new File(SparkFiles.getRootDirectory)) @@ -143,7 +376,14 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon } } - override def stop() { + def statusUpdate(taskId :Long, state: TaskState, serializedData: ByteBuffer) + { + val taskSetId = taskIdToTaskSetId(taskId) + val taskSetManager = activeTaskSets(taskSetId) + taskSetManager.statusUpdate(taskId, state, serializedData) + } + + override def stop() { threadPool.shutdownNow() } diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala index a39418b716..e6ad90192e 100644 --- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala @@ -16,7 +16,7 @@ class DummyTaskSetManager( initNumTasks: Int, clusterScheduler: ClusterScheduler, taskSet: TaskSet) - extends TaskSetManager(clusterScheduler,taskSet) { + extends ClusterTaskSetManager(clusterScheduler,taskSet) { parent = null weight = 1 -- cgit v1.2.3 From c3db3ea55467c3fb053453c8c567db357d939640 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Thu, 30 May 2013 20:49:40 +0800 Subject: 1. Add unit test for local scheduler 2. Move localTaskSetManager to a new file --- .../spark/scheduler/local/LocalScheduler.scala | 241 ++++----------------- .../scheduler/local/LocalTaskSetManager.scala | 173 +++++++++++++++ .../spark/scheduler/LocalSchedulerSuite.scala | 171 +++++++++++++++ 3 files changed, 385 insertions(+), 200 deletions(-) create mode 100644 core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala create mode 100644 core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 664dc9e886..69dacfc2bd 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -15,7 +15,7 @@ import spark.scheduler.cluster._ import akka.actor._ /** - * A simple TaskScheduler implementation that runs tasks locally in a thread pool. Optionally + * A FIFO or Fair TaskScheduler implementation that runs tasks locally in a thread pool. Optionally * the scheduler also allows each task to fail up to maxFailures times, which is useful for * testing fault recovery. */ @@ -26,10 +26,8 @@ private[spark] case class LocalStatusUpdate(taskId: Long, state: TaskState, seri private[spark] class LocalActor(localScheduler: LocalScheduler, var freeCores: Int) extends Actor with Logging { def receive = { case LocalReviveOffers => - logInfo("LocalReviveOffers") launchTask(localScheduler.resourceOffer(freeCores)) case LocalStatusUpdate(taskId, state, serializeData) => - logInfo("LocalStatusUpdate") freeCores += 1 localScheduler.statusUpdate(taskId, state, serializeData) launchTask(localScheduler.resourceOffer(freeCores)) @@ -48,168 +46,6 @@ private[spark] class LocalActor(localScheduler: LocalScheduler, var freeCores: I } } -private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) extends TaskSetManager with Logging { - var parent: Schedulable = null - var weight: Int = 1 - var minShare: Int = 0 - var runningTasks: Int = 0 - var priority: Int = taskSet.priority - var stageId: Int = taskSet.stageId - var name: String = "TaskSet_"+taskSet.stageId.toString - - - var failCount = new Array[Int](taskSet.tasks.size) - val taskInfos = new HashMap[Long, TaskInfo] - val numTasks = taskSet.tasks.size - var numFinished = 0 - val ser = SparkEnv.get.closureSerializer.newInstance() - val copiesRunning = new Array[Int](numTasks) - val finished = new Array[Boolean](numTasks) - val numFailures = new Array[Int](numTasks) - - def increaseRunningTasks(taskNum: Int): Unit = { - runningTasks += taskNum - if (parent != null) { - parent.increaseRunningTasks(taskNum) - } - } - - def decreaseRunningTasks(taskNum: Int): Unit = { - runningTasks -= taskNum - if (parent != null) { - parent.decreaseRunningTasks(taskNum) - } - } - - def addSchedulable(schedulable: Schedulable): Unit = { - //nothing - } - - def removeSchedulable(schedulable: Schedulable): Unit = { - //nothing - } - - def getSchedulableByName(name: String): Schedulable = { - return null - } - - def executorLost(executorId: String, host: String): Unit = { - //nothing - } - - def checkSpeculatableTasks(): Boolean = { - return true - } - - def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { - var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] - sortedTaskSetQueue += this - return sortedTaskSetQueue - } - - def hasPendingTasks(): Boolean = { - return true - } - - def findTask(): Option[Int] = { - for (i <- 0 to numTasks-1) { - if (copiesRunning(i) == 0 && !finished(i)) { - return Some(i) - } - } - return None - } - - def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = { - Thread.currentThread().setContextClassLoader(sched.classLoader) - SparkEnv.set(sched.env) - logInfo("availableCpus:%d,numFinished:%d,numTasks:%d".format(availableCpus.toInt, numFinished, numTasks)) - if (availableCpus > 0 && numFinished < numTasks) { - findTask() match { - case Some(index) => - logInfo(taskSet.tasks(index).toString) - val taskId = sched.attemptId.getAndIncrement() - val task = taskSet.tasks(index) - logInfo("taskId:%d,task:%s".format(index,task)) - val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) - taskInfos(taskId) = info - val bytes = Task.serializeWithDependencies(task, sched.sc.addedFiles, sched.sc.addedJars, ser) - logInfo("Size of task " + taskId + " is " + bytes.limit + " bytes") - val taskName = "task %s:%d".format(taskSet.id, index) - copiesRunning(index) += 1 - increaseRunningTasks(1) - return Some(new TaskDescription(taskId, null, taskName, bytes)) - case None => {} - } - } - return None - } - - def numPendingTasksForHostPort(hostPort: String): Int = { - return 0 - } - - def numRackLocalPendingTasksForHost(hostPort :String): Int = { - return 0 - } - - def numPendingTasksForHost(hostPort: String): Int = { - return 0 - } - - def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { - state match { - case TaskState.FINISHED => - taskEnded(tid, state, serializedData) - case TaskState.FAILED => - taskFailed(tid, state, serializedData) - case _ => {} - } - } - - def taskEnded(tid: Long, state: TaskState, serializedData: ByteBuffer) { - val info = taskInfos(tid) - val index = info.index - val task = taskSet.tasks(index) - info.markSuccessful() - val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) - result.metrics.resultSize = serializedData.limit() - sched.listener.taskEnded(task, Success, result.value, result.accumUpdates, info, result.metrics) - numFinished += 1 - decreaseRunningTasks(1) - finished(index) = true - if (numFinished == numTasks) { - sched.taskSetFinished(this) - } - } - - def taskFailed(tid: Long, state: TaskState, serializedData: ByteBuffer) { - val info = taskInfos(tid) - val index = info.index - val task = taskSet.tasks(index) - info.markFailed() - decreaseRunningTasks(1) - val reason: ExceptionFailure = ser.deserialize[ExceptionFailure](serializedData, getClass.getClassLoader) - if (!finished(index)) { - copiesRunning(index) -= 1 - numFailures(index) += 1 - val locs = reason.stackTrace.map(loc => "\tat %s".format(loc.toString)) - logInfo("Loss was due to %s\n%s\n%s".format(reason.className, reason.description, locs.mkString("\n"))) - if (numFailures(index) > 4) { - val errorMessage = "Task %s:%d failed more than %d times; aborting job %s".format(taskSet.id, index, 4, reason.description) - //logError(errorMessage) - //sched.listener.taskEnded(task, reason, null, null, info, null) - sched.listener.taskSetFailed(taskSet, errorMessage) - sched.taskSetFinished(this) - decreaseRunningTasks(runningTasks) - } - } - } - - def error(message: String) { - } -} - private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: SparkContext) extends TaskScheduler with Logging { @@ -233,7 +69,6 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: val taskSetTaskIds = new HashMap[String, HashSet[Long]] var localActor: ActorRef = null - // TODO: Need to take into account stage priority in scheduling override def start() { //default scheduler is FIFO @@ -250,7 +85,6 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: } schedulableBuilder.buildPools() - //val properties = new ArrayBuffer[(String, String)] localActor = env.actorSystem.actorOf( Props(new LocalActor(this, threads)), "Test") } @@ -260,51 +94,56 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: } override def submitTasks(taskSet: TaskSet) { - var manager = new LocalTaskSetManager(this, taskSet) - schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) - activeTaskSets(taskSet.id) = manager - taskSetTaskIds(taskSet.id) = new HashSet[Long]() - localActor ! LocalReviveOffers + synchronized { + var manager = new LocalTaskSetManager(this, taskSet) + schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) + activeTaskSets(taskSet.id) = manager + taskSetTaskIds(taskSet.id) = new HashSet[Long]() + localActor ! LocalReviveOffers + } } def resourceOffer(freeCores: Int): Seq[TaskDescription] = { - var freeCpuCores = freeCores - val tasks = new ArrayBuffer[TaskDescription](freeCores) - val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() - for (manager <- sortedTaskSetQueue) { - logInfo("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks)) - } + synchronized { + var freeCpuCores = freeCores + val tasks = new ArrayBuffer[TaskDescription](freeCores) + val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() + for (manager <- sortedTaskSetQueue) { + logInfo("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks)) + } - var launchTask = false - for (manager <- sortedTaskSetQueue) { + var launchTask = false + for (manager <- sortedTaskSetQueue) { do { launchTask = false - logInfo("freeCores is" + freeCpuCores) manager.slaveOffer(null,null,freeCpuCores) match { case Some(task) => - tasks += task - taskIdToTaskSetId(task.taskId) = manager.taskSet.id - taskSetTaskIds(manager.taskSet.id) += task.taskId - freeCpuCores -= 1 - launchTask = true + tasks += task + taskIdToTaskSetId(task.taskId) = manager.taskSet.id + taskSetTaskIds(manager.taskSet.id) += task.taskId + freeCpuCores -= 1 + launchTask = true case None => {} - } + } } while(launchTask) + } + return tasks } - return tasks } def taskSetFinished(manager: TaskSetManager) { - activeTaskSets -= manager.taskSet.id - manager.parent.removeSchedulable(manager) - logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id, manager.parent.name)) - taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id) - taskSetTaskIds -= manager.taskSet.id + synchronized { + activeTaskSets -= manager.taskSet.id + manager.parent.removeSchedulable(manager) + logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id, manager.parent.name)) + taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id) + taskSetTaskIds -= manager.taskSet.id + } } def runTask(taskId: Long, bytes: ByteBuffer) { logInfo("Running " + taskId) - val info = new TaskInfo(taskId, 0 , System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) + val info = new TaskInfo(taskId, 0, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) // Set the Spark execution environment for the worker thread SparkEnv.set(env) val ser = SparkEnv.get.closureSerializer.newInstance() @@ -344,8 +183,8 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: case t: Throwable => { val failure = new ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace) localActor ! LocalStatusUpdate(taskId, TaskState.FAILED, ser.serialize(failure)) - } } + } } /** @@ -376,11 +215,13 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: } } - def statusUpdate(taskId :Long, state: TaskState, serializedData: ByteBuffer) - { - val taskSetId = taskIdToTaskSetId(taskId) - val taskSetManager = activeTaskSets(taskSetId) - taskSetManager.statusUpdate(taskId, state, serializedData) + def statusUpdate(taskId :Long, state: TaskState, serializedData: ByteBuffer) { + synchronized { + val taskSetId = taskIdToTaskSetId(taskId) + val taskSetManager = activeTaskSets(taskSetId) + taskSetTaskIds(taskSetId) -= taskId + taskSetManager.statusUpdate(taskId, state, serializedData) + } } override def stop() { diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala new file mode 100644 index 0000000000..f2e07d162a --- /dev/null +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -0,0 +1,173 @@ +package spark.scheduler.local + +import java.io.File +import java.util.concurrent.atomic.AtomicInteger +import java.nio.ByteBuffer +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet + +import spark._ +import spark.TaskState.TaskState +import spark.scheduler._ +import spark.scheduler.cluster._ + +private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) extends TaskSetManager with Logging { + var parent: Schedulable = null + var weight: Int = 1 + var minShare: Int = 0 + var runningTasks: Int = 0 + var priority: Int = taskSet.priority + var stageId: Int = taskSet.stageId + var name: String = "TaskSet_"+taskSet.stageId.toString + + + var failCount = new Array[Int](taskSet.tasks.size) + val taskInfos = new HashMap[Long, TaskInfo] + val numTasks = taskSet.tasks.size + var numFinished = 0 + val ser = SparkEnv.get.closureSerializer.newInstance() + val copiesRunning = new Array[Int](numTasks) + val finished = new Array[Boolean](numTasks) + val numFailures = new Array[Int](numTasks) + val MAX_TASK_FAILURES = sched.maxFailures + + def increaseRunningTasks(taskNum: Int): Unit = { + runningTasks += taskNum + if (parent != null) { + parent.increaseRunningTasks(taskNum) + } + } + + def decreaseRunningTasks(taskNum: Int): Unit = { + runningTasks -= taskNum + if (parent != null) { + parent.decreaseRunningTasks(taskNum) + } + } + + def addSchedulable(schedulable: Schedulable): Unit = { + //nothing + } + + def removeSchedulable(schedulable: Schedulable): Unit = { + //nothing + } + + def getSchedulableByName(name: String): Schedulable = { + return null + } + + def executorLost(executorId: String, host: String): Unit = { + //nothing + } + + def checkSpeculatableTasks(): Boolean = { + return true + } + + def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { + var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] + sortedTaskSetQueue += this + return sortedTaskSetQueue + } + + def hasPendingTasks(): Boolean = { + return true + } + + def findTask(): Option[Int] = { + for (i <- 0 to numTasks-1) { + if (copiesRunning(i) == 0 && !finished(i)) { + return Some(i) + } + } + return None + } + + def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = { + SparkEnv.set(sched.env) + logDebug("availableCpus:%d,numFinished:%d,numTasks:%d".format(availableCpus.toInt, numFinished, numTasks)) + if (availableCpus > 0 && numFinished < numTasks) { + findTask() match { + case Some(index) => + logInfo(taskSet.tasks(index).toString) + val taskId = sched.attemptId.getAndIncrement() + val task = taskSet.tasks(index) + val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) + taskInfos(taskId) = info + val bytes = Task.serializeWithDependencies(task, sched.sc.addedFiles, sched.sc.addedJars, ser) + logInfo("Size of task " + taskId + " is " + bytes.limit + " bytes") + val taskName = "task %s:%d".format(taskSet.id, index) + copiesRunning(index) += 1 + increaseRunningTasks(1) + return Some(new TaskDescription(taskId, null, taskName, bytes)) + case None => {} + } + } + return None + } + + def numPendingTasksForHostPort(hostPort: String): Int = { + return 0 + } + + def numRackLocalPendingTasksForHost(hostPort :String): Int = { + return 0 + } + + def numPendingTasksForHost(hostPort: String): Int = { + return 0 + } + + def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { + state match { + case TaskState.FINISHED => + taskEnded(tid, state, serializedData) + case TaskState.FAILED => + taskFailed(tid, state, serializedData) + case _ => {} + } + } + + def taskEnded(tid: Long, state: TaskState, serializedData: ByteBuffer) { + val info = taskInfos(tid) + val index = info.index + val task = taskSet.tasks(index) + info.markSuccessful() + val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) + result.metrics.resultSize = serializedData.limit() + sched.listener.taskEnded(task, Success, result.value, result.accumUpdates, info, result.metrics) + numFinished += 1 + decreaseRunningTasks(1) + finished(index) = true + if (numFinished == numTasks) { + sched.taskSetFinished(this) + } + } + + def taskFailed(tid: Long, state: TaskState, serializedData: ByteBuffer) { + val info = taskInfos(tid) + val index = info.index + val task = taskSet.tasks(index) + info.markFailed() + decreaseRunningTasks(1) + val reason: ExceptionFailure = ser.deserialize[ExceptionFailure](serializedData, getClass.getClassLoader) + if (!finished(index)) { + copiesRunning(index) -= 1 + numFailures(index) += 1 + val locs = reason.stackTrace.map(loc => "\tat %s".format(loc.toString)) + logInfo("Loss was due to %s\n%s\n%s".format(reason.className, reason.description, locs.mkString("\n"))) + if (numFailures(index) > MAX_TASK_FAILURES) { + val errorMessage = "Task %s:%d failed more than %d times; aborting job %s".format(taskSet.id, index, 4, reason.description) + decreaseRunningTasks(runningTasks) + sched.listener.taskSetFailed(taskSet, errorMessage) + // need to delete failed Taskset from schedule queue + sched.taskSetFinished(this) + } + } + } + + def error(message: String) { + } +} diff --git a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala new file mode 100644 index 0000000000..37d14ed113 --- /dev/null +++ b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala @@ -0,0 +1,171 @@ +package spark.scheduler + +import org.scalatest.FunSuite +import org.scalatest.BeforeAndAfter + +import spark._ +import spark.scheduler._ +import spark.scheduler.cluster._ +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ConcurrentMap, HashMap} +import java.util.concurrent.Semaphore +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.atomic.AtomicInteger + +import java.util.Properties + +class Lock() { + var finished = false + def jobWait() = { + synchronized { + while(!finished) { + this.wait() + } + } + } + + def jobFinished() = { + synchronized { + finished = true + this.notifyAll() + } + } +} + +object TaskThreadInfo { + val threadToLock = HashMap[Int, Lock]() + val threadToRunning = HashMap[Int, Boolean]() +} + + +class LocalSchedulerSuite extends FunSuite with LocalSparkContext { + + def createThread(threadIndex: Int, poolName: String, sc: SparkContext, sem: Semaphore) { + + TaskThreadInfo.threadToRunning(threadIndex) = false + val nums = sc.parallelize(threadIndex to threadIndex, 1) + TaskThreadInfo.threadToLock(threadIndex) = new Lock() + new Thread { + if (poolName != null) { + sc.addLocalProperties("spark.scheduler.cluster.fair.pool",poolName) + } + override def run() { + val ans = nums.map(number => { + TaskThreadInfo.threadToRunning(number) = true + TaskThreadInfo.threadToLock(number).jobWait() + number + }).collect() + assert(ans.toList === List(threadIndex)) + sem.release() + TaskThreadInfo.threadToRunning(threadIndex) = false + } + }.start() + Thread.sleep(2000) + } + + test("Local FIFO scheduler end-to-end test") { + System.setProperty("spark.cluster.schedulingmode", "FIFO") + sc = new SparkContext("local[4]", "test") + val sem = new Semaphore(0) + + createThread(1,null,sc,sem) + createThread(2,null,sc,sem) + createThread(3,null,sc,sem) + createThread(4,null,sc,sem) + createThread(5,null,sc,sem) + createThread(6,null,sc,sem) + assert(TaskThreadInfo.threadToRunning(1) === true) + assert(TaskThreadInfo.threadToRunning(2) === true) + assert(TaskThreadInfo.threadToRunning(3) === true) + assert(TaskThreadInfo.threadToRunning(4) === true) + assert(TaskThreadInfo.threadToRunning(5) === false) + assert(TaskThreadInfo.threadToRunning(6) === false) + + TaskThreadInfo.threadToLock(1).jobFinished() + Thread.sleep(1000) + + assert(TaskThreadInfo.threadToRunning(1) === false) + assert(TaskThreadInfo.threadToRunning(2) === true) + assert(TaskThreadInfo.threadToRunning(3) === true) + assert(TaskThreadInfo.threadToRunning(4) === true) + assert(TaskThreadInfo.threadToRunning(5) === true) + assert(TaskThreadInfo.threadToRunning(6) === false) + + TaskThreadInfo.threadToLock(3).jobFinished() + Thread.sleep(1000) + + assert(TaskThreadInfo.threadToRunning(1) === false) + assert(TaskThreadInfo.threadToRunning(2) === true) + assert(TaskThreadInfo.threadToRunning(3) === false) + assert(TaskThreadInfo.threadToRunning(4) === true) + assert(TaskThreadInfo.threadToRunning(5) === true) + assert(TaskThreadInfo.threadToRunning(6) === true) + + TaskThreadInfo.threadToLock(2).jobFinished() + TaskThreadInfo.threadToLock(4).jobFinished() + TaskThreadInfo.threadToLock(5).jobFinished() + TaskThreadInfo.threadToLock(6).jobFinished() + sem.acquire(6) + } + + test("Local fair scheduler end-to-end test") { + sc = new SparkContext("local[8]", "LocalSchedulerSuite") + val sem = new Semaphore(0) + System.setProperty("spark.cluster.schedulingmode", "FAIR") + val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() + System.setProperty("spark.fairscheduler.allocation.file", xmlPath) + + createThread(10,"1",sc,sem) + createThread(20,"2",sc,sem) + createThread(30,"3",sc,sem) + + assert(TaskThreadInfo.threadToRunning(10) === true) + assert(TaskThreadInfo.threadToRunning(20) === true) + assert(TaskThreadInfo.threadToRunning(30) === true) + + createThread(11,"1",sc,sem) + createThread(21,"2",sc,sem) + createThread(31,"3",sc,sem) + + assert(TaskThreadInfo.threadToRunning(11) === true) + assert(TaskThreadInfo.threadToRunning(21) === true) + assert(TaskThreadInfo.threadToRunning(31) === true) + + createThread(12,"1",sc,sem) + createThread(22,"2",sc,sem) + createThread(32,"3",sc,sem) + + assert(TaskThreadInfo.threadToRunning(12) === true) + assert(TaskThreadInfo.threadToRunning(22) === true) + assert(TaskThreadInfo.threadToRunning(32) === false) + + TaskThreadInfo.threadToLock(10).jobFinished() + Thread.sleep(1000) + assert(TaskThreadInfo.threadToRunning(32) === true) + + createThread(23,"2",sc,sem) + createThread(33,"3",sc,sem) + + TaskThreadInfo.threadToLock(11).jobFinished() + Thread.sleep(1000) + + assert(TaskThreadInfo.threadToRunning(23) === true) + assert(TaskThreadInfo.threadToRunning(33) === false) + + TaskThreadInfo.threadToLock(12).jobFinished() + Thread.sleep(1000) + + assert(TaskThreadInfo.threadToRunning(33) === true) + + TaskThreadInfo.threadToLock(20).jobFinished() + TaskThreadInfo.threadToLock(21).jobFinished() + TaskThreadInfo.threadToLock(22).jobFinished() + TaskThreadInfo.threadToLock(23).jobFinished() + TaskThreadInfo.threadToLock(30).jobFinished() + TaskThreadInfo.threadToLock(31).jobFinished() + TaskThreadInfo.threadToLock(32).jobFinished() + TaskThreadInfo.threadToLock(33).jobFinished() + + sem.acquire(11) + } +} -- cgit v1.2.3 From bed1b08169df91e97cb9ebaf8e58daeb655ff55d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 30 May 2013 16:21:49 -0700 Subject: Do not create symlink for local add file. Instead, copy the file. This prevents Spark from changing the original file's permission, and also allow add file to work on non-posix operating systems. --- core/src/main/scala/spark/Utils.scala | 78 +++++++++++++++++------------------ 1 file changed, 39 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 84626df553..ec15326014 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -4,20 +4,26 @@ import java.io._ import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address, ServerSocket} import java.util.{Locale, Random, UUID} import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadFactory, ThreadPoolExecutor} -import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} +import java.util.regex.Pattern + import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.collection.JavaConversions._ import scala.io.Source + import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder + +import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} + import spark.serializer.SerializerInstance import spark.deploy.SparkHadoopUtil -import java.util.regex.Pattern + /** * Various utility methods used by Spark. */ private object Utils extends Logging { + /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -68,7 +74,6 @@ private object Utils extends Logging { return buf } - private val shutdownDeletePaths = new collection.mutable.HashSet[String]() // Register the path to be deleted via shutdown hook @@ -87,19 +92,19 @@ private object Utils extends Logging { } } - // Note: if file is child of some registered path, while not equal to it, then return true; else false - // This is to ensure that two shutdown hooks do not try to delete each others paths - resulting in IOException - // and incomplete cleanup + // Note: if file is child of some registered path, while not equal to it, then return true; + // else false. This is to ensure that two shutdown hooks do not try to delete each others + // paths - resulting in IOException and incomplete cleanup. def hasRootAsShutdownDeleteDir(file: File): Boolean = { - val absolutePath = file.getAbsolutePath() - val retval = shutdownDeletePaths.synchronized { - shutdownDeletePaths.find(path => ! absolutePath.equals(path) && absolutePath.startsWith(path) ).isDefined + shutdownDeletePaths.find { path => + !absolutePath.equals(path) && absolutePath.startsWith(path) + }.isDefined + } + if (retval) { + logInfo("path = " + file + ", already present as root for deletion.") } - - if (retval) logInfo("path = " + file + ", already present as root for deletion.") - retval } @@ -131,7 +136,7 @@ private object Utils extends Logging { if (! hasRootAsShutdownDeleteDir(dir)) Utils.deleteRecursively(dir) } }) - return dir + dir } /** Copy all data from an InputStream to an OutputStream */ @@ -174,35 +179,30 @@ private object Utils extends Logging { Utils.copyStream(in, out, true) if (targetFile.exists && !Files.equal(tempFile, targetFile)) { tempFile.delete() - throw new SparkException("File " + targetFile + " exists and does not match contents of" + - " " + url) + throw new SparkException( + "File " + targetFile + " exists and does not match contents of" + " " + url) } else { Files.move(tempFile, targetFile) } case "file" | null => - val sourceFile = if (uri.isAbsolute) { - new File(uri) - } else { - new File(url) - } - if (targetFile.exists && !Files.equal(sourceFile, targetFile)) { - throw new SparkException("File " + targetFile + " exists and does not match contents of" + - " " + url) - } else { - // Remove the file if it already exists - targetFile.delete() - // Symlink the file locally. - if (uri.isAbsolute) { - // url is absolute, i.e. it starts with "file:///". Extract the source - // file's absolute path from the url. - val sourceFile = new File(uri) - logInfo("Symlinking " + sourceFile.getAbsolutePath + " to " + targetFile.getAbsolutePath) - FileUtil.symLink(sourceFile.getAbsolutePath, targetFile.getAbsolutePath) + // In the case of a local file, copy the local file to the target directory. + // Note the difference between uri vs url. + val sourceFile = if (uri.isAbsolute) new File(uri) else new File(url) + if (targetFile.exists) { + // If the target file already exists, warn the user if + if (!Files.equal(sourceFile, targetFile)) { + throw new SparkException( + "File " + targetFile + " exists and does not match contents of" + " " + url) } else { - // url is not absolute, i.e. itself is the path to the source file. - logInfo("Symlinking " + url + " to " + targetFile.getAbsolutePath) - FileUtil.symLink(url, targetFile.getAbsolutePath) + // Do nothing if the file contents are the same, i.e. this file has been copied + // previously. + logInfo(sourceFile.getAbsolutePath + " has been previously copied to " + + targetFile.getAbsolutePath) } + } else { + // The file does not exist in the target directory. Copy it there. + logInfo("Copying " + sourceFile.getAbsolutePath + " to " + targetFile.getAbsolutePath) + Files.copy(sourceFile, targetFile) } case _ => // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others @@ -323,8 +323,6 @@ private object Utils extends Logging { InetAddress.getByName(address).getHostName } - - def localHostPort(): String = { val retval = System.getProperty("spark.hostPort", null) if (retval == null) { @@ -382,6 +380,7 @@ private object Utils extends Logging { // Typically, this will be of order of number of nodes in cluster // If not, we should change it to LRUCache or something. private val hostPortParseResults = new ConcurrentHashMap[String, (String, Int)]() + def parseHostPort(hostPort: String): (String, Int) = { { // Check cache first. @@ -390,7 +389,8 @@ private object Utils extends Logging { } val indx: Int = hostPort.lastIndexOf(':') - // This is potentially broken - when dealing with ipv6 addresses for example, sigh ... but then hadoop does not support ipv6 right now. + // This is potentially broken - when dealing with ipv6 addresses for example, sigh ... + // but then hadoop does not support ipv6 right now. // For now, we assume that if port exists, then it is valid - not check if it is an int > 0 if (-1 == indx) { val retval = (hostPort, 0) -- cgit v1.2.3 From f6ad3781b1d9a044789f114d13787b9d05223da3 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 30 May 2013 16:28:08 -0700 Subject: Fixed the flaky unpersist test in RDDSuite. --- core/src/test/scala/spark/RDDSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index a761dd77c5..3f69e99780 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -106,9 +106,9 @@ class RDDSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local", "test") val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() rdd.count - assert(sc.persistentRdds.isEmpty == false) + assert(sc.persistentRdds.isEmpty === false) rdd.unpersist() - assert(sc.persistentRdds.isEmpty == true) + assert(sc.persistentRdds.isEmpty === true) failAfter(Span(3000, Millis)) { try { @@ -116,12 +116,12 @@ class RDDSuite extends FunSuite with LocalSparkContext { Thread.sleep(200) } } catch { - case e: Exception => + case _ => { Thread.sleep(10) } // Do nothing. We might see exceptions because block manager // is racing this thread to remove entries from the driver. } } - assert(sc.getRDDStorageInfo.isEmpty == true) + assert(sc.getRDDStorageInfo.isEmpty === true) } test("caching with failures") { -- cgit v1.2.3 From 926f41cc522def181c167b71dc919a0759c5d3f6 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 30 May 2013 17:55:11 +0800 Subject: fix block manager UI display issue when enable spark.cleaner.ttl --- core/src/main/scala/spark/storage/StorageUtils.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index 8f52168c24..81e607868d 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -64,12 +64,12 @@ object StorageUtils { // Find the id of the RDD, e.g. rdd_1 => 1 val rddId = rddKey.split("_").last.toInt // Get the friendly name for the rdd, if available. - val rdd = sc.persistentRdds(rddId) - val rddName = Option(rdd.name).getOrElse(rddKey) - val rddStorageLevel = rdd.getStorageLevel - - RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, rdd.partitions.size, memSize, diskSize) - }.toArray + sc.persistentRdds.get(rddId).map { r => + val rddName = Option(r.name).getOrElse(rddKey) + val rddStorageLevel = r.getStorageLevel + RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, r.partitions.size, memSize, diskSize) + } + }.flatMap(x => x).toArray scala.util.Sorting.quickSort(rddInfos) -- cgit v1.2.3 From ba5e544461e8ca9216af703033f6b0de6dbc56ec Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 31 May 2013 01:48:16 -0700 Subject: More block manager cleanup. Implemented a removeRdd method in BlockManager, and use that to implement RDD.unpersist. Previously, unpersist needs to send B akka messages, where B = number of blocks. Now unpersist only needs to send W akka messages, where W = the number of workers. --- core/src/main/scala/spark/RDD.scala | 21 +-- .../main/scala/spark/storage/BlockManager.scala | 31 +++- .../scala/spark/storage/BlockManagerMaster.scala | 49 +++--- .../spark/storage/BlockManagerMasterActor.scala | 192 ++++++++++----------- .../scala/spark/storage/BlockManagerMessages.scala | 6 + .../spark/storage/BlockManagerSlaveActor.scala | 8 +- .../scala/spark/storage/BlockManagerWorker.scala | 10 +- .../scala/spark/storage/BlockManagerSuite.scala | 36 ++-- 8 files changed, 187 insertions(+), 166 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index dde131696f..e6c0438d76 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -1,13 +1,10 @@ package spark -import java.net.URL -import java.util.{Date, Random} -import java.util.{HashMap => JHashMap} +import java.util.Random import scala.collection.Map import scala.collection.JavaConversions.mapAsScalaMap import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap import org.apache.hadoop.io.BytesWritable import org.apache.hadoop.io.NullWritable @@ -32,7 +29,6 @@ import spark.rdd.MapPartitionsWithIndexRDD import spark.rdd.PipedRDD import spark.rdd.SampledRDD import spark.rdd.ShuffledRDD -import spark.rdd.SubtractedRDD import spark.rdd.UnionRDD import spark.rdd.ZippedRDD import spark.rdd.ZippedPartitionsRDD2 @@ -141,10 +137,15 @@ abstract class RDD[T: ClassManifest]( /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ def cache(): RDD[T] = persist() - /** Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. */ - def unpersist(): RDD[T] = { + /** + * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. + * + * @param blocking Whether to block until all blocks are deleted. + * @return This RDD. + */ + def unpersist(blocking: Boolean = true): RDD[T] = { logInfo("Removing RDD " + id + " from persistence list") - sc.env.blockManager.master.removeRdd(id) + sc.env.blockManager.master.removeRdd(id, blocking) sc.persistentRdds.remove(id) storageLevel = StorageLevel.NONE this @@ -269,8 +270,8 @@ abstract class RDD[T: ClassManifest]( def takeSample(withReplacement: Boolean, num: Int, seed: Int): Array[T] = { var fraction = 0.0 var total = 0 - var multiplier = 3.0 - var initialCount = count() + val multiplier = 3.0 + val initialCount = count() var maxSelected = 0 if (initialCount > Integer.MAX_VALUE - 1) { diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index d35c43f194..3a5d4ef448 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -3,8 +3,7 @@ package spark.storage import java.io.{InputStream, OutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} -import scala.collection.mutable.{HashMap, ArrayBuffer, HashSet, Queue} -import scala.collection.JavaConversions._ +import scala.collection.mutable.{HashMap, ArrayBuffer, HashSet} import akka.actor.{ActorSystem, Cancellable, Props} import akka.dispatch.{Await, Future} @@ -15,7 +14,7 @@ import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream -import spark.{Logging, SizeEstimator, SparkEnv, SparkException, Utils} +import spark.{Logging, SparkEnv, SparkException, Utils} import spark.network._ import spark.serializer.Serializer import spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStampedHashMap} @@ -95,9 +94,11 @@ private[spark] class BlockManager( new DiskStore(this, System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir"))) // If we use Netty for shuffle, start a new Netty-based shuffle sender service. - private val useNetty = System.getProperty("spark.shuffle.use.netty", "false").toBoolean - private val nettyPortConfig = System.getProperty("spark.shuffle.sender.port", "0").toInt - private val nettyPort = if (useNetty) diskStore.startShuffleBlockSender(nettyPortConfig) else 0 + private val nettyPort: Int = { + val useNetty = System.getProperty("spark.shuffle.use.netty", "false").toBoolean + val nettyPortConfig = System.getProperty("spark.shuffle.sender.port", "0").toInt + if (useNetty) diskStore.startShuffleBlockSender(nettyPortConfig) else 0 + } val connectionManager = new ConnectionManager(0) implicit val futureExecContext = connectionManager.futureExecContext @@ -824,10 +825,24 @@ private[spark] class BlockManager( } } + /** + * Remove all blocks belonging to the given RDD. + * @return The number of blocks removed. + */ + def removeRdd(rddId: Int): Int = { + // TODO: Instead of doing a linear scan on the blockInfo map, create another map that maps + // from RDD.id to blocks. + logInfo("Removing RDD " + rddId) + val rddPrefix = "rdd_" + rddId + "_" + val blocksToRemove = blockInfo.filter(_._1.startsWith(rddPrefix)).map(_._1) + blocksToRemove.foreach(blockId => removeBlock(blockId, false)) + blocksToRemove.size + } + /** * Remove a block from both memory and disk. */ - def removeBlock(blockId: String) { + def removeBlock(blockId: String, tellMaster: Boolean = true) { logInfo("Removing block " + blockId) val info = blockInfo.get(blockId).orNull if (info != null) info.synchronized { @@ -839,7 +854,7 @@ private[spark] class BlockManager( "the disk or memory store") } blockInfo.remove(blockId) - if (info.tellMaster) { + if (tellMaster && info.tellMaster) { reportBlockStatus(blockId, info) } } else { diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index ac26c16867..7099e40618 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -1,19 +1,11 @@ package spark.storage -import java.io._ -import java.util.{HashMap => JHashMap} - -import scala.collection.JavaConverters._ -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.util.Random - -import akka.actor.{Actor, ActorRef, ActorSystem, Props} -import akka.dispatch.Await +import akka.actor.ActorRef +import akka.dispatch.{Await, Future} import akka.pattern.ask -import akka.util.{Duration, Timeout} -import akka.util.duration._ +import akka.util.Duration -import spark.{Logging, SparkException, Utils} +import spark.{Logging, SparkException} private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging { @@ -91,15 +83,28 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi /** * Remove all blocks belonging to the given RDD. */ - def removeRdd(rddId: Int) { - val rddBlockPrefix = "rdd_" + rddId + "_" - // Get the list of blocks in block manager, and remove ones that are part of this RDD. - // The runtime complexity is linear to the number of blocks persisted in the cluster. - // It could be expensive if the cluster is large and has a lot of blocks persisted. - getStorageStatus.flatMap(_.blocks).foreach { case(blockId, status) => - if (blockId.startsWith(rddBlockPrefix)) { - removeBlock(blockId) - } + def removeRdd(rddId: Int, blocking: Boolean) { + // The logic to remove an RDD is somewhat complicated: + // 1. Send BlockManagerMasterActor a RemoveRdd message. + // 2. Upon receiving the RemoveRdd message, BlockManagerMasterActor will forward the message + // to all workers to remove blocks belonging to the RDD, and return a Future for the results. + // 3. The Future is sent back here, and on successful completion of the Future, this function + // sends a RemoveRddMetaData message to BlockManagerMasterActor. + // 4. Upon receiving the RemoveRddMetaData message, BlockManagerMasterActor will delete the meta + // data for the given RDD. + // + // The reason we are doing it this way is to reduce the amount of messages the driver sends. + // The number of messages that need to be sent is only the number of workers the cluster has, + // rather than the number of blocks in the cluster. Note that we can further reduce the number + // of messages by tracking for a given RDD, where are its blocks. Then we can send only to the + // workers that have the given RDD. But this remains future work. + val future = askDriverWithReply[Future[Seq[Int]]](RemoveRdd(rddId)) + future onComplete { + case Left(throwable) => logError("Failed to remove RDD " + rddId, throwable) + case Right(numBlocks) => tell(RemoveRddMetaData(rddId, numBlocks.sum)) + } + if (blocking) { + Await.result(future, timeout) } } @@ -114,7 +119,7 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi } def getStorageStatus: Array[StorageStatus] = { - askDriverWithReply[ArrayBuffer[StorageStatus]](GetStorageStatus).toArray + askDriverWithReply[Array[StorageStatus]](GetStorageStatus) } /** Stop the driver actor, called only on the Spark driver node */ diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index 9b64f95df8..00aa97bf78 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -2,15 +2,16 @@ package spark.storage import java.util.{HashMap => JHashMap} -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable import scala.collection.JavaConversions._ -import scala.util.Random import akka.actor.{Actor, ActorRef, Cancellable} -import akka.util.{Duration, Timeout} +import akka.dispatch.Future +import akka.pattern.ask +import akka.util.Duration import akka.util.duration._ -import spark.{Logging, Utils} +import spark.{Logging, Utils, SparkException} /** * BlockManagerMasterActor is an actor on the master node to track statuses of @@ -21,13 +22,16 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { // Mapping from block manager id to the block manager's information. private val blockManagerInfo = - new HashMap[BlockManagerId, BlockManagerMasterActor.BlockManagerInfo] + new mutable.HashMap[BlockManagerId, BlockManagerMasterActor.BlockManagerInfo] // Mapping from executor ID to block manager ID. - private val blockManagerIdByExecutor = new HashMap[String, BlockManagerId] + private val blockManagerIdByExecutor = new mutable.HashMap[String, BlockManagerId] // Mapping from block id to the set of block managers that have the block. - private val blockLocations = new JHashMap[String, Pair[Int, HashSet[BlockManagerId]]] + private val blockLocations = new JHashMap[String, mutable.HashSet[BlockManagerId]] + + val akkaTimeout = Duration.create( + System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") initLogging() @@ -50,28 +54,38 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { def receive = { case RegisterBlockManager(blockManagerId, maxMemSize, slaveActor) => register(blockManagerId, maxMemSize, slaveActor) + sender ! true case UpdateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) => + // TODO: Ideally we want to handle all the message replies in receive instead of in the + // individual private methods. updateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) case GetLocations(blockId) => - getLocations(blockId) + sender ! getLocations(blockId) case GetLocationsMultipleBlockIds(blockIds) => - getLocationsMultipleBlockIds(blockIds) + sender ! getLocationsMultipleBlockIds(blockIds) case GetPeers(blockManagerId, size) => - getPeersDeterministic(blockManagerId, size) - /*getPeers(blockManagerId, size)*/ + sender ! getPeers(blockManagerId, size) case GetMemoryStatus => - getMemoryStatus + sender ! memoryStatus case GetStorageStatus => - getStorageStatus + sender ! storageStatus + + case RemoveRdd(rddId) => + sender ! removeRdd(rddId) + + case RemoveRddMetaData(rddId, numBlocks) => + removeRddMetaData(rddId, numBlocks) + sender ! true case RemoveBlock(blockId) => - removeBlock(blockId) + removeBlockFromWorkers(blockId) + sender ! true case RemoveExecutor(execId) => removeExecutor(execId) @@ -81,7 +95,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { logInfo("Stopping BlockManagerMaster") sender ! true if (timeoutCheckingTask != null) { - timeoutCheckingTask.cancel + timeoutCheckingTask.cancel() } context.stop(self) @@ -89,13 +103,34 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { expireDeadHosts() case HeartBeat(blockManagerId) => - heartBeat(blockManagerId) + sender ! heartBeat(blockManagerId) case other => - logInfo("Got unknown message: " + other) + logWarning("Got unknown message: " + other) + } + + private def removeRdd(rddId: Int): Future[Seq[Int]] = { + // Ask the slaves to remove the RDD, and put the result in a sequence of Futures. + // The dispatcher is used as an implicit argument into the Future sequence construction. + import context.dispatcher + Future.sequence(blockManagerInfo.values.map { bm => + bm.slaveActor.ask(RemoveRdd(rddId))(akkaTimeout).mapTo[Int] + }.toSeq) + } + + private def removeRddMetaData(rddId: Int, numBlocks: Int) { + val prefix = "rdd_" + rddId + "_" + // Find all blocks for the given RDD, remove the block from both blockLocations and + // the blockManagerInfo that is tracking the blocks. + val blocks = blockLocations.keySet().filter(_.startsWith(prefix)) + blocks.foreach { blockId => + val bms: mutable.HashSet[BlockManagerId] = blockLocations.get(blockId) + bms.foreach(bm => blockManagerInfo.get(bm).foreach(_.removeBlock(blockId))) + blockLocations.remove(blockId) + } } - def removeBlockManager(blockManagerId: BlockManagerId) { + private def removeBlockManager(blockManagerId: BlockManagerId) { val info = blockManagerInfo(blockManagerId) // Remove the block manager from blockManagerIdByExecutor. @@ -106,7 +141,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { val iterator = info.blocks.keySet.iterator while (iterator.hasNext) { val blockId = iterator.next - val locations = blockLocations.get(blockId)._2 + val locations = blockLocations.get(blockId) locations -= blockManagerId if (locations.size == 0) { blockLocations.remove(locations) @@ -114,11 +149,11 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { } } - def expireDeadHosts() { + private def expireDeadHosts() { logTrace("Checking for hosts with no recent heart beats in BlockManagerMaster.") val now = System.currentTimeMillis() val minSeenTime = now - slaveTimeout - val toRemove = new HashSet[BlockManagerId] + val toRemove = new mutable.HashSet[BlockManagerId] for (info <- blockManagerInfo.values) { if (info.lastSeenMs < minSeenTime) { logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats: " + @@ -129,31 +164,26 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { toRemove.foreach(removeBlockManager) } - def removeExecutor(execId: String) { + private def removeExecutor(execId: String) { logInfo("Trying to remove executor " + execId + " from BlockManagerMaster.") blockManagerIdByExecutor.get(execId).foreach(removeBlockManager) - sender ! true } - def heartBeat(blockManagerId: BlockManagerId) { + private def heartBeat(blockManagerId: BlockManagerId): Boolean = { if (!blockManagerInfo.contains(blockManagerId)) { - if (blockManagerId.executorId == "" && !isLocal) { - sender ! true - } else { - sender ! false - } + blockManagerId.executorId == "" && !isLocal } else { blockManagerInfo(blockManagerId).updateLastSeenMs() - sender ! true + true } } // Remove a block from the slaves that have it. This can only be used to remove // blocks that the master knows about. - private def removeBlock(blockId: String) { - val block = blockLocations.get(blockId) - if (block != null) { - block._2.foreach { blockManagerId: BlockManagerId => + private def removeBlockFromWorkers(blockId: String) { + val locations = blockLocations.get(blockId) + if (locations != null) { + locations.foreach { blockManagerId: BlockManagerId => val blockManager = blockManagerInfo.get(blockManagerId) if (blockManager.isDefined) { // Remove the block from the slave's BlockManager. @@ -163,23 +193,20 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { } } } - sender ! true } // Return a map from the block manager id to max memory and remaining memory. - private def getMemoryStatus() { - val res = blockManagerInfo.map { case(blockManagerId, info) => + private def memoryStatus: Map[BlockManagerId, (Long, Long)] = { + blockManagerInfo.map { case(blockManagerId, info) => (blockManagerId, (info.maxMem, info.remainingMem)) }.toMap - sender ! res } - private def getStorageStatus() { - val res = blockManagerInfo.map { case(blockManagerId, info) => + private def storageStatus: Array[StorageStatus] = { + blockManagerInfo.map { case(blockManagerId, info) => import collection.JavaConverters._ StorageStatus(blockManagerId, info.maxMem, info.blocks.asScala.toMap) - } - sender ! res + }.toArray } private def register(id: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { @@ -188,7 +215,8 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { } else if (!blockManagerInfo.contains(id)) { blockManagerIdByExecutor.get(id.executorId) match { case Some(manager) => - // A block manager of the same host name already exists + // A block manager of the same executor already exists. + // This should never happen. Let's just quit. logError("Got two different block manager registrations on " + id.executorId) System.exit(1) case None => @@ -197,7 +225,6 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { blockManagerInfo(id) = new BlockManagerMasterActor.BlockManagerInfo( id, System.currentTimeMillis(), maxMemSize, slaveActor) } - sender ! true } private def updateBlockInfo( @@ -226,12 +253,12 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { blockManagerInfo(blockManagerId).updateBlockInfo(blockId, storageLevel, memSize, diskSize) - var locations: HashSet[BlockManagerId] = null + var locations: mutable.HashSet[BlockManagerId] = null if (blockLocations.containsKey(blockId)) { - locations = blockLocations.get(blockId)._2 + locations = blockLocations.get(blockId) } else { - locations = new HashSet[BlockManagerId] - blockLocations.put(blockId, (storageLevel.replication, locations)) + locations = new mutable.HashSet[BlockManagerId] + blockLocations.put(blockId, locations) } if (storageLevel.isValid) { @@ -247,70 +274,24 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { sender ! true } - private def getLocations(blockId: String) { - val startTimeMs = System.currentTimeMillis() - val tmp = " " + blockId + " " - if (blockLocations.containsKey(blockId)) { - var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] - res.appendAll(blockLocations.get(blockId)._2) - sender ! res.toSeq - } else { - var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] - sender ! res - } - } - - private def getLocationsMultipleBlockIds(blockIds: Array[String]) { - def getLocations(blockId: String): Seq[BlockManagerId] = { - val tmp = blockId - if (blockLocations.containsKey(blockId)) { - var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] - res.appendAll(blockLocations.get(blockId)._2) - return res.toSeq - } else { - var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] - return res.toSeq - } - } - - var res: ArrayBuffer[Seq[BlockManagerId]] = new ArrayBuffer[Seq[BlockManagerId]] - for (blockId <- blockIds) { - res.append(getLocations(blockId)) - } - sender ! res.toSeq + private def getLocations(blockId: String): Seq[BlockManagerId] = { + if (blockLocations.containsKey(blockId)) blockLocations.get(blockId).toSeq else Seq.empty } - private def getPeers(blockManagerId: BlockManagerId, size: Int) { - var peers: Array[BlockManagerId] = blockManagerInfo.keySet.toArray - var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] - res.appendAll(peers) - res -= blockManagerId - val rand = new Random(System.currentTimeMillis()) - while (res.length > size) { - res.remove(rand.nextInt(res.length)) - } - sender ! res.toSeq + private def getLocationsMultipleBlockIds(blockIds: Array[String]): Seq[Seq[BlockManagerId]] = { + blockIds.map(blockId => getLocations(blockId)) } - private def getPeersDeterministic(blockManagerId: BlockManagerId, size: Int) { - var peers: Array[BlockManagerId] = blockManagerInfo.keySet.toArray - var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId] + private def getPeers(blockManagerId: BlockManagerId, size: Int): Seq[BlockManagerId] = { + val peers: Array[BlockManagerId] = blockManagerInfo.keySet.toArray val selfIndex = peers.indexOf(blockManagerId) if (selfIndex == -1) { - throw new Exception("Self index for " + blockManagerId + " not found") + throw new SparkException("Self index for " + blockManagerId + " not found") } // Note that this logic will select the same node multiple times if there aren't enough peers - var index = selfIndex - while (res.size < size) { - index += 1 - if (index == selfIndex) { - throw new Exception("More peer expected than available") - } - res += peers(index % peers.size) - } - sender ! res.toSeq + Array.tabulate[BlockManagerId](size) { i => peers((selfIndex + i + 1) % peers.length) }.toSeq } } @@ -384,6 +365,13 @@ object BlockManagerMasterActor { } } + def removeBlock(blockId: String) { + if (_blocks.containsKey(blockId)) { + _remainingMem += _blocks.get(blockId).memSize + _blocks.remove(blockId) + } + } + def remainingMem: Long = _remainingMem def lastSeenMs: Long = _lastSeenMs diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/spark/storage/BlockManagerMessages.scala index cff48d9909..88268fd41b 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMessages.scala @@ -16,6 +16,12 @@ sealed trait ToBlockManagerSlave private[spark] case class RemoveBlock(blockId: String) extends ToBlockManagerSlave +// Remove all blocks belonging to a specific RDD. +private[spark] case class RemoveRdd(rddId: Int) extends ToBlockManagerSlave + +// Remove the meta data for a RDD. This is only sent to the master by the master. +private[spark] case class RemoveRddMetaData(rddId: Int, numBlocks: Int) extends ToBlockManagerMaster + ////////////////////////////////////////////////////////////////////////////////// // Messages from slaves to the master. diff --git a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala index f570cdc52d..b264d1deb5 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala @@ -11,6 +11,12 @@ import spark.{Logging, SparkException, Utils} */ class BlockManagerSlaveActor(blockManager: BlockManager) extends Actor { override def receive = { - case RemoveBlock(blockId) => blockManager.removeBlock(blockId) + + case RemoveBlock(blockId) => + blockManager.removeBlock(blockId) + + case RemoveRdd(rddId) => + val numBlocksRemoved = blockManager.removeRdd(rddId) + sender ! numBlocksRemoved } } diff --git a/core/src/main/scala/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/spark/storage/BlockManagerWorker.scala index 15225f93a6..3057ade233 100644 --- a/core/src/main/scala/spark/storage/BlockManagerWorker.scala +++ b/core/src/main/scala/spark/storage/BlockManagerWorker.scala @@ -2,13 +2,7 @@ package spark.storage import java.nio.ByteBuffer -import scala.actors._ -import scala.actors.Actor._ -import scala.actors.remote._ -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.util.Random - -import spark.{Logging, Utils, SparkEnv} +import spark.{Logging, Utils} import spark.network._ /** @@ -88,8 +82,6 @@ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends private[spark] object BlockManagerWorker extends Logging { private var blockManagerWorker: BlockManagerWorker = null - private val DATA_TRANSFER_TIME_OUT_MS: Long = 500 - private val REQUEST_RETRY_INTERVAL_MS: Long = 1000 initLogging() diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index bff2475686..b9d5f9668e 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -15,10 +15,10 @@ import org.scalatest.time.SpanSugar._ import spark.JavaSerializer import spark.KryoSerializer import spark.SizeEstimator -import spark.Utils import spark.util.AkkaUtils import spark.util.ByteBufferInputStream + class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester { var store: BlockManager = null var store2: BlockManager = null @@ -124,7 +124,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT // Putting a1, a2 and a3 in memory and telling master only about a1 and a2 store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY) - store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY, false) + store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY, tellMaster = false) // Checking whether blocks are in memory assert(store.getSingle("a1") != None, "a1 was not in store") @@ -170,7 +170,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT // Putting a1, a2 and a3 in memory and telling master only about a1 and a2 store.putSingle("a1-to-remove", a1, StorageLevel.MEMORY_ONLY) store.putSingle("a2-to-remove", a2, StorageLevel.MEMORY_ONLY) - store.putSingle("a3-to-remove", a3, StorageLevel.MEMORY_ONLY, false) + store.putSingle("a3-to-remove", a3, StorageLevel.MEMORY_ONLY, tellMaster = false) // Checking whether blocks are in memory and memory size val memStatus = master.getMemoryStatus.head._2 @@ -218,7 +218,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store.putSingle("rdd_0_0", a1, StorageLevel.MEMORY_ONLY) store.putSingle("rdd_0_1", a2, StorageLevel.MEMORY_ONLY) store.putSingle("nonrddblock", a3, StorageLevel.MEMORY_ONLY) - master.removeRdd(0) + master.removeRdd(0, blocking = false) eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { store.getSingle("rdd_0_0") should be (None) @@ -232,6 +232,14 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store.getSingle("nonrddblock") should not be (None) master.getLocations("nonrddblock") should have size (1) } + + store.putSingle("rdd_0_0", a1, StorageLevel.MEMORY_ONLY) + store.putSingle("rdd_0_1", a2, StorageLevel.MEMORY_ONLY) + master.removeRdd(0, blocking = true) + store.getSingle("rdd_0_0") should be (None) + master.getLocations("rdd_0_0") should have size 0 + store.getSingle("rdd_0_1") should be (None) + master.getLocations("rdd_0_1") should have size 0 } test("reregistration on heart beat") { @@ -262,7 +270,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT master.removeExecutor(store.blockManagerId.executorId) assert(master.getLocations("a1").size == 0, "a1 was not removed from master") - store.putSingle("a2", a1, StorageLevel.MEMORY_ONLY) + store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY) store.waitForAsyncReregister() assert(master.getLocations("a1").size > 0, "a1 was not reregistered with master") @@ -280,7 +288,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT master.removeExecutor(store.blockManagerId.executorId) val t1 = new Thread { override def run() { - store.put("a2", a2.iterator, StorageLevel.MEMORY_ONLY, true) + store.put("a2", a2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) } } val t2 = new Thread { @@ -490,9 +498,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, true) - store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY, true) - store.put("list3", list3.iterator, StorageLevel.MEMORY_ONLY, true) + store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.put("list3", list3.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.get("list2") != None, "list2 was not in store") assert(store.get("list2").get.size == 2) assert(store.get("list3") != None, "list3 was not in store") @@ -501,7 +509,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(store.get("list2") != None, "list2 was not in store") assert(store.get("list2").get.size == 2) // At this point list2 was gotten last, so LRU will getSingle rid of list3 - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, true) + store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.get("list1") != None, "list1 was not in store") assert(store.get("list1").get.size == 2) assert(store.get("list2") != None, "list2 was not in store") @@ -516,9 +524,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT val list3 = List(new Array[Byte](200), new Array[Byte](200)) val list4 = List(new Array[Byte](200), new Array[Byte](200)) // First store list1 and list2, both in memory, and list3, on disk only - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY_SER, true) - store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY_SER, true) - store.put("list3", list3.iterator, StorageLevel.DISK_ONLY, true) + store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) + store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) + store.put("list3", list3.iterator, StorageLevel.DISK_ONLY, tellMaster = true) // At this point LRU should not kick in because list3 is only on disk assert(store.get("list1") != None, "list2 was not in store") assert(store.get("list1").get.size === 2) @@ -533,7 +541,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(store.get("list3") != None, "list1 was not in store") assert(store.get("list3").get.size === 2) // Now let's add in list4, which uses both disk and memory; list1 should drop out - store.put("list4", list4.iterator, StorageLevel.MEMORY_AND_DISK_SER, true) + store.put("list4", list4.iterator, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true) assert(store.get("list1") === None, "list1 was in store") assert(store.get("list2") != None, "list3 was not in store") assert(store.get("list2").get.size === 2) -- cgit v1.2.3 From de1167bf2c32d52c865a4a0c7213b665ebd61f93 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 31 May 2013 15:54:57 -0700 Subject: Incorporated Charles' feedback to put rdd metadata removal in BlockManagerMasterActor. --- .../scala/spark/storage/BlockManagerMaster.scala | 21 +++------------------ .../spark/storage/BlockManagerMasterActor.scala | 22 ++++++++++------------ .../scala/spark/storage/BlockManagerMessages.scala | 3 --- 3 files changed, 13 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 7099e40618..58888b1ebb 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -84,24 +84,9 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi * Remove all blocks belonging to the given RDD. */ def removeRdd(rddId: Int, blocking: Boolean) { - // The logic to remove an RDD is somewhat complicated: - // 1. Send BlockManagerMasterActor a RemoveRdd message. - // 2. Upon receiving the RemoveRdd message, BlockManagerMasterActor will forward the message - // to all workers to remove blocks belonging to the RDD, and return a Future for the results. - // 3. The Future is sent back here, and on successful completion of the Future, this function - // sends a RemoveRddMetaData message to BlockManagerMasterActor. - // 4. Upon receiving the RemoveRddMetaData message, BlockManagerMasterActor will delete the meta - // data for the given RDD. - // - // The reason we are doing it this way is to reduce the amount of messages the driver sends. - // The number of messages that need to be sent is only the number of workers the cluster has, - // rather than the number of blocks in the cluster. Note that we can further reduce the number - // of messages by tracking for a given RDD, where are its blocks. Then we can send only to the - // workers that have the given RDD. But this remains future work. val future = askDriverWithReply[Future[Seq[Int]]](RemoveRdd(rddId)) - future onComplete { - case Left(throwable) => logError("Failed to remove RDD " + rddId, throwable) - case Right(numBlocks) => tell(RemoveRddMetaData(rddId, numBlocks.sum)) + future onFailure { + case e: Throwable => logError("Failed to remove RDD " + rddId, e) } if (blocking) { Await.result(future, timeout) @@ -156,7 +141,7 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi val future = driverActor.ask(message)(timeout) val result = Await.result(future, timeout) if (result == null) { - throw new Exception("BlockManagerMaster returned null") + throw new SparkException("BlockManagerMaster returned null") } return result.asInstanceOf[T] } catch { diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index 00aa97bf78..2d05e0ccf1 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -79,10 +79,6 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { case RemoveRdd(rddId) => sender ! removeRdd(rddId) - case RemoveRddMetaData(rddId, numBlocks) => - removeRddMetaData(rddId, numBlocks) - sender ! true - case RemoveBlock(blockId) => removeBlockFromWorkers(blockId) sender ! true @@ -110,15 +106,9 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { } private def removeRdd(rddId: Int): Future[Seq[Int]] = { - // Ask the slaves to remove the RDD, and put the result in a sequence of Futures. - // The dispatcher is used as an implicit argument into the Future sequence construction. - import context.dispatcher - Future.sequence(blockManagerInfo.values.map { bm => - bm.slaveActor.ask(RemoveRdd(rddId))(akkaTimeout).mapTo[Int] - }.toSeq) - } + // First remove the metadata for the given RDD, and then asynchronously remove the blocks + // from the slaves. - private def removeRddMetaData(rddId: Int, numBlocks: Int) { val prefix = "rdd_" + rddId + "_" // Find all blocks for the given RDD, remove the block from both blockLocations and // the blockManagerInfo that is tracking the blocks. @@ -128,6 +118,14 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { bms.foreach(bm => blockManagerInfo.get(bm).foreach(_.removeBlock(blockId))) blockLocations.remove(blockId) } + + // Ask the slaves to remove the RDD, and put the result in a sequence of Futures. + // The dispatcher is used as an implicit argument into the Future sequence construction. + import context.dispatcher + val removeMsg = RemoveRdd(rddId) + Future.sequence(blockManagerInfo.values.map { bm => + bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Int] + }.toSeq) } private def removeBlockManager(blockManagerId: BlockManagerId) { diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/spark/storage/BlockManagerMessages.scala index 88268fd41b..0010726c8d 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMessages.scala @@ -19,9 +19,6 @@ case class RemoveBlock(blockId: String) extends ToBlockManagerSlave // Remove all blocks belonging to a specific RDD. private[spark] case class RemoveRdd(rddId: Int) extends ToBlockManagerSlave -// Remove the meta data for a RDD. This is only sent to the master by the master. -private[spark] case class RemoveRddMetaData(rddId: Int, numBlocks: Int) extends ToBlockManagerMaster - ////////////////////////////////////////////////////////////////////////////////// // Messages from slaves to the master. -- cgit v1.2.3 From 9f84315c055d7a53da8787eb26b336726fc33e8a Mon Sep 17 00:00:00 2001 From: Gavin Li Date: Sat, 1 Jun 2013 00:26:10 +0000 Subject: enhance pipe to support what we can do in hadoop streaming --- core/src/main/scala/spark/RDD.scala | 18 ++++++++++++++++++ core/src/main/scala/spark/rdd/PipedRDD.scala | 25 +++++++++++++++++++++++-- 2 files changed, 41 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index dde131696f..5a41db23c2 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -348,17 +348,35 @@ abstract class RDD[T: ClassManifest]( */ def pipe(command: String): RDD[String] = new PipedRDD(this, command) + /** + * Return an RDD created by piping elements to a forked external process. + */ + def pipe(command: String, transform: (T,String => Unit) => Any, arguments: Seq[String]): RDD[String] = + new PipedRDD(this, command, transform, arguments) + /** * Return an RDD created by piping elements to a forked external process. */ def pipe(command: Seq[String]): RDD[String] = new PipedRDD(this, command) + /** + * Return an RDD created by piping elements to a forked external process. + */ + def pipe(command: Seq[String], transform: (T,String => Unit) => Any, arguments: Seq[String]): RDD[String] = + new PipedRDD(this, command, transform, arguments) + /** * Return an RDD created by piping elements to a forked external process. */ def pipe(command: Seq[String], env: Map[String, String]): RDD[String] = new PipedRDD(this, command, env) + /** + * Return an RDD created by piping elements to a forked external process. + */ + def pipe(command: Seq[String], env: Map[String, String], transform: (T,String => Unit) => Any, arguments: Seq[String]): RDD[String] = + new PipedRDD(this, command, env, transform, arguments) + /** * Return a new RDD by applying a function to each partition of this RDD. */ diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala index 962a1b21ad..969404c95f 100644 --- a/core/src/main/scala/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/spark/rdd/PipedRDD.scala @@ -18,14 +18,21 @@ import spark.{RDD, SparkEnv, Partition, TaskContext} class PipedRDD[T: ClassManifest]( prev: RDD[T], command: Seq[String], - envVars: Map[String, String]) + envVars: Map[String, String], + transform: (T, String => Unit) => Any, + arguments: Seq[String] + ) extends RDD[String](prev) { + def this(prev: RDD[T], command: Seq[String], envVars : Map[String, String]) = this(prev, command, envVars, null, null) def this(prev: RDD[T], command: Seq[String]) = this(prev, command, Map()) + def this(prev: RDD[T], command: Seq[String], transform: (T,String => Unit) => Any, arguments: Seq[String]) = this(prev, command, Map(), transform, arguments) // Similar to Runtime.exec(), if we are given a single string, split it into words // using a standard StringTokenizer (i.e. by spaces) def this(prev: RDD[T], command: String) = this(prev, PipedRDD.tokenize(command)) + def this(prev: RDD[T], command: String, transform: (T,String => Unit) => Any, arguments: Seq[String]) = this(prev, PipedRDD.tokenize(command), Map(), transform, arguments) + override def getPartitions: Array[Partition] = firstParent[T].partitions @@ -52,8 +59,22 @@ class PipedRDD[T: ClassManifest]( override def run() { SparkEnv.set(env) val out = new PrintWriter(proc.getOutputStream) + + // input the arguments firstly + if ( arguments != null) { + for (elem <- arguments) { + out.println(elem) + } + // ^A \n as the marker of the end of the arguments + out.println("\u0001") + } for (elem <- firstParent[T].iterator(split, context)) { - out.println(elem) + if (transform != null) { + transform(elem, out.println(_)) + } + else { + out.println(elem) + } } out.close() } -- cgit v1.2.3 From 91aca9224936da84b16ea789cb81914579a0db03 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 31 May 2013 23:21:38 -0700 Subject: Another round of Netty fixes. 1. Avoid race condition between stop and copier completion 2. Handle socket exceptions by reporting them and filling in a failed FetchResult --- .../main/java/spark/network/netty/FileClient.java | 24 +++------ .../spark/network/netty/FileClientHandler.java | 8 +++ .../scala/spark/network/netty/ShuffleCopier.scala | 62 ++++++++++++++-------- .../scala/spark/storage/BlockFetcherIterator.scala | 9 ++-- 4 files changed, 58 insertions(+), 45 deletions(-) diff --git a/core/src/main/java/spark/network/netty/FileClient.java b/core/src/main/java/spark/network/netty/FileClient.java index 3a62dacbc8..9c9b976ebe 100644 --- a/core/src/main/java/spark/network/netty/FileClient.java +++ b/core/src/main/java/spark/network/netty/FileClient.java @@ -8,9 +8,12 @@ import io.netty.channel.ChannelOption; import io.netty.channel.oio.OioEventLoopGroup; import io.netty.channel.socket.oio.OioSocketChannel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class FileClient { + private Logger LOG = LoggerFactory.getLogger(this.getClass().getName()); private FileClientHandler handler = null; private Channel channel = null; private Bootstrap bootstrap = null; @@ -25,25 +28,10 @@ class FileClient { .channel(OioSocketChannel.class) .option(ChannelOption.SO_KEEPALIVE, true) .option(ChannelOption.TCP_NODELAY, true) + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, 0) // Disable connect timeout .handler(new FileClientChannelInitializer(handler)); } - public static final class ChannelCloseListener implements ChannelFutureListener { - private FileClient fc = null; - - public ChannelCloseListener(FileClient fc){ - this.fc = fc; - } - - @Override - public void operationComplete(ChannelFuture future) { - if (fc.bootstrap!=null){ - fc.bootstrap.shutdown(); - fc.bootstrap = null; - } - } - } - public void connect(String host, int port) { try { // Start the connection attempt. @@ -58,8 +46,8 @@ class FileClient { public void waitForClose() { try { channel.closeFuture().sync(); - } catch (InterruptedException e){ - e.printStackTrace(); + } catch (InterruptedException e) { + LOG.warn("FileClient interrupted", e); } } diff --git a/core/src/main/java/spark/network/netty/FileClientHandler.java b/core/src/main/java/spark/network/netty/FileClientHandler.java index 2069dee5ca..9fc9449827 100644 --- a/core/src/main/java/spark/network/netty/FileClientHandler.java +++ b/core/src/main/java/spark/network/netty/FileClientHandler.java @@ -9,7 +9,14 @@ abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter { private FileHeader currentHeader = null; + private volatile boolean handlerCalled = false; + + public boolean isComplete() { + return handlerCalled; + } + public abstract void handle(ChannelHandlerContext ctx, ByteBuf in, FileHeader header); + public abstract void handleError(String blockId); @Override public ByteBuf newInboundBuffer(ChannelHandlerContext ctx) { @@ -26,6 +33,7 @@ abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter { // get file if(in.readableBytes() >= currentHeader.fileLen()) { handle(ctx, in, currentHeader); + handlerCalled = true; currentHeader = null; ctx.close(); } diff --git a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala index a91f5a886d..8ec46d42fa 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala @@ -9,19 +9,35 @@ import io.netty.util.CharsetUtil import spark.Logging import spark.network.ConnectionManagerId +import scala.collection.JavaConverters._ + private[spark] class ShuffleCopier extends Logging { - def getBlock(cmId: ConnectionManagerId, blockId: String, + def getBlock(host: String, port: Int, blockId: String, resultCollectCallback: (String, Long, ByteBuf) => Unit) { val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) val fc = new FileClient(handler) - fc.init() - fc.connect(cmId.host, cmId.port) - fc.sendRequest(blockId) - fc.waitForClose() - fc.close() + try { + fc.init() + fc.connect(host, port) + fc.sendRequest(blockId) + fc.waitForClose() + fc.close() + } catch { + // Handle any socket-related exceptions in FileClient + case e: Exception => { + logError("Shuffle copy of block " + blockId + " from " + host + ":" + port + + " failed", e) + handler.handleError(blockId) + } + } + } + + def getBlock(cmId: ConnectionManagerId, blockId: String, + resultCollectCallback: (String, Long, ByteBuf) => Unit) { + getBlock(cmId.host, cmId.port, blockId, resultCollectCallback) } def getBlocks(cmId: ConnectionManagerId, @@ -44,20 +60,18 @@ private[spark] object ShuffleCopier extends Logging { logDebug("Received Block: " + header.blockId + " (" + header.fileLen + "B)"); resultCollectCallBack(header.blockId, header.fileLen.toLong, in.readBytes(header.fileLen)) } - } - def echoResultCollectCallBack(blockId: String, size: Long, content: ByteBuf) { - logInfo("File: " + blockId + " content is : \" " + content.toString(CharsetUtil.UTF_8) + "\"") + override def handleError(blockId: String) { + if (!isComplete) { + resultCollectCallBack(blockId, -1, null) + } + } } - def runGetBlock(host:String, port:Int, file:String){ - val handler = new ShuffleClientHandler(echoResultCollectCallBack) - val fc = new FileClient(handler) - fc.init(); - fc.connect(host, port) - fc.sendRequest(file) - fc.waitForClose(); - fc.close() + def echoResultCollectCallBack(blockId: String, size: Long, content: ByteBuf) { + if (size != -1) { + logInfo("File: " + blockId + " content is : \" " + content.toString(CharsetUtil.UTF_8) + "\"") + } } def main(args: Array[String]) { @@ -71,14 +85,16 @@ private[spark] object ShuffleCopier extends Logging { val threads = if (args.length > 3) args(3).toInt else 10 val copiers = Executors.newFixedThreadPool(80) - for (i <- Range(0, threads)) { - val runnable = new Runnable() { + val tasks = (for (i <- Range(0, threads)) yield { + Executors.callable(new Runnable() { def run() { - runGetBlock(host, port, file) + val copier = new ShuffleCopier() + copier.getBlock(host, port, file, echoResultCollectCallBack) } - } - copiers.execute(runnable) - } + }) + }).asJava + copiers.invokeAll(tasks) copiers.shutdown + System.exit(0) } } diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index 1d69d658f7..fac416a5b3 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -265,7 +265,7 @@ object BlockFetcherIterator { }).toList } - //keep this to interrupt the threads when necessary + // keep this to interrupt the threads when necessary private def stopCopiers() { for (copier <- copiers) { copier.interrupt() @@ -312,9 +312,10 @@ object BlockFetcherIterator { resultsGotten += 1 val result = results.take() // if all the results has been retrieved, shutdown the copiers - if (resultsGotten == _totalBlocks && copiers != null) { - stopCopiers() - } + // NO need to stop the copiers if we got all the blocks ? + // if (resultsGotten == _totalBlocks && copiers != null) { + // stopCopiers() + // } (result.blockId, if (result.failed) None else Some(result.deserialize())) } } -- cgit v1.2.3 From 038cfc1a9acb32f8c17d883ea64f8cbb324ed82c Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 31 May 2013 23:32:18 -0700 Subject: Make connect timeout configurable --- core/src/main/java/spark/network/netty/FileClient.java | 6 ++++-- core/src/main/scala/spark/network/netty/ShuffleCopier.scala | 3 ++- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/spark/network/netty/FileClient.java b/core/src/main/java/spark/network/netty/FileClient.java index 9c9b976ebe..517772202f 100644 --- a/core/src/main/java/spark/network/netty/FileClient.java +++ b/core/src/main/java/spark/network/netty/FileClient.java @@ -17,9 +17,11 @@ class FileClient { private FileClientHandler handler = null; private Channel channel = null; private Bootstrap bootstrap = null; + private int connectTimeout = 60*1000; // 1 min - public FileClient(FileClientHandler handler) { + public FileClient(FileClientHandler handler, int connectTimeout) { this.handler = handler; + this.connectTimeout = connectTimeout; } public void init() { @@ -28,7 +30,7 @@ class FileClient { .channel(OioSocketChannel.class) .option(ChannelOption.SO_KEEPALIVE, true) .option(ChannelOption.TCP_NODELAY, true) - .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, 0) // Disable connect timeout + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, connectTimeout) // Disable connect timeout .handler(new FileClientChannelInitializer(handler)); } diff --git a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala index 8ec46d42fa..afb2cdbb3a 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala @@ -18,7 +18,8 @@ private[spark] class ShuffleCopier extends Logging { resultCollectCallback: (String, Long, ByteBuf) => Unit) { val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) - val fc = new FileClient(handler) + val fc = new FileClient(handler, + System.getProperty("spark.shuffle.netty.connect.timeout", "60000").toInt) try { fc.init() fc.connect(host, port) -- cgit v1.2.3 From 3be7bdcefda13d67633f9b9f6d901722fd5649de Mon Sep 17 00:00:00 2001 From: Rohit Rai Date: Sat, 1 Jun 2013 19:32:17 +0530 Subject: Adding example to make Spark RDD from Cassandra --- .../main/scala/spark/examples/CassandraTest.scala | 154 +++++++++++++++++++++ project/SparkBuild.scala | 4 +- 2 files changed, 156 insertions(+), 2 deletions(-) create mode 100644 examples/src/main/scala/spark/examples/CassandraTest.scala diff --git a/examples/src/main/scala/spark/examples/CassandraTest.scala b/examples/src/main/scala/spark/examples/CassandraTest.scala new file mode 100644 index 0000000000..790b24e6f3 --- /dev/null +++ b/examples/src/main/scala/spark/examples/CassandraTest.scala @@ -0,0 +1,154 @@ +package spark.examples + +import org.apache.hadoop.mapreduce.Job +import org.apache.cassandra.hadoop.{ConfigHelper, ColumnFamilyInputFormat} +import org.apache.cassandra.thrift.{IndexExpression, SliceRange, SlicePredicate} +import spark.{RDD, SparkContext} +import SparkContext._ +import java.nio.ByteBuffer +import java.util.SortedMap +import org.apache.cassandra.db.IColumn +import org.apache.cassandra.utils.ByteBufferUtil +import scala.collection.JavaConversions._ + + +/* + * This example demonstrates using Spark with Cassandra with the New Hadoop API and Cassandra support for Hadoop. + * + * To run this example, run this file with the following command params - + * + * + * So if you want to run this on localhost this will be, + * local[3] localhost 9160 + * + * The example makes some assumptions: + * 1. You have already created a keyspace called casDemo and it has a column family named Words + * 2. There are column family has a column named "para" which has test content. + * + * You can create the content by running the following script at the bottom of this file with cassandra-cli. + * + */ +object CassandraTest { + def main(args: Array[String]) { + + //Get a SparkContext + val sc = new SparkContext(args(0), "casDemo") + + //Build the job configuration with ConfigHelper provided by Cassandra + val job = new Job() + job.setInputFormatClass(classOf[ColumnFamilyInputFormat]) + + ConfigHelper.setInputInitialAddress(job.getConfiguration(), args(1)) + + ConfigHelper.setInputRpcPort(job.getConfiguration(), args(2)) + + ConfigHelper.setInputColumnFamily(job.getConfiguration(), "casDemo", "Words") + + val predicate = new SlicePredicate() + val sliceRange = new SliceRange() + sliceRange.setStart(Array.empty[Byte]) + sliceRange.setFinish(Array.empty[Byte]) + predicate.setSlice_range(sliceRange) + ConfigHelper.setInputSlicePredicate(job.getConfiguration(), predicate) + + ConfigHelper.setInputPartitioner(job.getConfiguration(), "Murmur3Partitioner") + + //Make a new Hadoop RDD + val casRdd = sc.newAPIHadoopRDD(job.getConfiguration(), + classOf[ColumnFamilyInputFormat], + classOf[ByteBuffer], + classOf[SortedMap[ByteBuffer, IColumn]]) + + // Let us first get all the paragraphs from the retrieved rows + val paraRdd = casRdd flatMap { + case (key, value) => { + value.filter(v => ByteBufferUtil.string(v._1).compareTo("para") == 0).map(v => ByteBufferUtil.string(v._2.value())) + } + } + + //Lets get the word count in paras + val counts = paraRdd.flatMap(p => p.split(" ")).map(word => (word, 1)).reduceByKey(_ + _) + + counts.collect() foreach { + case(word, count) => println(word + ":" + count) + } + } +} + +/* +create keyspace casDemo; +use casDemo; + +create column family Words with comparator = UTF8Type; +update column family Words with column_metadata = [{column_name: book, validation_class: UTF8Type}, {column_name: para, validation_class: UTF8Type}]; + +assume Words keys as utf8; + +set Words['3musk001']['book'] = 'The Three Musketeers'; +set Words['3musk001']['para'] = 'On the first Monday of the month of April, 1625, the market town of + Meung, in which the author of ROMANCE OF THE ROSE was born, appeared to + be in as perfect a state of revolution as if the Huguenots had just made + a second La Rochelle of it. Many citizens, seeing the women flying + toward the High Street, leaving their children crying at the open doors, + hastened to don the cuirass, and supporting their somewhat uncertain + courage with a musket or a partisan, directed their steps toward the + hostelry of the Jolly Miller, before which was gathered, increasing + every minute, a compact group, vociferous and full of curiosity.'; + +set Words['3musk002']['book'] = 'The Three Musketeers'; +set Words['3musk002']['para'] = 'In those times panics were common, and few days passed without some city + or other registering in its archives an event of this kind. There were + nobles, who made war against each other; there was the king, who made + war against the cardinal; there was Spain, which made war against the + king. Then, in addition to these concealed or public, secret or open + wars, there were robbers, mendicants, Huguenots, wolves, and scoundrels, + who made war upon everybody. The citizens always took up arms readily + against thieves, wolves or scoundrels, often against nobles or + Huguenots, sometimes against the king, but never against cardinal or + Spain. It resulted, then, from this habit that on the said first Monday + of April, 1625, the citizens, on hearing the clamor, and seeing neither + the red-and-yellow standard nor the livery of the Duc de Richelieu, + rushed toward the hostel of the Jolly Miller. When arrived there, the + cause of the hubbub was apparent to all'; + +set Words['3musk003']['book'] = 'The Three Musketeers'; +set Words['3musk003']['para'] = 'You ought, I say, then, to husband the means you have, however large + the sum may be; but you ought also to endeavor to perfect yourself in + the exercises becoming a gentleman. I will write a letter today to the + Director of the Royal Academy, and tomorrow he will admit you without + any expense to yourself. Do not refuse this little service. Our + best-born and richest gentlemen sometimes solicit it without being able + to obtain it. You will learn horsemanship, swordsmanship in all its + branches, and dancing. You will make some desirable acquaintances; and + from time to time you can call upon me, just to tell me how you are + getting on, and to say whether I can be of further service to you.'; + + +set Words['thelostworld001']['book'] = 'The Lost World'; +set Words['thelostworld001']['para'] = 'She sat with that proud, delicate profile of hers outlined against the + red curtain. How beautiful she was! And yet how aloof! We had been + friends, quite good friends; but never could I get beyond the same + comradeship which I might have established with one of my + fellow-reporters upon the Gazette,--perfectly frank, perfectly kindly, + and perfectly unsexual. My instincts are all against a woman being too + frank and at her ease with me. It is no compliment to a man. Where + the real sex feeling begins, timidity and distrust are its companions, + heritage from old wicked days when love and violence went often hand in + hand. The bent head, the averted eye, the faltering voice, the wincing + figure--these, and not the unshrinking gaze and frank reply, are the + true signals of passion. Even in my short life I had learned as much + as that--or had inherited it in that race memory which we call instinct.'; + +set Words['thelostworld002']['book'] = 'The Lost World'; +set Words['thelostworld002']['para'] = 'I always liked McArdle, the crabbed, old, round-backed, red-headed news + editor, and I rather hoped that he liked me. Of course, Beaumont was + the real boss; but he lived in the rarefied atmosphere of some Olympian + height from which he could distinguish nothing smaller than an + international crisis or a split in the Cabinet. Sometimes we saw him + passing in lonely majesty to his inner sanctum, with his eyes staring + vaguely and his mind hovering over the Balkans or the Persian Gulf. He + was above and beyond us. But McArdle was his first lieutenant, and it + was he that we knew. The old man nodded as I entered the room, and he + pushed his spectacles far up on his bald forehead.'; + +*/ diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 0ea23b446f..5152b7b79b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -201,8 +201,8 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", - libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.11") - ) + libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.11", + "org.apache.cassandra" % "cassandra-all" % "1.2.5" exclude("com.google.guava", "guava") exclude("com.googlecode.concurrentlinkedhashmap", "concurrentlinkedhashmap-lru:1.3") exclude("com.ning","compress-lzf") exclude("io.netty","netty") exclude("jline","jline") exclude("log4j","log4j") exclude("org.apache.cassandra.deps", "avro"))) def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") -- cgit v1.2.3 From 81c2adc15c9e232846d4ad0adf14d007039409fa Mon Sep 17 00:00:00 2001 From: Rohit Rai Date: Sun, 2 Jun 2013 12:51:15 +0530 Subject: Removing infix call --- examples/src/main/scala/spark/examples/CassandraTest.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/examples/src/main/scala/spark/examples/CassandraTest.scala b/examples/src/main/scala/spark/examples/CassandraTest.scala index 790b24e6f3..49b940d8a7 100644 --- a/examples/src/main/scala/spark/examples/CassandraTest.scala +++ b/examples/src/main/scala/spark/examples/CassandraTest.scala @@ -60,7 +60,7 @@ object CassandraTest { classOf[SortedMap[ByteBuffer, IColumn]]) // Let us first get all the paragraphs from the retrieved rows - val paraRdd = casRdd flatMap { + val paraRdd = casRdd.flatMap { case (key, value) => { value.filter(v => ByteBufferUtil.string(v._1).compareTo("para") == 0).map(v => ByteBufferUtil.string(v._2.value())) } @@ -69,8 +69,8 @@ object CassandraTest { //Lets get the word count in paras val counts = paraRdd.flatMap(p => p.split(" ")).map(word => (word, 1)).reduceByKey(_ + _) - counts.collect() foreach { - case(word, count) => println(word + ":" + count) + counts.collect().foreach { + case (word, count) => println(word + ":" + count) } } } -- cgit v1.2.3 From 6d8423fd1b490d541f0ea379068b8954002d624f Mon Sep 17 00:00:00 2001 From: Rohit Rai Date: Sun, 2 Jun 2013 13:03:45 +0530 Subject: Adding deps to examples/pom.xml Fixing exclusion in examples deps in SparkBuild.scala --- examples/pom.xml | 35 +++++++++++++++++++++++++++++++++++ project/SparkBuild.scala | 2 +- 2 files changed, 36 insertions(+), 1 deletion(-) diff --git a/examples/pom.xml b/examples/pom.xml index c42d2bcdb9..b4c5251d68 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -34,6 +34,41 @@ scalacheck_${scala.version} test + + org.apache.cassandra + cassandra-all + 1.2.5 + + + com.google.guava + guava + + + com.googlecode.concurrentlinkedhashmap + concurrentlinkedhashmap-lru + + + com.ning + compress-lzf + + + io.netty + netty + + + jline + jline + + + log4j + log4j + + + org.apache.cassandra.deps + avro + + + target/scala-${scala.version}/classes diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5152b7b79b..7f3e223c2e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -202,7 +202,7 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.11", - "org.apache.cassandra" % "cassandra-all" % "1.2.5" exclude("com.google.guava", "guava") exclude("com.googlecode.concurrentlinkedhashmap", "concurrentlinkedhashmap-lru:1.3") exclude("com.ning","compress-lzf") exclude("io.netty","netty") exclude("jline","jline") exclude("log4j","log4j") exclude("org.apache.cassandra.deps", "avro"))) + "org.apache.cassandra" % "cassandra-all" % "1.2.5" exclude("com.google.guava", "guava") exclude("com.googlecode.concurrentlinkedhashmap", "concurrentlinkedhashmap-lru") exclude("com.ning","compress-lzf") exclude("io.netty","netty") exclude("jline","jline") exclude("log4j","log4j") exclude("org.apache.cassandra.deps", "avro"))) def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") -- cgit v1.2.3 From 4a9913d66a61ac9ef9cab0e08f6151dc2624fd11 Mon Sep 17 00:00:00 2001 From: Gavin Li Date: Sun, 2 Jun 2013 23:21:09 +0000 Subject: add ut for pipe enhancement --- core/src/test/scala/spark/PipedRDDSuite.scala | 31 +++++++++++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/spark/PipedRDDSuite.scala index a6344edf8f..ee55952a94 100644 --- a/core/src/test/scala/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/spark/PipedRDDSuite.scala @@ -19,6 +19,37 @@ class PipedRDDSuite extends FunSuite with LocalSparkContext { assert(c(3) === "4") } + test("advanced pipe") { + sc = new SparkContext("local", "test") + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + + val piped = nums.pipe(Seq("cat"), (i:Int, f: String=> Unit) => f(i + "_"), Array("0")) + + val c = piped.collect() + + assert(c.size === 8) + assert(c(0) === "0") + assert(c(1) === "\u0001") + assert(c(2) === "1_") + assert(c(3) === "2_") + assert(c(4) === "0") + assert(c(5) === "\u0001") + assert(c(6) === "3_") + assert(c(7) === "4_") + + val nums1 = sc.makeRDD(Array("a\t1", "b\t2", "a\t3", "b\t4"), 2) + val d = nums1.groupBy(str=>str.split("\t")(0)).pipe(Seq("cat"), (i:Tuple2[String, Seq[String]], f: String=> Unit) => {for (e <- i._2){ f(e + "_")}}, Array("0")).collect() + assert(d.size === 8) + assert(d(0) === "0") + assert(d(1) === "\u0001") + assert(d(2) === "b\t2_") + assert(d(3) === "b\t4_") + assert(d(4) === "0") + assert(d(5) === "\u0001") + assert(d(6) === "a\t1_") + assert(d(7) === "a\t3_") + } + test("pipe with env variable") { sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) -- cgit v1.2.3 From 606bb1b450064a2b909e4275ce45325dbbef4eca Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Fri, 31 May 2013 15:40:41 +0800 Subject: Fix schedulingAlgorithm bugs for unit test --- .../spark/scheduler/cluster/SchedulingAlgorithm.scala | 17 +++++++++++++---- .../scala/spark/scheduler/ClusterSchedulerSuite.scala | 9 ++++++--- 2 files changed, 19 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala index a5d6285c99..13120edf63 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala @@ -40,15 +40,24 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { val taskToWeightRatio1 = runningTasks1.toDouble / s1.weight.toDouble val taskToWeightRatio2 = runningTasks2.toDouble / s2.weight.toDouble var res:Boolean = true + var compare:Int = 0 if (s1Needy && !s2Needy) { - res = true + return true } else if (!s1Needy && s2Needy) { - res = false + return false } else if (s1Needy && s2Needy) { - res = minShareRatio1 <= minShareRatio2 + compare = minShareRatio1.compareTo(minShareRatio2) + } else { + compare = taskToWeightRatio1.compareTo(taskToWeightRatio2) + } + + if (compare < 0) { + res = true + } else if (compare > 0) { + res = false } else { - res = taskToWeightRatio1 <= taskToWeightRatio2 + return s1.name < s2.name } return res } diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala index a39418b716..c861597c6b 100644 --- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala @@ -88,7 +88,7 @@ class DummyTask(stageId: Int) extends Task[Int](stageId) } } -class ClusterSchedulerSuite extends FunSuite with LocalSparkContext { +class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging { def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: ClusterScheduler, taskSet: TaskSet): DummyTaskSetManager = { new DummyTaskSetManager(priority, stage, numTasks, cs , taskSet) @@ -96,8 +96,11 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext { def resourceOffer(rootPool: Pool): Int = { val taskSetQueue = rootPool.getSortedTaskSetQueue() - for (taskSet <- taskSetQueue) - { + /* Just for Test*/ + for (manager <- taskSetQueue) { + logInfo("parentName:%s, parent running tasks:%d, name:%s,runningTasks:%d".format(manager.parent.name, manager.parent.runningTasks, manager.name, manager.runningTasks)) + } + for (taskSet <- taskSetQueue) { taskSet.slaveOffer("execId_1", "hostname_1", 1) match { case Some(task) => return taskSet.stageId -- cgit v1.2.3 From 56c64c403383e90a5fd33b6a1f72527377d9bee0 Mon Sep 17 00:00:00 2001 From: Rohit Rai Date: Mon, 3 Jun 2013 12:48:35 +0530 Subject: A better way to read column value if you are sure the column exists in every row. --- examples/src/main/scala/spark/examples/CassandraTest.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/examples/src/main/scala/spark/examples/CassandraTest.scala b/examples/src/main/scala/spark/examples/CassandraTest.scala index 49b940d8a7..6b9fd502e2 100644 --- a/examples/src/main/scala/spark/examples/CassandraTest.scala +++ b/examples/src/main/scala/spark/examples/CassandraTest.scala @@ -10,6 +10,8 @@ import java.util.SortedMap import org.apache.cassandra.db.IColumn import org.apache.cassandra.utils.ByteBufferUtil import scala.collection.JavaConversions._ +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /* @@ -60,9 +62,9 @@ object CassandraTest { classOf[SortedMap[ByteBuffer, IColumn]]) // Let us first get all the paragraphs from the retrieved rows - val paraRdd = casRdd.flatMap { + val paraRdd = casRdd.map { case (key, value) => { - value.filter(v => ByteBufferUtil.string(v._1).compareTo("para") == 0).map(v => ByteBufferUtil.string(v._2.value())) + ByteBufferUtil.string(value.get(ByteBufferUtil.bytes("para")).value()) } } -- cgit v1.2.3 From b104c7f5c7e2b173fe1b10035efbc00e43df13ec Mon Sep 17 00:00:00 2001 From: Rohit Rai Date: Mon, 3 Jun 2013 15:15:52 +0530 Subject: Example to write the output to cassandra --- .../main/scala/spark/examples/CassandraTest.scala | 48 +++++++++++++++++++--- 1 file changed, 43 insertions(+), 5 deletions(-) diff --git a/examples/src/main/scala/spark/examples/CassandraTest.scala b/examples/src/main/scala/spark/examples/CassandraTest.scala index 6b9fd502e2..2cc62b9fe9 100644 --- a/examples/src/main/scala/spark/examples/CassandraTest.scala +++ b/examples/src/main/scala/spark/examples/CassandraTest.scala @@ -1,17 +1,16 @@ package spark.examples import org.apache.hadoop.mapreduce.Job -import org.apache.cassandra.hadoop.{ConfigHelper, ColumnFamilyInputFormat} -import org.apache.cassandra.thrift.{IndexExpression, SliceRange, SlicePredicate} +import org.apache.cassandra.hadoop.{ColumnFamilyOutputFormat, ConfigHelper, ColumnFamilyInputFormat} +import org.apache.cassandra.thrift._ import spark.{RDD, SparkContext} -import SparkContext._ +import spark.SparkContext._ import java.nio.ByteBuffer import java.util.SortedMap import org.apache.cassandra.db.IColumn import org.apache.cassandra.utils.ByteBufferUtil import scala.collection.JavaConversions._ -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + /* @@ -44,8 +43,15 @@ object CassandraTest { ConfigHelper.setInputRpcPort(job.getConfiguration(), args(2)) + ConfigHelper.setOutputInitialAddress(job.getConfiguration(), args(1)) + + ConfigHelper.setOutputRpcPort(job.getConfiguration(), args(2)) + ConfigHelper.setInputColumnFamily(job.getConfiguration(), "casDemo", "Words") + ConfigHelper.setOutputColumnFamily(job.getConfiguration(), "casDemo", "WordCount") + + val predicate = new SlicePredicate() val sliceRange = new SliceRange() sliceRange.setStart(Array.empty[Byte]) @@ -55,6 +61,8 @@ object CassandraTest { ConfigHelper.setInputPartitioner(job.getConfiguration(), "Murmur3Partitioner") + ConfigHelper.setOutputPartitioner(job.getConfiguration(), "Murmur3Partitioner") + //Make a new Hadoop RDD val casRdd = sc.newAPIHadoopRDD(job.getConfiguration(), classOf[ColumnFamilyInputFormat], @@ -74,6 +82,33 @@ object CassandraTest { counts.collect().foreach { case (word, count) => println(word + ":" + count) } + + counts.map { + case (word, count) => { + val colWord = new org.apache.cassandra.thrift.Column() + colWord.setName(ByteBufferUtil.bytes("word")) + colWord.setValue(ByteBufferUtil.bytes(word)) + colWord.setTimestamp(System.currentTimeMillis) + + val colCount = new org.apache.cassandra.thrift.Column() + colCount.setName(ByteBufferUtil.bytes("wcount")) + colCount.setValue(ByteBufferUtil.bytes(count.toLong)) + colCount.setTimestamp(System.currentTimeMillis) + + + val outputkey = ByteBufferUtil.bytes(word + "-COUNT-" + System.currentTimeMillis) + + val mutations: java.util.List[Mutation] = new Mutation() :: new Mutation() :: Nil + mutations.get(0).setColumn_or_supercolumn(new ColumnOrSuperColumn()) + mutations.get(0).column_or_supercolumn.setColumn(colWord) + + mutations.get(1).setColumn_or_supercolumn(new ColumnOrSuperColumn()) + mutations.get(1).column_or_supercolumn.setColumn(colCount) + (outputkey, mutations) + } + }.saveAsNewAPIHadoopFile("casDemo", classOf[ByteBuffer], classOf[List[Mutation]], + classOf[ColumnFamilyOutputFormat], job.getConfiguration) + } } @@ -81,6 +116,9 @@ object CassandraTest { create keyspace casDemo; use casDemo; +create column family WordCount with comparator = UTF8Type; +update column family WordCount with column_metadata = [{column_name: word, validation_class: UTF8Type}, {column_name: wcount, validation_class: LongType}]; + create column family Words with comparator = UTF8Type; update column family Words with column_metadata = [{column_name: book, validation_class: UTF8Type}, {column_name: para, validation_class: UTF8Type}]; -- cgit v1.2.3 From a058b0acf3e5ae41e64640feeace3d4e32f47401 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 3 Jun 2013 12:10:00 -0700 Subject: Delete a file for a block if it already exists. --- core/src/main/scala/spark/storage/DiskStore.scala | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index c7281200e7..2be5d01e31 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -195,9 +195,15 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) } private def createFile(blockId: String, allowAppendExisting: Boolean = false): File = { - val file = getFile(blockId) + var file = getFile(blockId) if (!allowAppendExisting && file.exists()) { - throw new Exception("File for block " + blockId + " already exists on disk: " + file) + // NOTE(shivaram): Delete the file if it exists. This might happen if a ShuffleMap task + // was rescheduled on the same machine as the old task ? + logWarning("File for block " + blockId + " already exists on disk: " + file + ". Deleting") + file.delete() + // Reopen the file + file = getFile(blockId) + // throw new Exception("File for block " + blockId + " already exists on disk: " + file) } file } -- cgit v1.2.3 From cd347f547a9a9b7bdd0d3f4734ae5c13be54f75d Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 3 Jun 2013 12:27:51 -0700 Subject: Reuse the file object as it is valid after delete --- core/src/main/scala/spark/storage/DiskStore.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 2be5d01e31..e51d258a21 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -201,8 +201,6 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) // was rescheduled on the same machine as the old task ? logWarning("File for block " + blockId + " already exists on disk: " + file + ". Deleting") file.delete() - // Reopen the file - file = getFile(blockId) // throw new Exception("File for block " + blockId + " already exists on disk: " + file) } file -- cgit v1.2.3 From 96943a1cc054d7cf80eb8d3dfc7fb19ce48d3c0a Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 3 Jun 2013 12:29:38 -0700 Subject: var to val --- core/src/main/scala/spark/storage/DiskStore.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index e51d258a21..cd85fa1e9d 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -195,7 +195,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) } private def createFile(blockId: String, allowAppendExisting: Boolean = false): File = { - var file = getFile(blockId) + val file = getFile(blockId) if (!allowAppendExisting && file.exists()) { // NOTE(shivaram): Delete the file if it exists. This might happen if a ShuffleMap task // was rescheduled on the same machine as the old task ? -- cgit v1.2.3 From d1286231e0db15e480bd7d6a600b419db3391b27 Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Wed, 29 May 2013 20:14:59 -0700 Subject: Sometime Maven build runs out of PermGen space. --- pom.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/pom.xml b/pom.xml index 6ee64d07c2..ce77ba37c6 100644 --- a/pom.xml +++ b/pom.xml @@ -59,6 +59,9 @@ 1.6.1 4.1.2 1.2.17 + + 0m + 512m @@ -392,6 +395,10 @@ -Xms64m -Xmx1024m + -XX:PermSize + ${PermGen} + -XX:MaxPermSize + ${MaxPermGen} -source -- cgit v1.2.3 From 8bd4e1210422d9985e6105fd9529e813fe45c14e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 4 Jun 2013 18:14:24 -0400 Subject: Bump akka and blockmanager timeouts to 60 seconds --- core/src/main/scala/spark/storage/BlockManager.scala | 2 +- core/src/main/scala/spark/util/AkkaUtils.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 09572b19db..150c98f57c 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -950,7 +950,7 @@ object BlockManager extends Logging { } def getHeartBeatFrequencyFromSystemProperties: Long = - System.getProperty("spark.storage.blockManagerHeartBeatMs", "5000").toLong + System.getProperty("spark.storage.blockManagerHeartBeatMs", "60000").toLong def getDisableHeartBeatsForTesting: Boolean = System.getProperty("spark.test.disableBlockManagerHeartBeat", "false").toBoolean diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index 9fb7e001ba..def993236b 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -29,7 +29,7 @@ private[spark] object AkkaUtils { def createActorSystem(name: String, host: String, port: Int): (ActorSystem, Int) = { val akkaThreads = System.getProperty("spark.akka.threads", "4").toInt val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt - val akkaTimeout = System.getProperty("spark.akka.timeout", "20").toInt + val akkaTimeout = System.getProperty("spark.akka.timeout", "60").toInt val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt val lifecycleEvents = if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" // 10 seconds is the default akka timeout, but in a cluster, we need higher by default. -- cgit v1.2.3 From 061fd3ae369e744f076e21044de26a00982a408f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 4 Jun 2013 18:50:15 -0400 Subject: Fixing bug in BlockManager timeout --- core/src/main/scala/spark/storage/BlockManager.scala | 2 +- core/src/main/scala/spark/storage/BlockManagerMasterActor.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 150c98f57c..65c789ea8f 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -950,7 +950,7 @@ object BlockManager extends Logging { } def getHeartBeatFrequencyFromSystemProperties: Long = - System.getProperty("spark.storage.blockManagerHeartBeatMs", "60000").toLong + System.getProperty("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4 def getDisableHeartBeatsForTesting: Boolean = System.getProperty("spark.test.disableBlockManagerHeartBeat", "false").toBoolean diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index 9b64f95df8..0dcb9fb2ac 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -35,7 +35,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { "" + (BlockManager.getHeartBeatFrequencyFromSystemProperties * 3)).toLong val checkTimeoutInterval = System.getProperty("spark.storage.blockManagerTimeoutIntervalMs", - "5000").toLong + "60000").toLong var timeoutCheckingTask: Cancellable = null -- cgit v1.2.3 From 9d359043574f6801ba15ec9d016eba0f00ac2349 Mon Sep 17 00:00:00 2001 From: Christopher Nguyen Date: Tue, 4 Jun 2013 22:12:47 -0700 Subject: In the current code, when both partitions happen to have zero-length, the return mean will be NaN. Consequently, the result of mean after reducing over all partitions will also be NaN, which is not correct if there are partitions with non-zero length. This patch fixes this issue. --- core/src/main/scala/spark/util/StatCounter.scala | 26 +++++++++++++++--------- 1 file changed, 16 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/util/StatCounter.scala b/core/src/main/scala/spark/util/StatCounter.scala index 5f80180339..2b980340b7 100644 --- a/core/src/main/scala/spark/util/StatCounter.scala +++ b/core/src/main/scala/spark/util/StatCounter.scala @@ -37,17 +37,23 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { if (other == this) { merge(other.copy()) // Avoid overwriting fields in a weird order } else { - val delta = other.mu - mu - if (other.n * 10 < n) { - mu = mu + (delta * other.n) / (n + other.n) - } else if (n * 10 < other.n) { - mu = other.mu - (delta * n) / (n + other.n) - } else { - mu = (mu * n + other.mu * other.n) / (n + other.n) + if (n == 0) { + mu = other.mu + m2 = other.m2 + n = other.n + } else if (other.n != 0) { + val delta = other.mu - mu + if (other.n * 10 < n) { + mu = mu + (delta * other.n) / (n + other.n) + } else if (n * 10 < other.n) { + mu = other.mu - (delta * n) / (n + other.n) + } else { + mu = (mu * n + other.mu * other.n) / (n + other.n) + } + m2 += other.m2 + (delta * delta * n * other.n) / (n + other.n) + n += other.n } - m2 += other.m2 + (delta * delta * n * other.n) / (n + other.n) - n += other.n - this + this } } -- cgit v1.2.3 From c851957fe4798d5dfb8deba7bf79a035a0543c74 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 5 Jun 2013 14:28:38 -0700 Subject: Don't write zero block files with java serializer --- .../scala/spark/storage/BlockFetcherIterator.scala | 5 ++- core/src/main/scala/spark/storage/DiskStore.scala | 46 ++++++++++++++-------- .../scala/spark/storage/ShuffleBlockManager.scala | 2 +- core/src/test/scala/spark/ShuffleSuite.scala | 26 ++++++++++++ 4 files changed, 61 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index fac416a5b3..843069239c 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -71,6 +71,7 @@ object BlockFetcherIterator { logDebug("Getting " + _totalBlocks + " blocks") protected var startTime = System.currentTimeMillis protected val localBlockIds = new ArrayBuffer[String]() + protected val localNonZeroBlocks = new ArrayBuffer[String]() protected val remoteBlockIds = new HashSet[String]() // A queue to hold our results. @@ -129,6 +130,8 @@ object BlockFetcherIterator { for ((address, blockInfos) <- blocksByAddress) { if (address == blockManagerId) { localBlockIds ++= blockInfos.map(_._1) + localNonZeroBlocks ++= blockInfos.filter(_._2 != 0).map(_._1) + _totalBlocks -= (localBlockIds.size - localNonZeroBlocks.size) } else { remoteBlockIds ++= blockInfos.map(_._1) // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them @@ -172,7 +175,7 @@ object BlockFetcherIterator { // Get the local blocks while remote blocks are being fetched. Note that it's okay to do // these all at once because they will just memory-map some files, so they won't consume // any memory that might exceed our maxBytesInFlight - for (id <- localBlockIds) { + for (id <- localNonZeroBlocks) { getLocalFromDisk(id, serializer) match { case Some(iter) => { // Pass 0 as size since it's not in flight diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index cd85fa1e9d..c1cff25552 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -35,21 +35,25 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) private var bs: OutputStream = null private var objOut: SerializationStream = null private var lastValidPosition = 0L + private var initialized = false override def open(): DiskBlockObjectWriter = { val fos = new FileOutputStream(f, true) channel = fos.getChannel() bs = blockManager.wrapForCompression(blockId, new FastBufferedOutputStream(fos)) objOut = serializer.newInstance().serializeStream(bs) + initialized = true this } override def close() { - objOut.close() - bs.close() - channel = null - bs = null - objOut = null + if (initialized) { + objOut.close() + bs.close() + channel = null + bs = null + objOut = null + } // Invoke the close callback handler. super.close() } @@ -59,23 +63,33 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) // Flush the partial writes, and set valid length to be the length of the entire file. // Return the number of bytes written for this commit. override def commit(): Long = { - // NOTE: Flush the serializer first and then the compressed/buffered output stream - objOut.flush() - bs.flush() - val prevPos = lastValidPosition - lastValidPosition = channel.position() - lastValidPosition - prevPos + if (initialized) { + // NOTE: Flush the serializer first and then the compressed/buffered output stream + objOut.flush() + bs.flush() + val prevPos = lastValidPosition + lastValidPosition = channel.position() + lastValidPosition - prevPos + } else { + // lastValidPosition is zero if stream is uninitialized + lastValidPosition + } } override def revertPartialWrites() { - // Discard current writes. We do this by flushing the outstanding writes and - // truncate the file to the last valid position. - objOut.flush() - bs.flush() - channel.truncate(lastValidPosition) + if (initialized) { + // Discard current writes. We do this by flushing the outstanding writes and + // truncate the file to the last valid position. + objOut.flush() + bs.flush() + channel.truncate(lastValidPosition) + } } override def write(value: Any) { + if (!initialized) { + open() + } objOut.writeObject(value) } diff --git a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala index 49eabfb0d2..44638e0c2d 100644 --- a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala @@ -24,7 +24,7 @@ class ShuffleBlockManager(blockManager: BlockManager) { val bufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024 val writers = Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => val blockId = ShuffleBlockManager.blockId(shuffleId, bucketId, mapId) - blockManager.getDiskBlockWriter(blockId, serializer, bufferSize).open() + blockManager.getDiskBlockWriter(blockId, serializer, bufferSize) } new ShuffleWriterGroup(mapId, writers) } diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index b967016cf7..33b02fff80 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -367,6 +367,32 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { assert(nonEmptyBlocks.size <= 4) } + test("zero sized blocks without kryo") { + // Use a local cluster with 2 processes to make sure there are both local and remote blocks + sc = new SparkContext("local-cluster[2,1,512]", "test") + + // 10 partitions from 4 keys + val NUM_BLOCKS = 10 + val a = sc.parallelize(1 to 4, NUM_BLOCKS) + val b = a.map(x => (x, x*2)) + + // NOTE: The default Java serializer doesn't create zero-sized blocks. + // So, use Kryo + val c = new ShuffledRDD(b, new HashPartitioner(10)) + + val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId + assert(c.count === 4) + + val blockSizes = (0 until NUM_BLOCKS).flatMap { id => + val statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, id) + statuses.map(x => x._2) + } + val nonEmptyBlocks = blockSizes.filter(x => x > 0) + + // We should have at most 4 non-zero sized partitions + assert(nonEmptyBlocks.size <= 4) + } + } object ShuffleSuite { -- cgit v1.2.3 From cb2f5046ee99582a5038a78478c23468b14c134e Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 5 Jun 2013 15:09:02 -0700 Subject: Pass in bufferSize to BufferedOutputStream --- core/src/main/scala/spark/storage/DiskStore.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index c1cff25552..0af6e4a359 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -40,7 +40,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) override def open(): DiskBlockObjectWriter = { val fos = new FileOutputStream(f, true) channel = fos.getChannel() - bs = blockManager.wrapForCompression(blockId, new FastBufferedOutputStream(fos)) + bs = blockManager.wrapForCompression(blockId, new FastBufferedOutputStream(fos, bufferSize)) objOut = serializer.newInstance().serializeStream(bs) initialized = true this -- cgit v1.2.3 From e179ff8a32fc08cc308dc99bac2527d350d0d970 Mon Sep 17 00:00:00 2001 From: Gavin Li Date: Wed, 5 Jun 2013 22:41:05 +0000 Subject: update according to comments --- core/src/main/scala/spark/RDD.scala | 89 +++++++++++++++++++++++---- core/src/main/scala/spark/rdd/PipedRDD.scala | 33 +++++----- core/src/test/scala/spark/PipedRDDSuite.scala | 7 ++- 3 files changed, 99 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 5a41db23c2..a1c9604324 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -16,6 +16,7 @@ import org.apache.hadoop.mapred.TextOutputFormat import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} +import spark.broadcast.Broadcast import spark.Partitioner._ import spark.partial.BoundedDouble import spark.partial.CountEvaluator @@ -351,31 +352,93 @@ abstract class RDD[T: ClassManifest]( /** * Return an RDD created by piping elements to a forked external process. */ - def pipe(command: String, transform: (T,String => Unit) => Any, arguments: Seq[String]): RDD[String] = - new PipedRDD(this, command, transform, arguments) + def pipe(command: String, env: Map[String, String]): RDD[String] = + new PipedRDD(this, command, env) /** * Return an RDD created by piping elements to a forked external process. - */ - def pipe(command: Seq[String]): RDD[String] = new PipedRDD(this, command) + * How each record in RDD is outputed to the process can be controled by providing a + * function trasnform(T, outputFunction: String => Unit). transform() will be called with + * the currnet record in RDD as the 1st parameter, and the function to output the record to + * the external process (like out.println()) as the 2nd parameter. + * Here's an example on how to pipe the RDD data of groupBy() in a streaming way, + * instead of constructing a huge String to concat all the records: + * def tranform(record:(String, Seq[String]), f:String=>Unit) = for (e <- record._2){f(e)} + * pipeContext can be used to transfer additional context data to the external process + * besides the RDD. pipeContext is a broadcast Seq[String], each line would be piped to + * external process with "^A" as the delimiter in the end of context data. Delimiter can also + * be customized by the last parameter delimiter. + */ + def pipe[U<: Seq[String]]( + command: String, + env: Map[String, String], + transform: (T,String => Unit) => Any, + pipeContext: Broadcast[U], + delimiter: String): RDD[String] = + new PipedRDD(this, command, env, transform, pipeContext, delimiter) /** * Return an RDD created by piping elements to a forked external process. - */ - def pipe(command: Seq[String], transform: (T,String => Unit) => Any, arguments: Seq[String]): RDD[String] = - new PipedRDD(this, command, transform, arguments) + * How each record in RDD is outputed to the process can be controled by providing a + * function trasnform(T, outputFunction: String => Unit). transform() will be called with + * the currnet record in RDD as the 1st parameter, and the function to output the record to + * the external process (like out.println()) as the 2nd parameter. + * Here's an example on how to pipe the RDD data of groupBy() in a streaming way, + * instead of constructing a huge String to concat all the records: + * def tranform(record:(String, Seq[String]), f:String=>Unit) = for (e <- record._2){f(e)} + * pipeContext can be used to transfer additional context data to the external process + * besides the RDD. pipeContext is a broadcast Seq[String], each line would be piped to + * external process with "^A" as the delimiter in the end of context data. Delimiter can also + * be customized by the last parameter delimiter. + */ + def pipe[U<: Seq[String]]( + command: String, + transform: (T,String => Unit) => Any, + pipeContext: Broadcast[U]): RDD[String] = + new PipedRDD(this, command, Map[String, String](), transform, pipeContext, "\u0001") /** * Return an RDD created by piping elements to a forked external process. - */ - def pipe(command: Seq[String], env: Map[String, String]): RDD[String] = - new PipedRDD(this, command, env) + * How each record in RDD is outputed to the process can be controled by providing a + * function trasnform(T, outputFunction: String => Unit). transform() will be called with + * the currnet record in RDD as the 1st parameter, and the function to output the record to + * the external process (like out.println()) as the 2nd parameter. + * Here's an example on how to pipe the RDD data of groupBy() in a streaming way, + * instead of constructing a huge String to concat all the records: + * def tranform(record:(String, Seq[String]), f:String=>Unit) = for (e <- record._2){f(e)} + * pipeContext can be used to transfer additional context data to the external process + * besides the RDD. pipeContext is a broadcast Seq[String], each line would be piped to + * external process with "^A" as the delimiter in the end of context data. Delimiter can also + * be customized by the last parameter delimiter. + */ + def pipe[U<: Seq[String]]( + command: String, + env: Map[String, String], + transform: (T,String => Unit) => Any, + pipeContext: Broadcast[U]): RDD[String] = + new PipedRDD(this, command, env, transform, pipeContext, "\u0001") /** * Return an RDD created by piping elements to a forked external process. - */ - def pipe(command: Seq[String], env: Map[String, String], transform: (T,String => Unit) => Any, arguments: Seq[String]): RDD[String] = - new PipedRDD(this, command, env, transform, arguments) + * How each record in RDD is outputed to the process can be controled by providing a + * function trasnform(T, outputFunction: String => Unit). transform() will be called with + * the currnet record in RDD as the 1st parameter, and the function to output the record to + * the external process (like out.println()) as the 2nd parameter. + * Here's an example on how to pipe the RDD data of groupBy() in a streaming way, + * instead of constructing a huge String to concat all the records: + * def tranform(record:(String, Seq[String]), f:String=>Unit) = for (e <- record._2){f(e)} + * pipeContext can be used to transfer additional context data to the external process + * besides the RDD. pipeContext is a broadcast Seq[String], each line would be piped to + * external process with "^A" as the delimiter in the end of context data. Delimiter can also + * be customized by the last parameter delimiter. + */ + def pipe[U<: Seq[String]]( + command: Seq[String], + env: Map[String, String] = Map(), + transform: (T,String => Unit) => Any = null, + pipeContext: Broadcast[U] = null, + delimiter: String = "\u0001"): RDD[String] = + new PipedRDD(this, command, env, transform, pipeContext, delimiter) /** * Return a new RDD by applying a function to each partition of this RDD. diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala index 969404c95f..d58aaae709 100644 --- a/core/src/main/scala/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/spark/rdd/PipedRDD.scala @@ -9,29 +9,33 @@ import scala.collection.mutable.ArrayBuffer import scala.io.Source import spark.{RDD, SparkEnv, Partition, TaskContext} +import spark.broadcast.Broadcast /** * An RDD that pipes the contents of each parent partition through an external command * (printing them one per line) and returns the output as a collection of strings. */ -class PipedRDD[T: ClassManifest]( +class PipedRDD[T: ClassManifest, U <: Seq[String]]( prev: RDD[T], command: Seq[String], envVars: Map[String, String], transform: (T, String => Unit) => Any, - arguments: Seq[String] + pipeContext: Broadcast[U], + delimiter: String ) extends RDD[String](prev) { - def this(prev: RDD[T], command: Seq[String], envVars : Map[String, String]) = this(prev, command, envVars, null, null) - def this(prev: RDD[T], command: Seq[String]) = this(prev, command, Map()) - def this(prev: RDD[T], command: Seq[String], transform: (T,String => Unit) => Any, arguments: Seq[String]) = this(prev, command, Map(), transform, arguments) - // Similar to Runtime.exec(), if we are given a single string, split it into words // using a standard StringTokenizer (i.e. by spaces) - def this(prev: RDD[T], command: String) = this(prev, PipedRDD.tokenize(command)) - def this(prev: RDD[T], command: String, transform: (T,String => Unit) => Any, arguments: Seq[String]) = this(prev, PipedRDD.tokenize(command), Map(), transform, arguments) + def this( + prev: RDD[T], + command: String, + envVars: Map[String, String] = Map(), + transform: (T, String => Unit) => Any = null, + pipeContext: Broadcast[U] = null, + delimiter: String = "\u0001") = + this(prev, PipedRDD.tokenize(command), envVars, transform, pipeContext, delimiter) override def getPartitions: Array[Partition] = firstParent[T].partitions @@ -60,19 +64,18 @@ class PipedRDD[T: ClassManifest]( SparkEnv.set(env) val out = new PrintWriter(proc.getOutputStream) - // input the arguments firstly - if ( arguments != null) { - for (elem <- arguments) { + // input the pipeContext firstly + if ( pipeContext != null) { + for (elem <- pipeContext.value) { out.println(elem) } - // ^A \n as the marker of the end of the arguments - out.println("\u0001") + // delimiter\n as the marker of the end of the pipeContext + out.println(delimiter) } for (elem <- firstParent[T].iterator(split, context)) { if (transform != null) { transform(elem, out.println(_)) - } - else { + } else { out.println(elem) } } diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/spark/PipedRDDSuite.scala index ee55952a94..d2852867de 100644 --- a/core/src/test/scala/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/spark/PipedRDDSuite.scala @@ -23,7 +23,8 @@ class PipedRDDSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) - val piped = nums.pipe(Seq("cat"), (i:Int, f: String=> Unit) => f(i + "_"), Array("0")) + val piped = nums.pipe(Seq("cat"), Map[String, String](), + (i:Int, f: String=> Unit) => f(i + "_"), sc.broadcast(List("0"))) val c = piped.collect() @@ -38,7 +39,9 @@ class PipedRDDSuite extends FunSuite with LocalSparkContext { assert(c(7) === "4_") val nums1 = sc.makeRDD(Array("a\t1", "b\t2", "a\t3", "b\t4"), 2) - val d = nums1.groupBy(str=>str.split("\t")(0)).pipe(Seq("cat"), (i:Tuple2[String, Seq[String]], f: String=> Unit) => {for (e <- i._2){ f(e + "_")}}, Array("0")).collect() + val d = nums1.groupBy(str=>str.split("\t")(0)). + pipe(Seq("cat"), Map[String, String](), (i:Tuple2[String, Seq[String]], f: String=> Unit) => + {for (e <- i._2){ f(e + "_")}}, sc.broadcast(List("0"))).collect() assert(d.size === 8) assert(d(0) === "0") assert(d(1) === "\u0001") -- cgit v1.2.3 From ac480fd977e0de97bcfe646e39feadbd239c1c29 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 6 Jun 2013 16:34:27 -0700 Subject: Clean up variables and counters in BlockFetcherIterator --- .../scala/spark/storage/BlockFetcherIterator.scala | 54 +++++++++++++--------- 1 file changed, 31 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index 843069239c..bb78207c9f 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -67,12 +67,20 @@ object BlockFetcherIterator { throw new IllegalArgumentException("BlocksByAddress is null") } - protected var _totalBlocks = blocksByAddress.map(_._2.size).sum - logDebug("Getting " + _totalBlocks + " blocks") + // Total number blocks fetched (local + remote). Also number of FetchResults expected + protected var _numBlocksToFetch = 0 + protected var startTime = System.currentTimeMillis - protected val localBlockIds = new ArrayBuffer[String]() - protected val localNonZeroBlocks = new ArrayBuffer[String]() - protected val remoteBlockIds = new HashSet[String]() + + // This represents the number of local blocks, also counting zero-sized blocks + private var numLocal = 0 + // BlockIds for local blocks that need to be fetched. Excludes zero-sized blocks + protected val localBlocksToFetch = new ArrayBuffer[String]() + + // This represents the number of remote blocks, also counting zero-sized blocks + private var numRemote = 0 + // BlockIds for remote blocks that need to be fetched. Excludes zero-sized blocks + protected val remoteBlocksToFetch = new HashSet[String]() // A queue to hold our results. protected val results = new LinkedBlockingQueue[FetchResult] @@ -125,15 +133,15 @@ object BlockFetcherIterator { protected def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { // Split local and remote blocks. Remote blocks are further split into FetchRequests of size // at most maxBytesInFlight in order to limit the amount of data in flight. - val originalTotalBlocks = _totalBlocks val remoteRequests = new ArrayBuffer[FetchRequest] for ((address, blockInfos) <- blocksByAddress) { if (address == blockManagerId) { - localBlockIds ++= blockInfos.map(_._1) - localNonZeroBlocks ++= blockInfos.filter(_._2 != 0).map(_._1) - _totalBlocks -= (localBlockIds.size - localNonZeroBlocks.size) + numLocal = blockInfos.size + // Filter out zero-sized blocks + localBlocksToFetch ++= blockInfos.filter(_._2 != 0).map(_._1) + _numBlocksToFetch += localBlocksToFetch.size } else { - remoteBlockIds ++= blockInfos.map(_._1) + numRemote += blockInfos.size // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 // nodes, rather than blocking on reading output from one node. @@ -147,10 +155,10 @@ object BlockFetcherIterator { // Skip empty blocks if (size > 0) { curBlocks += ((blockId, size)) + remoteBlocksToFetch += blockId + _numBlocksToFetch += 1 curRequestSize += size - } else if (size == 0) { - _totalBlocks -= 1 - } else { + } else if (size < 0) { throw new BlockException(blockId, "Negative block size " + size) } if (curRequestSize >= minRequestSize) { @@ -166,8 +174,8 @@ object BlockFetcherIterator { } } } - logInfo("Getting " + _totalBlocks + " non-zero-bytes blocks out of " + - originalTotalBlocks + " blocks") + logInfo("Getting " + _numBlocksToFetch + " non-zero-bytes blocks out of " + + totalBlocks + " blocks") remoteRequests } @@ -175,7 +183,7 @@ object BlockFetcherIterator { // Get the local blocks while remote blocks are being fetched. Note that it's okay to do // these all at once because they will just memory-map some files, so they won't consume // any memory that might exceed our maxBytesInFlight - for (id <- localNonZeroBlocks) { + for (id <- localBlocksToFetch) { getLocalFromDisk(id, serializer) match { case Some(iter) => { // Pass 0 as size since it's not in flight @@ -201,7 +209,7 @@ object BlockFetcherIterator { sendRequest(fetchRequests.dequeue()) } - val numGets = remoteBlockIds.size - fetchRequests.size + val numGets = remoteRequests.size - fetchRequests.size logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime)) // Get Local Blocks @@ -213,7 +221,7 @@ object BlockFetcherIterator { //an iterator that will read fetched blocks off the queue as they arrive. @volatile protected var resultsGotten = 0 - override def hasNext: Boolean = resultsGotten < _totalBlocks + override def hasNext: Boolean = resultsGotten < _numBlocksToFetch override def next(): (String, Option[Iterator[Any]]) = { resultsGotten += 1 @@ -230,9 +238,9 @@ object BlockFetcherIterator { } // Implementing BlockFetchTracker trait. - override def totalBlocks: Int = _totalBlocks - override def numLocalBlocks: Int = localBlockIds.size - override def numRemoteBlocks: Int = remoteBlockIds.size + override def totalBlocks: Int = numLocal + numRemote + override def numLocalBlocks: Int = numLocal + override def numRemoteBlocks: Int = numRemote override def remoteFetchTime: Long = _remoteFetchTime override def fetchWaitTime: Long = _fetchWaitTime override def remoteBytesRead: Long = _remoteBytesRead @@ -294,7 +302,7 @@ object BlockFetcherIterator { private var copiers: List[_ <: Thread] = null override def initialize() { - // Split Local Remote Blocks and adjust totalBlocks to include only the non 0-byte blocks + // Split Local Remote Blocks and set numBlocksToFetch val remoteRequests = splitLocalRemoteBlocks() // Add the remote requests into our queue in a random order for (request <- Utils.randomize(remoteRequests)) { @@ -316,7 +324,7 @@ object BlockFetcherIterator { val result = results.take() // if all the results has been retrieved, shutdown the copiers // NO need to stop the copiers if we got all the blocks ? - // if (resultsGotten == _totalBlocks && copiers != null) { + // if (resultsGotten == _numBlocksToFetch && copiers != null) { // stopCopiers() // } (result.blockId, if (result.failed) None else Some(result.deserialize())) -- cgit v1.2.3 From c9ca0a4a588b4c7dc553b155336ae5b95aa9ddd4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 7 Jun 2013 22:40:44 -0700 Subject: Small code style fix to SchedulingAlgorithm.scala --- .../src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala index 13120edf63..e071917c00 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala @@ -53,13 +53,12 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { } if (compare < 0) { - res = true + return true } else if (compare > 0) { - res = false + return false } else { return s1.name < s2.name } - return res } } -- cgit v1.2.3 From b58a29295b2e610cadf1cac44438337ce9b51537 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 7 Jun 2013 22:51:28 -0700 Subject: Small formatting and style fixes --- .../spark/scheduler/cluster/SchedulingAlgorithm.scala | 8 ++++---- core/src/main/scala/spark/storage/StorageUtils.scala | 14 +++++++------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala index e071917c00..f33310a34a 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala @@ -13,11 +13,11 @@ private[spark] class FIFOSchedulingAlgorithm extends SchedulingAlgorithm { override def comparator(s1: Schedulable, s2: Schedulable): Boolean = { val priority1 = s1.priority val priority2 = s2.priority - var res = Math.signum(priority1 - priority2) + var res = math.signum(priority1 - priority2) if (res == 0) { val stageId1 = s1.stageId val stageId2 = s2.stageId - res = Math.signum(stageId1 - stageId2) + res = math.signum(stageId1 - stageId2) } if (res < 0) { return true @@ -35,8 +35,8 @@ private[spark] class FairSchedulingAlgorithm extends SchedulingAlgorithm { val runningTasks2 = s2.runningTasks val s1Needy = runningTasks1 < minShare1 val s2Needy = runningTasks2 < minShare2 - val minShareRatio1 = runningTasks1.toDouble / Math.max(minShare1, 1.0).toDouble - val minShareRatio2 = runningTasks2.toDouble / Math.max(minShare2, 1.0).toDouble + val minShareRatio1 = runningTasks1.toDouble / math.max(minShare1, 1.0).toDouble + val minShareRatio2 = runningTasks2.toDouble / math.max(minShare2, 1.0).toDouble val taskToWeightRatio1 = runningTasks1.toDouble / s1.weight.toDouble val taskToWeightRatio2 = runningTasks2.toDouble / s2.weight.toDouble var res:Boolean = true diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index 81e607868d..950c0cdf35 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -55,21 +55,21 @@ object StorageUtils { }.mapValues(_.values.toArray) // For each RDD, generate an RDDInfo object - val rddInfos = groupedRddBlocks.map { case(rddKey, rddBlocks) => - + val rddInfos = groupedRddBlocks.map { case (rddKey, rddBlocks) => // Add up memory and disk sizes val memSize = rddBlocks.map(_.memSize).reduce(_ + _) val diskSize = rddBlocks.map(_.diskSize).reduce(_ + _) // Find the id of the RDD, e.g. rdd_1 => 1 val rddId = rddKey.split("_").last.toInt - // Get the friendly name for the rdd, if available. + + // Get the friendly name and storage level for the RDD, if available sc.persistentRdds.get(rddId).map { r => - val rddName = Option(r.name).getOrElse(rddKey) - val rddStorageLevel = r.getStorageLevel - RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, r.partitions.size, memSize, diskSize) + val rddName = Option(r.name).getOrElse(rddKey) + val rddStorageLevel = r.getStorageLevel + RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, r.partitions.size, memSize, diskSize) } - }.flatMap(x => x).toArray + }.flatten.toArray scala.util.Sorting.quickSort(rddInfos) -- cgit v1.2.3 From 1a4d93c025e5d3679257a622f49dfaade4ac18c2 Mon Sep 17 00:00:00 2001 From: Mingfei Date: Sat, 8 Jun 2013 14:23:39 +0800 Subject: modify to pass job annotation by localProperties and use daeamon thread to do joblogger's work --- .../scala/spark/BlockStoreShuffleFetcher.scala | 1 + core/src/main/scala/spark/RDD.scala | 10 +- core/src/main/scala/spark/SparkContext.scala | 8 +- core/src/main/scala/spark/Utils.scala | 10 +- core/src/main/scala/spark/executor/Executor.scala | 1 + .../main/scala/spark/executor/TaskMetrics.scala | 12 +- .../main/scala/spark/scheduler/DAGScheduler.scala | 8 + .../src/main/scala/spark/scheduler/JobLogger.scala | 317 +++++++++++++++++++++ .../main/scala/spark/scheduler/SparkListener.scala | 33 ++- .../scala/spark/scheduler/JobLoggerSuite.scala | 105 +++++++ .../scala/spark/scheduler/SparkListenerSuite.scala | 2 +- 11 files changed, 495 insertions(+), 12 deletions(-) create mode 100644 core/src/main/scala/spark/scheduler/JobLogger.scala create mode 100644 core/src/test/scala/spark/scheduler/JobLoggerSuite.scala diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index e1fb02157a..3239f4c385 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -58,6 +58,7 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin CompletionIterator[(K,V), Iterator[(K,V)]](itr, { val shuffleMetrics = new ShuffleReadMetrics + shuffleMetrics.shuffleFinishTime = System.currentTimeMillis shuffleMetrics.remoteFetchTime = blockFetcherItr.remoteFetchTime shuffleMetrics.fetchWaitTime = blockFetcherItr.fetchWaitTime shuffleMetrics.remoteBytesRead = blockFetcherItr.remoteBytesRead diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index e6c0438d76..8c0b7ca417 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -114,6 +114,14 @@ abstract class RDD[T: ClassManifest]( this } + /**User-defined generator of this RDD*/ + var generator = Utils.getCallSiteInfo._4 + + /**reset generator*/ + def setGenerator(_generator: String) = { + generator = _generator + } + /** * Set this RDD's storage level to persist its values across operations after the first time * it is computed. This can only be used to assign a new storage level if the RDD does not @@ -788,7 +796,7 @@ abstract class RDD[T: ClassManifest]( private var storageLevel: StorageLevel = StorageLevel.NONE /** Record user function generating this RDD. */ - private[spark] val origin = Utils.getSparkCallSite + private[spark] val origin = Utils.formatSparkCallSite private[spark] def elementClassManifest: ClassManifest[T] = classManifest[T] diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index bc05d08fd6..b67a2066c8 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -48,7 +48,7 @@ import spark.scheduler.local.LocalScheduler import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import spark.storage.{BlockManagerUI, StorageStatus, StorageUtils, RDDInfo} import spark.util.{MetadataCleaner, TimeStampedHashMap} - +import spark.scheduler.JobLogger /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -510,7 +510,7 @@ class SparkContext( def addSparkListener(listener: SparkListener) { dagScheduler.sparkListeners += listener } - + addSparkListener(new JobLogger) /** * Return a map from the slave to the max memory available for caching and the remaining * memory available for caching. @@ -630,7 +630,7 @@ class SparkContext( partitions: Seq[Int], allowLocal: Boolean, resultHandler: (Int, U) => Unit) { - val callSite = Utils.getSparkCallSite + val callSite = Utils.formatSparkCallSite logInfo("Starting job: " + callSite) val start = System.nanoTime val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal, resultHandler, localProperties.value) @@ -713,7 +713,7 @@ class SparkContext( func: (TaskContext, Iterator[T]) => U, evaluator: ApproximateEvaluator[U, R], timeout: Long): PartialResult[R] = { - val callSite = Utils.getSparkCallSite + val callSite = Utils.formatSparkCallSite logInfo("Starting job: " + callSite) val start = System.nanoTime val result = dagScheduler.runApproximateJob(rdd, func, evaluator, callSite, timeout, localProperties.value) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index ec15326014..1630b2b4b0 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -528,7 +528,7 @@ private object Utils extends Logging { * (outside the spark package) that called into Spark, as well as which Spark method they called. * This is used, for example, to tell users where in their code each RDD got created. */ - def getSparkCallSite: String = { + def getCallSiteInfo = { val trace = Thread.currentThread.getStackTrace().filter( el => (!el.getMethodName.contains("getStackTrace"))) @@ -540,6 +540,7 @@ private object Utils extends Logging { var firstUserFile = "" var firstUserLine = 0 var finished = false + var firstUserClass = "" for (el <- trace) { if (!finished) { @@ -554,13 +555,18 @@ private object Utils extends Logging { else { firstUserLine = el.getLineNumber firstUserFile = el.getFileName + firstUserClass = el.getClassName finished = true } } } - "%s at %s:%s".format(lastSparkMethod, firstUserFile, firstUserLine) + (lastSparkMethod, firstUserFile, firstUserLine, firstUserClass) } + def formatSparkCallSite = { + val callSiteInfo = getCallSiteInfo + "%s at %s:%s".format(callSiteInfo._1, callSiteInfo._2, callSiteInfo._3) + } /** * Try to find a free port to bind to on the local host. This should ideally never be needed, * except that, unfortunately, some of the networking libraries we currently rely on (e.g. Spray) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 890938d48b..8bebfafce4 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -104,6 +104,7 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert val value = task.run(taskId.toInt) val taskFinish = System.currentTimeMillis() task.metrics.foreach{ m => + m.hostname = Utils.localHostName m.executorDeserializeTime = (taskStart - startTime).toInt m.executorRunTime = (taskFinish - taskStart).toInt } diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index a7c56c2371..26e8029365 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -1,6 +1,11 @@ package spark.executor class TaskMetrics extends Serializable { + /** + * host's name the task runs on + */ + var hostname: String = _ + /** * Time taken on the executor to deserialize this task */ @@ -33,10 +38,15 @@ object TaskMetrics { class ShuffleReadMetrics extends Serializable { + /** + * Time when shuffle finishs + */ + var shuffleFinishTime: Long = _ + /** * Total number of blocks fetched in a shuffle (remote or local) */ - var totalBlocksFetched : Int = _ + var totalBlocksFetched: Int = _ /** * Number of remote blocks fetched in a shuffle diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 7feeb97542..43dd7d6534 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -289,6 +289,7 @@ class DAGScheduler( val finalStage = newStage(finalRDD, None, runId) val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener, properties) clearCacheLocs() + sparkListeners.foreach(_.onJobStart(job, properties)) logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length + " output partitions (allowLocal=" + allowLocal + ")") logInfo("Final stage: " + finalStage + " (" + finalStage.origin + ")") @@ -311,6 +312,7 @@ class DAGScheduler( handleExecutorLost(execId) case completion: CompletionEvent => + sparkListeners.foreach(_.onTaskEnd(completion)) handleTaskCompletion(completion) case TaskSetFailed(taskSet, reason) => @@ -321,6 +323,8 @@ class DAGScheduler( for (job <- activeJobs) { val error = new SparkException("Job cancelled because SparkContext was shut down") job.listener.jobFailed(error) + val JobCancelEvent = new SparkListenerJobCancelled("SPARKCONTEXT_SHUTDOWN") + sparkListeners.foreach(_.onJobEnd(job, JobCancelEvent)) } return true } @@ -468,6 +472,7 @@ class DAGScheduler( } } if (tasks.size > 0) { + sparkListeners.foreach(_.onStageSubmitted(stage, "TASKS_SIZE=" + tasks.size)) logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks logDebug("New pending tasks: " + myPending) @@ -522,6 +527,7 @@ class DAGScheduler( activeJobs -= job resultStageToJob -= stage markStageAsFinished(stage) + sparkListeners.foreach(_.onJobEnd(job, SparkListenerJobSuccess)) } job.listener.taskSucceeded(rt.outputId, event.result) } @@ -665,6 +671,8 @@ class DAGScheduler( job.listener.jobFailed(new SparkException("Job failed: " + reason)) activeJobs -= job resultStageToJob -= resultStage + val jobFailedEvent = new SparkListenerJobFailed(failedStage) + sparkListeners.foreach(_.onJobEnd(job, jobFailedEvent)) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala new file mode 100644 index 0000000000..f87acfd0b6 --- /dev/null +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -0,0 +1,317 @@ +package spark.scheduler + +import java.io.PrintWriter +import java.io.File +import java.io.FileNotFoundException +import java.text.SimpleDateFormat +import java.util.{Date, Properties} +import java.util.concurrent.LinkedBlockingQueue +import scala.collection.mutable.{Map, HashMap, ListBuffer} +import scala.io.Source +import spark._ +import spark.executor.TaskMetrics +import spark.scheduler.cluster.TaskInfo + +// used to record runtime information for each job, including RDD graph +// tasks' start/stop shuffle information and information from outside + +sealed trait JobLoggerEvent +case class JobLoggerOnJobStart(job: ActiveJob, properties: Properties) extends JobLoggerEvent +case class JobLoggerOnStageSubmitted(stage: Stage, info: String) extends JobLoggerEvent +case class JobLoggerOnStageCompleted(stageCompleted: StageCompleted) extends JobLoggerEvent +case class JobLoggerOnJobEnd(job: ActiveJob, event: SparkListenerEvents) extends JobLoggerEvent +case class JobLoggerOnTaskEnd(event: CompletionEvent) extends JobLoggerEvent + +class JobLogger(val logDirName: String) extends SparkListener with Logging { + private val logDir = + if (System.getenv("SPARK_LOG_DIR") != null) + System.getenv("SPARK_LOG_DIR") + else + "/tmp/spark" + private val jobIDToPrintWriter = new HashMap[Int, PrintWriter] + private val stageIDToJobID = new HashMap[Int, Int] + private val jobIDToStages = new HashMap[Int, ListBuffer[Stage]] + private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + private val eventQueue = new LinkedBlockingQueue[JobLoggerEvent] + + createLogDir() + def this() = this(String.valueOf(System.currentTimeMillis())) + + def getLogDir = logDir + def getJobIDtoPrintWriter = jobIDToPrintWriter + def getStageIDToJobID = stageIDToJobID + def getJobIDToStages = jobIDToStages + def getEventQueue = eventQueue + + new Thread("JobLogger") { + setDaemon(true) + override def run() { + while (true) { + val event = eventQueue.take + if (event != null) { + logDebug("Got event of type " + event.getClass.getName) + event match { + case JobLoggerOnJobStart(job, info) => + processJobStartEvent(job, info) + case JobLoggerOnStageSubmitted(stage, info) => + processStageSubmittedEvent(stage, info) + case JobLoggerOnStageCompleted(stageCompleted) => + processStageCompletedEvent(stageCompleted) + case JobLoggerOnJobEnd(job, event) => + processJobEndEvent(job, event) + case JobLoggerOnTaskEnd(event) => + processTaskEndEvent(event) + case _ => + } + } + } + } + }.start() + + //create a folder for log files, the folder's name is the creation time of the jobLogger + protected def createLogDir() { + val dir = new File(logDir + "/" + logDirName + "/") + if (dir.exists()) { + return + } + if (dir.mkdirs() == false) { + logError("create log directory error:" + logDir + "/" + logDirName + "/") + } + } + + // create a log file for one job, the file name is the jobID + protected def createLogWriter(jobID: Int) { + try{ + val fileWriter = new PrintWriter(logDir + "/" + logDirName + "/" + jobID) + jobIDToPrintWriter += (jobID -> fileWriter) + } catch { + case e: FileNotFoundException => e.printStackTrace() + } + } + + // close log file for one job, and clean the stage relationship in stageIDToJobID + protected def closeLogWriter(jobID: Int) = + jobIDToPrintWriter.get(jobID).foreach { fileWriter => + fileWriter.close() + jobIDToStages.get(jobID).foreach(_.foreach{ stage => + stageIDToJobID -= stage.id + }) + jobIDToPrintWriter -= jobID + jobIDToStages -= jobID + } + + // write log information to log file, withTime parameter controls whether to recored + // time stamp for the information + protected def jobLogInfo(jobID: Int, info: String, withTime: Boolean = true) { + var writeInfo = info + if (withTime) { + val date = new Date(System.currentTimeMillis()) + writeInfo = DATE_FORMAT.format(date) + ": " +info + } + jobIDToPrintWriter.get(jobID).foreach(_.println(writeInfo)) + } + + protected def stageLogInfo(stageID: Int, info: String, withTime: Boolean = true) = + stageIDToJobID.get(stageID).foreach(jobID => jobLogInfo(jobID, info, withTime)) + + protected def buildJobDep(jobID: Int, stage: Stage) { + if (stage.priority == jobID) { + jobIDToStages.get(jobID) match { + case Some(stageList) => stageList += stage + case None => val stageList = new ListBuffer[Stage] + stageList += stage + jobIDToStages += (jobID -> stageList) + } + stageIDToJobID += (stage.id -> jobID) + stage.parents.foreach(buildJobDep(jobID, _)) + } + } + + protected def recordStageDep(jobID: Int) { + def getRddsInStage(rdd: RDD[_]): ListBuffer[RDD[_]] = { + var rddList = new ListBuffer[RDD[_]] + rddList += rdd + rdd.dependencies.foreach{ dep => dep match { + case shufDep: ShuffleDependency[_,_] => + case _ => rddList ++= getRddsInStage(dep.rdd) + } + } + rddList + } + jobIDToStages.get(jobID).foreach {_.foreach { stage => + var depRddDesc: String = "" + getRddsInStage(stage.rdd).foreach { rdd => + depRddDesc += rdd.id + "," + } + var depStageDesc: String = "" + stage.parents.foreach { stage => + depStageDesc += "(" + stage.id + "," + stage.shuffleDep.get.shuffleId + ")" + } + jobLogInfo(jobID, "STAGE_ID=" + stage.id + " RDD_DEP=(" + + depRddDesc.substring(0, depRddDesc.length - 1) + ")" + + " STAGE_DEP=" + depStageDesc, false) + } + } + } + + // generate indents and convert to String + protected def indentString(indent: Int) = { + val sb = new StringBuilder() + for (i <- 1 to indent) { + sb.append(" ") + } + sb.toString() + } + + protected def getRddName(rdd: RDD[_]) = { + var rddName = rdd.getClass.getName + if (rdd.name != null) { + rddName = rdd.name + } + rddName + } + + protected def recordRddInStageGraph(jobID: Int, rdd: RDD[_], indent: Int) { + val rddInfo = "RDD_ID=" + rdd.id + "(" + getRddName(rdd) + "," + rdd.generator + ")" + jobLogInfo(jobID, indentString(indent) + rddInfo, false) + rdd.dependencies.foreach{ dep => dep match { + case shufDep: ShuffleDependency[_,_] => + val depInfo = "SHUFFLE_ID=" + shufDep.shuffleId + jobLogInfo(jobID, indentString(indent + 1) + depInfo, false) + case _ => recordRddInStageGraph(jobID, dep.rdd, indent + 1) + } + } + } + + protected def recordStageDepGraph(jobID: Int, stage: Stage, indent: Int = 0) { + var stageInfo: String = "" + if (stage.isShuffleMap) { + stageInfo = "STAGE_ID=" + stage.id + " MAP_STAGE SHUFFLE_ID=" + + stage.shuffleDep.get.shuffleId + }else{ + stageInfo = "STAGE_ID=" + stage.id + " RESULT_STAGE" + } + if (stage.priority == jobID) { + jobLogInfo(jobID, indentString(indent) + stageInfo, false) + recordRddInStageGraph(jobID, stage.rdd, indent) + stage.parents.foreach(recordStageDepGraph(jobID, _, indent + 2)) + } else + jobLogInfo(jobID, indentString(indent) + stageInfo + " JOB_ID=" + stage.priority, false) + } + + // record task metrics into job log files + protected def recordTaskMetrics(stageID: Int, status: String, + taskInfo: TaskInfo, taskMetrics: TaskMetrics) { + val info = " TID=" + taskInfo.taskId + " STAGE_ID=" + stageID + + " START_TIME=" + taskInfo.launchTime + " FINISH_TIME=" + taskInfo.finishTime + + " EXECUTOR_ID=" + taskInfo.executorId + " HOST=" + taskMetrics.hostname + val executorRunTime = " EXECUTOR_RUN_TIME=" + taskMetrics.executorRunTime + val readMetrics = + taskMetrics.shuffleReadMetrics match { + case Some(metrics) => + " SHUFFLE_FINISH_TIME=" + metrics.shuffleFinishTime + + " BLOCK_FETCHED_TOTAL=" + metrics.totalBlocksFetched + + " BLOCK_FETCHED_LOCAL=" + metrics.localBlocksFetched + + " BLOCK_FETCHED_REMOTE=" + metrics.remoteBlocksFetched + + " REMOTE_FETCH_WAIT_TIME=" + metrics.fetchWaitTime + + " REMOTE_FETCH_TIME=" + metrics.remoteFetchTime + + " REMOTE_BYTES_READ=" + metrics.remoteBytesRead + case None => "" + } + val writeMetrics = + taskMetrics.shuffleWriteMetrics match { + case Some(metrics) => + " SHUFFLE_BYTES_WRITTEN=" + metrics.shuffleBytesWritten + case None => "" + } + stageLogInfo(stageID, status + info + executorRunTime + readMetrics + writeMetrics) + } + + override def onStageSubmitted(stage: Stage, info: String = "") { + eventQueue.put(JobLoggerOnStageSubmitted(stage, info)) + } + + protected def processStageSubmittedEvent(stage: Stage, info: String) { + stageLogInfo(stage.id, "STAGE_ID=" + stage.id + " STATUS=SUBMITTED " + info) + } + + override def onStageCompleted(stageCompleted: StageCompleted) { + eventQueue.put(JobLoggerOnStageCompleted(stageCompleted)) + } + + protected def processStageCompletedEvent(stageCompleted: StageCompleted) { + stageLogInfo(stageCompleted.stageInfo.stage.id, "STAGE_ID=" + + stageCompleted.stageInfo.stage.id + " STATUS=COMPLETED") + + } + + override def onTaskEnd(event: CompletionEvent) { + eventQueue.put(JobLoggerOnTaskEnd(event)) + } + + protected def processTaskEndEvent(event: CompletionEvent) { + var taskStatus = "" + event.task match { + case resultTask: ResultTask[_, _] => taskStatus = "TASK_TYPE=RESULT_TASK" + case shuffleMapTask: ShuffleMapTask => taskStatus = "TASK_TYPE=SHUFFLE_MAP_TASK" + } + event.reason match { + case Success => taskStatus += " STATUS=SUCCESS" + recordTaskMetrics(event.task.stageId, taskStatus, event.taskInfo, event.taskMetrics) + case Resubmitted => + taskStatus += " STATUS=RESUBMITTED TID=" + event.taskInfo.taskId + + " STAGE_ID=" + event.task.stageId + stageLogInfo(event.task.stageId, taskStatus) + case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => + taskStatus += " STATUS=FETCHFAILED TID=" + event.taskInfo.taskId + " STAGE_ID=" + + event.task.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" + + mapId + " REDUCE_ID=" + reduceId + stageLogInfo(event.task.stageId, taskStatus) + case OtherFailure(message) => + taskStatus += " STATUS=FAILURE TID=" + event.taskInfo.taskId + + " STAGE_ID=" + event.task.stageId + " INFO=" + message + stageLogInfo(event.task.stageId, taskStatus) + case _ => + } + } + + override def onJobEnd(job: ActiveJob, event: SparkListenerEvents) { + eventQueue.put(JobLoggerOnJobEnd(job, event)) + } + + protected def processJobEndEvent(job: ActiveJob, event: SparkListenerEvents) { + var info = "JOB_ID=" + job.runId + " STATUS=" + var validEvent = true + event match { + case SparkListenerJobSuccess => info += "SUCCESS" + case SparkListenerJobFailed(failedStage) => + info += "FAILED REASON=STAGE_FAILED FAILED_STAGE_ID=" + failedStage.id + case SparkListenerJobCancelled(reason) => info += "CANCELLED REASON=" + reason + case _ => validEvent = false + } + if (validEvent) { + jobLogInfo(job.runId, info) + closeLogWriter(job.runId) + } + } + + protected def recordJobProperties(jobID: Int, properties: Properties) { + if(properties != null) { + val annotation = properties.getProperty("spark.job.annotation", "") + jobLogInfo(jobID, annotation, false) + } + } + + override def onJobStart(job: ActiveJob, properties: Properties = null) { + eventQueue.put(JobLoggerOnJobStart(job, properties)) + } + + protected def processJobStartEvent(job: ActiveJob, properties: Properties) { + createLogWriter(job.runId) + recordJobProperties(job.runId, properties) + buildJobDep(job.runId, job.finalStage) + recordStageDep(job.runId) + recordStageDepGraph(job.runId, job.finalStage) + jobLogInfo(job.runId, "JOB_ID=" + job.runId + " STATUS=STARTED") + } +} diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index a65140b145..9cf7f3ffc0 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -1,27 +1,54 @@ package spark.scheduler +import java.util.Properties import spark.scheduler.cluster.TaskInfo import spark.util.Distribution -import spark.{Utils, Logging} +import spark.{Utils, Logging, SparkContext, TaskEndReason} import spark.executor.TaskMetrics trait SparkListener { /** * called when a stage is completed, with information on the completed stage */ - def onStageCompleted(stageCompleted: StageCompleted) + def onStageCompleted(stageCompleted: StageCompleted) { } + + /** + * called when a stage is submitted + */ + def onStageSubmitted(stage: Stage, info: String = "") { } + + /** + * called when a task ends + */ + def onTaskEnd(event: CompletionEvent) { } + + /** + * called when a job starts + */ + def onJobStart(job: ActiveJob, properties: Properties = null) { } + + /** + * called when a job ends + */ + def onJobEnd(job: ActiveJob, event: SparkListenerEvents) { } + } sealed trait SparkListenerEvents case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents +case object SparkListenerJobSuccess extends SparkListenerEvents + +case class SparkListenerJobFailed(failedStage: Stage) extends SparkListenerEvents + +case class SparkListenerJobCancelled(reason: String) extends SparkListenerEvents /** * Simple SparkListener that logs a few summary statistics when each stage completes */ class StatsReportListener extends SparkListener with Logging { - def onStageCompleted(stageCompleted: StageCompleted) { + override def onStageCompleted(stageCompleted: StageCompleted) { import spark.scheduler.StatsReportListener._ implicit val sc = stageCompleted this.logInfo("Finished stage: " + stageCompleted.stageInfo) diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala new file mode 100644 index 0000000000..34fd8b995e --- /dev/null +++ b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala @@ -0,0 +1,105 @@ +package spark.scheduler + +import java.util.Properties +import java.util.concurrent.LinkedBlockingQueue +import org.scalatest.FunSuite +import org.scalatest.matchers.ShouldMatchers +import scala.collection.mutable +import spark._ +import spark.SparkContext._ + + +class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers { + + test("inner method") { + sc = new SparkContext("local", "joblogger") + val joblogger = new JobLogger { + def createLogWriterTest(jobID: Int) = createLogWriter(jobID) + def closeLogWriterTest(jobID: Int) = closeLogWriter(jobID) + def getRddNameTest(rdd: RDD[_]) = getRddName(rdd) + def buildJobDepTest(jobID: Int, stage: Stage) = buildJobDep(jobID, stage) + } + type MyRDD = RDD[(Int, Int)] + def makeRdd( + numPartitions: Int, + dependencies: List[Dependency[_]] + ): MyRDD = { + val maxPartition = numPartitions - 1 + return new MyRDD(sc, dependencies) { + override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = + throw new RuntimeException("should not be reached") + override def getPartitions = (0 to maxPartition).map(i => new Partition { + override def index = i + }).toArray + } + } + val jobID = 5 + val parentRdd = makeRdd(4, Nil) + val shuffleDep = new ShuffleDependency(parentRdd, null) + val rootRdd = makeRdd(4, List(shuffleDep)) + val shuffleMapStage = new Stage(1, parentRdd, Some(shuffleDep), Nil, jobID) + val rootStage = new Stage(0, rootRdd, None, List(shuffleMapStage), jobID) + + joblogger.onStageSubmitted(rootStage) + joblogger.getEventQueue.size should be (1) + joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getName) + parentRdd.setName("MyRDD") + joblogger.getRddNameTest(parentRdd) should be ("MyRDD") + joblogger.createLogWriterTest(jobID) + joblogger.getJobIDtoPrintWriter.size should be (1) + joblogger.buildJobDepTest(jobID, rootStage) + joblogger.getJobIDToStages.get(jobID).get.size should be (2) + joblogger.getStageIDToJobID.get(0) should be (Some(jobID)) + joblogger.getStageIDToJobID.get(1) should be (Some(jobID)) + joblogger.closeLogWriterTest(jobID) + joblogger.getStageIDToJobID.size should be (0) + joblogger.getJobIDToStages.size should be (0) + joblogger.getJobIDtoPrintWriter.size should be (0) + } + + test("inner variables") { + sc = new SparkContext("local[4]", "joblogger") + val joblogger = new JobLogger { + override protected def closeLogWriter(jobID: Int) = + getJobIDtoPrintWriter.get(jobID).foreach { fileWriter => + fileWriter.close() + } + } + sc.addSparkListener(joblogger) + val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) } + rdd.reduceByKey(_+_).collect() + + joblogger.getLogDir should be ("/tmp/spark") + joblogger.getJobIDtoPrintWriter.size should be (1) + joblogger.getStageIDToJobID.size should be (2) + joblogger.getStageIDToJobID.get(0) should be (Some(0)) + joblogger.getStageIDToJobID.get(1) should be (Some(0)) + joblogger.getJobIDToStages.size should be (1) + } + + + test("interface functions") { + sc = new SparkContext("local[4]", "joblogger") + val joblogger = new JobLogger { + var onTaskEndCount = 0 + var onJobEndCount = 0 + var onJobStartCount = 0 + var onStageCompletedCount = 0 + var onStageSubmittedCount = 0 + override def onTaskEnd(event: CompletionEvent) = onTaskEndCount += 1 + override def onJobEnd(job: ActiveJob, event: SparkListenerEvents) = onJobEndCount += 1 + override def onJobStart(job: ActiveJob, properties: Properties) = onJobStartCount += 1 + override def onStageCompleted(stageCompleted: StageCompleted) = onStageCompletedCount += 1 + override def onStageSubmitted(stage: Stage, info: String = "") = onStageSubmittedCount += 1 + } + sc.addSparkListener(joblogger) + val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) } + rdd.reduceByKey(_+_).collect() + + joblogger.onJobStartCount should be (1) + joblogger.onJobEndCount should be (1) + joblogger.onTaskEndCount should be (8) + joblogger.onStageSubmittedCount should be (2) + joblogger.onStageCompletedCount should be (2) + } +} diff --git a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala index 42a87d8b90..48aa67c543 100644 --- a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala @@ -77,7 +77,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc class SaveStageInfo extends SparkListener { val stageInfos = mutable.Buffer[StageInfo]() - def onStageCompleted(stage: StageCompleted) { + override def onStageCompleted(stage: StageCompleted) { stageInfos += stage.stageInfo } } -- cgit v1.2.3 From 4fd86e0e10149ad1803831a308a056c7105cbe67 Mon Sep 17 00:00:00 2001 From: Mingfei Date: Sat, 8 Jun 2013 15:45:47 +0800 Subject: delete test code for joblogger in SparkContext --- core/src/main/scala/spark/SparkContext.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index b67a2066c8..70a9d7698c 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -48,7 +48,6 @@ import spark.scheduler.local.LocalScheduler import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import spark.storage.{BlockManagerUI, StorageStatus, StorageUtils, RDDInfo} import spark.util.{MetadataCleaner, TimeStampedHashMap} -import spark.scheduler.JobLogger /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -510,7 +509,7 @@ class SparkContext( def addSparkListener(listener: SparkListener) { dagScheduler.sparkListeners += listener } - addSparkListener(new JobLogger) + /** * Return a map from the slave to the max memory available for caching and the remaining * memory available for caching. -- cgit v1.2.3 From ade822011d44bd43e9ac78c1d29ec924a1f6e8e7 Mon Sep 17 00:00:00 2001 From: Mingfei Date: Sat, 8 Jun 2013 16:26:45 +0800 Subject: not check return value of eventQueue.take --- .../src/main/scala/spark/scheduler/JobLogger.scala | 28 ++++++++++------------ 1 file changed, 13 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index f87acfd0b6..46b9fa974b 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -48,21 +48,19 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { override def run() { while (true) { val event = eventQueue.take - if (event != null) { - logDebug("Got event of type " + event.getClass.getName) - event match { - case JobLoggerOnJobStart(job, info) => - processJobStartEvent(job, info) - case JobLoggerOnStageSubmitted(stage, info) => - processStageSubmittedEvent(stage, info) - case JobLoggerOnStageCompleted(stageCompleted) => - processStageCompletedEvent(stageCompleted) - case JobLoggerOnJobEnd(job, event) => - processJobEndEvent(job, event) - case JobLoggerOnTaskEnd(event) => - processTaskEndEvent(event) - case _ => - } + logDebug("Got event of type " + event.getClass.getName) + event match { + case JobLoggerOnJobStart(job, info) => + processJobStartEvent(job, info) + case JobLoggerOnStageSubmitted(stage, info) => + processStageSubmittedEvent(stage, info) + case JobLoggerOnStageCompleted(stageCompleted) => + processStageCompletedEvent(stageCompleted) + case JobLoggerOnJobEnd(job, event) => + processJobEndEvent(job, event) + case JobLoggerOnTaskEnd(event) => + processTaskEndEvent(event) + case _ => } } } -- cgit v1.2.3 From d1bbcebae580220076ceaa65f84dcf984ab51a16 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 8 Jun 2013 16:58:42 -0700 Subject: Adding compression to Hadoop save functions --- core/src/main/scala/spark/PairRDDFunctions.scala | 39 +++++++++++++++++- core/src/main/scala/spark/RDD.scala | 9 ++++ .../scala/spark/SequenceFileRDDFunctions.scala | 15 ++++--- core/src/test/scala/spark/FileSuite.scala | 48 ++++++++++++++++++++++ 4 files changed, 105 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 2b0e697337..9bf1227d65 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -10,6 +10,8 @@ import scala.collection.JavaConversions._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.compress.CompressionCodec +import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.mapred.FileOutputCommitter import org.apache.hadoop.mapred.FileOutputFormat import org.apache.hadoop.mapred.HadoopWriter @@ -515,6 +517,16 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]]) } + /** + * Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class + * supporting the key and value types K and V in this RDD. Compress the result with the + * supplied codec. + */ + def saveAsHadoopFile[F <: OutputFormat[K, V]]( + path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassManifest[F]) { + saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]], codec) + } + /** * Output the RDD to any Hadoop-supported file system, using a new Hadoop API `OutputFormat` * (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD. @@ -574,6 +586,20 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( jobCommitter.cleanupJob(jobTaskContext) } + /** + * Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class + * supporting the key and value types K and V in this RDD. Compress with the supplied codec. + */ + def saveAsHadoopFile( + path: String, + keyClass: Class[_], + valueClass: Class[_], + outputFormatClass: Class[_ <: OutputFormat[_, _]], + codec: Class[_ <: CompressionCodec]) { + saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass, + new JobConf(self.context.hadoopConfiguration), Some(codec)) + } + /** * Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class * supporting the key and value types K and V in this RDD. @@ -583,11 +609,22 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( keyClass: Class[_], valueClass: Class[_], outputFormatClass: Class[_ <: OutputFormat[_, _]], - conf: JobConf = new JobConf(self.context.hadoopConfiguration)) { + conf: JobConf = new JobConf(self.context.hadoopConfiguration), + codec: Option[Class[_ <: CompressionCodec]] = None) { conf.setOutputKeyClass(keyClass) conf.setOutputValueClass(valueClass) // conf.setOutputFormat(outputFormatClass) // Doesn't work in Scala 2.9 due to what may be a generics bug conf.set("mapred.output.format.class", outputFormatClass.getName) + codec match { + case Some(c) => { + conf.setCompressMapOutput(true) + conf.set("mapred.output.compress", "true") + conf.setMapOutputCompressorClass(c) + conf.set("mapred.output.compression.codec", c.getCanonicalName) + conf.set("mapred.output.compression.type", CompressionType.BLOCK.toString) + } + case _ => + } conf.setOutputCommitter(classOf[FileOutputCommitter]) FileOutputFormat.setOutputPath(conf, HadoopWriter.createPathFromString(path, conf)) saveAsHadoopDataset(conf) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index e6c0438d76..e5995bea22 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -7,6 +7,7 @@ import scala.collection.JavaConversions.mapAsScalaMap import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.io.BytesWritable +import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.io.NullWritable import org.apache.hadoop.io.Text import org.apache.hadoop.mapred.TextOutputFormat @@ -730,6 +731,14 @@ abstract class RDD[T: ClassManifest]( .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path) } + /** + * Save this RDD as a compressed text file, using string representations of elements. + */ + def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) { + this.map(x => (NullWritable.get(), new Text(x.toString))) + .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path, codec) + } + /** * Save this RDD as a SequenceFile of serialized objects. */ diff --git a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala index 518034e07b..2911f9036e 100644 --- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala @@ -18,6 +18,7 @@ import org.apache.hadoop.mapred.TextOutputFormat import org.apache.hadoop.mapred.SequenceFileOutputFormat import org.apache.hadoop.mapred.OutputCommitter import org.apache.hadoop.mapred.FileOutputCommitter +import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.io.Writable import org.apache.hadoop.io.NullWritable import org.apache.hadoop.io.BytesWritable @@ -62,7 +63,7 @@ class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : Cla * byte arrays to BytesWritable, and Strings to Text. The `path` can be on any Hadoop-supported * file system. */ - def saveAsSequenceFile(path: String) { + def saveAsSequenceFile(path: String, codec: Option[Class[_ <: CompressionCodec]] = None) { def anyToWritable[U <% Writable](u: U): Writable = u val keyClass = getWritableClass[K] @@ -72,14 +73,18 @@ class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : Cla logInfo("Saving as sequence file of type (" + keyClass.getSimpleName + "," + valueClass.getSimpleName + ")" ) val format = classOf[SequenceFileOutputFormat[Writable, Writable]] + val jobConf = new JobConf(self.context.hadoopConfiguration) if (!convertKey && !convertValue) { - self.saveAsHadoopFile(path, keyClass, valueClass, format) + self.saveAsHadoopFile(path, keyClass, valueClass, format, jobConf, codec) } else if (!convertKey && convertValue) { - self.map(x => (x._1,anyToWritable(x._2))).saveAsHadoopFile(path, keyClass, valueClass, format) + self.map(x => (x._1,anyToWritable(x._2))).saveAsHadoopFile( + path, keyClass, valueClass, format, jobConf, codec) } else if (convertKey && !convertValue) { - self.map(x => (anyToWritable(x._1),x._2)).saveAsHadoopFile(path, keyClass, valueClass, format) + self.map(x => (anyToWritable(x._1),x._2)).saveAsHadoopFile( + path, keyClass, valueClass, format, jobConf, codec) } else if (convertKey && convertValue) { - self.map(x => (anyToWritable(x._1),anyToWritable(x._2))).saveAsHadoopFile(path, keyClass, valueClass, format) + self.map(x => (anyToWritable(x._1),anyToWritable(x._2))).saveAsHadoopFile( + path, keyClass, valueClass, format, jobConf, codec) } } } diff --git a/core/src/test/scala/spark/FileSuite.scala b/core/src/test/scala/spark/FileSuite.scala index 91b48c7456..a5d2028591 100644 --- a/core/src/test/scala/spark/FileSuite.scala +++ b/core/src/test/scala/spark/FileSuite.scala @@ -7,6 +7,8 @@ import scala.io.Source import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.hadoop.io._ +import org.apache.hadoop.io.compress.{DefaultCodec, CompressionCodec, GzipCodec} + import SparkContext._ @@ -26,6 +28,29 @@ class FileSuite extends FunSuite with LocalSparkContext { assert(sc.textFile(outputDir).collect().toList === List("1", "2", "3", "4")) } + test("text files (compressed)") { + sc = new SparkContext("local", "test") + val tempDir = Files.createTempDir() + val normalDir = new File(tempDir, "output_normal").getAbsolutePath + val compressedOutputDir = new File(tempDir, "output_compressed").getAbsolutePath + val codec = new DefaultCodec() + + val data = sc.parallelize("a" * 10000, 1) + data.saveAsTextFile(normalDir) + data.saveAsTextFile(compressedOutputDir, classOf[DefaultCodec]) + + val normalFile = new File(normalDir, "part-00000") + val normalContent = sc.textFile(normalDir).collect + assert(normalContent === Array.fill(10000)("a")) + + val compressedFile = new File(compressedOutputDir, "part-00000" + codec.getDefaultExtension) + val compressedContent = sc.textFile(compressedOutputDir).collect + assert(compressedContent === Array.fill(10000)("a")) + + assert(compressedFile.length < normalFile.length) + } + + test("SequenceFiles") { sc = new SparkContext("local", "test") val tempDir = Files.createTempDir() @@ -37,6 +62,29 @@ class FileSuite extends FunSuite with LocalSparkContext { assert(output.map(_.toString).collect().toList === List("(1,a)", "(2,aa)", "(3,aaa)")) } + test("SequenceFile (compressed)") { + sc = new SparkContext("local", "test") + val tempDir = Files.createTempDir() + val normalDir = new File(tempDir, "output_normal").getAbsolutePath + val compressedOutputDir = new File(tempDir, "output_compressed").getAbsolutePath + val codec = new DefaultCodec() + + val data = sc.parallelize(Seq.fill(100)("abc"), 1).map(x => (x, x)) + data.saveAsSequenceFile(normalDir) + data.saveAsSequenceFile(compressedOutputDir, Some(classOf[DefaultCodec])) + + val normalFile = new File(normalDir, "part-00000") + val normalContent = sc.sequenceFile[String, String](normalDir).collect + assert(normalContent === Array.fill(100)("abc", "abc")) + + val compressedFile = new File(compressedOutputDir, "part-00000" + codec.getDefaultExtension) + val compressedContent = sc.sequenceFile[String, String](compressedOutputDir).collect + assert(compressedContent === Array.fill(100)("abc", "abc")) + + assert(compressedFile.length < normalFile.length) + } + + test("SequenceFile with writable key") { sc = new SparkContext("local", "test") val tempDir = Files.createTempDir() -- cgit v1.2.3 From 083a3485abdcda5913c2186c4a7930ac07b061c4 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 9 Jun 2013 11:49:33 -0700 Subject: Clean extra whitespace --- core/src/test/scala/spark/FileSuite.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/test/scala/spark/FileSuite.scala b/core/src/test/scala/spark/FileSuite.scala index a5d2028591..e61ff7793d 100644 --- a/core/src/test/scala/spark/FileSuite.scala +++ b/core/src/test/scala/spark/FileSuite.scala @@ -50,7 +50,6 @@ class FileSuite extends FunSuite with LocalSparkContext { assert(compressedFile.length < normalFile.length) } - test("SequenceFiles") { sc = new SparkContext("local", "test") val tempDir = Files.createTempDir() @@ -84,7 +83,6 @@ class FileSuite extends FunSuite with LocalSparkContext { assert(compressedFile.length < normalFile.length) } - test("SequenceFile with writable key") { sc = new SparkContext("local", "test") val tempDir = Files.createTempDir() -- cgit v1.2.3 From df592192e736edca9e382a7f92e15bead390ef65 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 9 Jun 2013 18:09:24 -0700 Subject: Monads FTW --- core/src/main/scala/spark/PairRDDFunctions.scala | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 9bf1227d65..15593db0d9 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -615,15 +615,12 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( conf.setOutputValueClass(valueClass) // conf.setOutputFormat(outputFormatClass) // Doesn't work in Scala 2.9 due to what may be a generics bug conf.set("mapred.output.format.class", outputFormatClass.getName) - codec match { - case Some(c) => { - conf.setCompressMapOutput(true) - conf.set("mapred.output.compress", "true") - conf.setMapOutputCompressorClass(c) - conf.set("mapred.output.compression.codec", c.getCanonicalName) - conf.set("mapred.output.compression.type", CompressionType.BLOCK.toString) - } - case _ => + for (c <- codec) { + conf.setCompressMapOutput(true) + conf.set("mapred.output.compress", "true") + conf.setMapOutputCompressorClass(c) + conf.set("mapred.output.compression.codec", c.getCanonicalName) + conf.set("mapred.output.compression.type", CompressionType.BLOCK.toString) } conf.setOutputCommitter(classOf[FileOutputCommitter]) FileOutputFormat.setOutputPath(conf, HadoopWriter.createPathFromString(path, conf)) -- cgit v1.2.3 From ef14dc2e7736732932d4edceb3be8d81ba9f8bc7 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 9 Jun 2013 18:09:46 -0700 Subject: Adding Java-API version of compression codec --- .../main/scala/spark/api/java/JavaPairRDD.scala | 11 ++++++ .../main/scala/spark/api/java/JavaRDDLike.scala | 8 ++++ core/src/test/scala/spark/JavaAPISuite.java | 46 ++++++++++++++++++++++ 3 files changed, 65 insertions(+) diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index 30084df4e2..76051597b6 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -6,6 +6,7 @@ import java.util.Comparator import scala.Tuple2 import scala.collection.JavaConversions._ +import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.OutputFormat import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} @@ -459,6 +460,16 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif rdd.saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass) } + /** Output the RDD to any Hadoop-supported file system, compressing with the supplied codec. */ + def saveAsHadoopFile[F <: OutputFormat[_, _]]( + path: String, + keyClass: Class[_], + valueClass: Class[_], + outputFormatClass: Class[F], + codec: Class[_ <: CompressionCodec]) { + rdd.saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass, codec) + } + /** Output the RDD to any Hadoop-supported file system. */ def saveAsNewAPIHadoopFile[F <: NewOutputFormat[_, _]]( path: String, diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index 9b74d1226f..76b14e2e04 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -4,6 +4,7 @@ import java.util.{List => JList} import scala.Tuple2 import scala.collection.JavaConversions._ +import org.apache.hadoop.io.compress.CompressionCodec import spark.{SparkContext, Partition, RDD, TaskContext} import spark.api.java.JavaPairRDD._ import spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _} @@ -310,6 +311,13 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def saveAsTextFile(path: String) = rdd.saveAsTextFile(path) + + /** + * Save this RDD as a compressed text file, using string representations of elements. + */ + def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) = + rdd.saveAsTextFile(path, codec) + /** * Save this RDD as a SequenceFile of serialized objects. */ diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 93bb69b41c..6caa85119a 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -8,6 +8,7 @@ import java.util.*; import scala.Tuple2; import com.google.common.base.Charsets; +import org.apache.hadoop.io.compress.DefaultCodec; import com.google.common.io.Files; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; @@ -473,6 +474,19 @@ public class JavaAPISuite implements Serializable { Assert.assertEquals(expected, readRDD.collect()); } + @Test + public void textFilesCompressed() throws IOException { + File tempDir = Files.createTempDir(); + String outputDir = new File(tempDir, "output").getAbsolutePath(); + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + rdd.saveAsTextFile(outputDir, DefaultCodec.class); + + // Try reading it in as a text file RDD + List expected = Arrays.asList("1", "2", "3", "4"); + JavaRDD readRDD = sc.textFile(outputDir); + Assert.assertEquals(expected, readRDD.collect()); + } + @Test public void sequenceFile() { File tempDir = Files.createTempDir(); @@ -619,6 +633,38 @@ public class JavaAPISuite implements Serializable { }).collect().toString()); } + @Test + public void hadoopFileCompressed() { + File tempDir = Files.createTempDir(); + String outputDir = new File(tempDir, "output_compressed").getAbsolutePath(); + List> pairs = Arrays.asList( + new Tuple2(1, "a"), + new Tuple2(2, "aa"), + new Tuple2(3, "aaa") + ); + JavaPairRDD rdd = sc.parallelizePairs(pairs); + + rdd.map(new PairFunction, IntWritable, Text>() { + @Override + public Tuple2 call(Tuple2 pair) { + return new Tuple2(new IntWritable(pair._1()), new Text(pair._2())); + } + }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class, + DefaultCodec.class); + + System.out.println(outputDir); + JavaPairRDD output = sc.hadoopFile(outputDir, + SequenceFileInputFormat.class, IntWritable.class, Text.class); + + Assert.assertEquals(pairs.toString(), output.map(new Function, + String>() { + @Override + public String call(Tuple2 x) { + return x.toString(); + } + }).collect().toString()); + } + @Test public void zip() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); -- cgit v1.2.3 From 190ec617997d621c11ed1aab662a6e3a06815d2f Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Mon, 10 Jun 2013 15:27:02 +0800 Subject: change code style and debug info --- core/src/main/scala/spark/scheduler/local/LocalScheduler.scala | 8 +++----- .../main/scala/spark/scheduler/local/LocalTaskSetManager.scala | 1 - 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 69dacfc2bd..93d4318b29 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -34,8 +34,7 @@ private[spark] class LocalActor(localScheduler: LocalScheduler, var freeCores: I } def launchTask(tasks : Seq[TaskDescription]) { - for (task <- tasks) - { + for (task <- tasks) { freeCores -= 1 localScheduler.threadPool.submit(new Runnable { def run() { @@ -85,8 +84,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: } schedulableBuilder.buildPools() - localActor = env.actorSystem.actorOf( - Props(new LocalActor(this, threads)), "Test") + localActor = env.actorSystem.actorOf(Props(new LocalActor(this, threads)), "Test") } override def setListener(listener: TaskSchedulerListener) { @@ -109,7 +107,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: val tasks = new ArrayBuffer[TaskDescription](freeCores) val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() for (manager <- sortedTaskSetQueue) { - logInfo("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks)) + logDebug("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks)) } var launchTask = false diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index f2e07d162a..70b69bb26f 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -91,7 +91,6 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas if (availableCpus > 0 && numFinished < numTasks) { findTask() match { case Some(index) => - logInfo(taskSet.tasks(index).toString) val taskId = sched.attemptId.getAndIncrement() val task = taskSet.tasks(index) val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) -- cgit v1.2.3 From fd6148c8b20bc051786ff574d3b8f3b5e79b391a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 10 Jun 2013 10:27:25 -0700 Subject: Removing print statement --- core/src/test/scala/spark/JavaAPISuite.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 6caa85119a..d306124fca 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -652,7 +652,6 @@ public class JavaAPISuite implements Serializable { }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class, DefaultCodec.class); - System.out.println(outputDir); JavaPairRDD output = sc.hadoopFile(outputDir, SequenceFileInputFormat.class, IntWritable.class, Text.class); -- cgit v1.2.3 From dc4073654b1707f115de30088938f6e53efda0ba Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 11 Jun 2013 00:08:02 -0400 Subject: Revert "Fix start-slave not passing instance number to spark-daemon." This reverts commit a674d67c0aebb940e3b816e2307206115baec175. --- bin/start-slave.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/start-slave.sh b/bin/start-slave.sh index dfcbc6981b..26b5b9d462 100755 --- a/bin/start-slave.sh +++ b/bin/start-slave.sh @@ -12,4 +12,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then fi fi -"$bin"/spark-daemon.sh start spark.deploy.worker.Worker 1 "$@" +"$bin"/spark-daemon.sh start spark.deploy.worker.Worker "$@" -- cgit v1.2.3 From db5bca08ff00565732946a9c0a0244a9f7021d82 Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 12 Jun 2013 10:54:16 -0700 Subject: add a new top K method to RDD using a bounded priority queue --- core/src/main/scala/spark/RDD.scala | 24 +++++++++++ .../scala/spark/util/BoundedPriorityQueue.scala | 48 ++++++++++++++++++++++ core/src/test/scala/spark/RDDSuite.scala | 19 +++++++++ 3 files changed, 91 insertions(+) create mode 100644 core/src/main/scala/spark/util/BoundedPriorityQueue.scala diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index e6c0438d76..ec5e5e2433 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -35,6 +35,7 @@ import spark.rdd.ZippedPartitionsRDD2 import spark.rdd.ZippedPartitionsRDD3 import spark.rdd.ZippedPartitionsRDD4 import spark.storage.StorageLevel +import spark.util.BoundedPriorityQueue import SparkContext._ @@ -722,6 +723,29 @@ abstract class RDD[T: ClassManifest]( case _ => throw new UnsupportedOperationException("empty collection") } + /** + * Returns the top K elements from this RDD as defined by + * the specified implicit Ordering[T]. + * @param num the number of top elements to return + * @param ord the implicit ordering for T + * @return an array of top elements + */ + def top(num: Int)(implicit ord: Ordering[T]): Array[T] = { + val topK = mapPartitions { items => + val queue = new BoundedPriorityQueue[T](num) + queue ++= items + Iterator(queue) + }.reduce { (queue1, queue2) => + queue1 ++= queue2 + queue1 + } + + val builder = Array.newBuilder[T] + builder.sizeHint(topK.size) + builder ++= topK + builder.result() + } + /** * Save this RDD as a text file, using string representations of elements. */ diff --git a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala new file mode 100644 index 0000000000..53ee95a02e --- /dev/null +++ b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala @@ -0,0 +1,48 @@ +package spark.util + +import java.util.{PriorityQueue => JPriorityQueue} +import scala.collection.generic.Growable + +/** + * Bounded priority queue. This class modifies the original PriorityQueue's + * add/offer methods such that only the top K elements are retained. The top + * K elements are defined by an implicit Ordering[A]. + */ +class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A], mf: ClassManifest[A]) + extends JPriorityQueue[A](maxSize, ord) with Growable[A] { + + override def offer(a: A): Boolean = { + if (size < maxSize) super.offer(a) + else maybeReplaceLowest(a) + } + + override def add(a: A): Boolean = offer(a) + + override def ++=(xs: TraversableOnce[A]): this.type = { + xs.foreach(add) + this + } + + override def +=(elem: A): this.type = { + add(elem) + this + } + + override def +=(elem1: A, elem2: A, elems: A*): this.type = { + this += elem1 += elem2 ++= elems + } + + private def maybeReplaceLowest(a: A): Boolean = { + val head = peek() + if (head != null && ord.gt(a, head)) { + poll() + super.offer(a) + } else false + } +} + +object BoundedPriorityQueue { + import scala.collection.JavaConverters._ + implicit def asIterable[A](queue: BoundedPriorityQueue[A]): Iterable[A] = queue.asScala +} + diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 3f69e99780..67f3332d44 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -317,4 +317,23 @@ class RDDSuite extends FunSuite with LocalSparkContext { assert(sample.size === checkSample.size) for (i <- 0 until sample.size) assert(sample(i) === checkSample(i)) } + + test("top with predefined ordering") { + sc = new SparkContext("local", "test") + val nums = Array.range(1, 100000) + val ints = sc.makeRDD(scala.util.Random.shuffle(nums), 2) + val topK = ints.top(5) + assert(topK.size === 5) + assert(topK.sorted === nums.sorted.takeRight(5)) + } + + test("top with custom ordering") { + sc = new SparkContext("local", "test") + val words = Vector("a", "b", "c", "d") + implicit val ord = implicitly[Ordering[String]].reverse + val rdd = sc.makeRDD(words, 2) + val topK = rdd.top(2) + assert(topK.size === 2) + assert(topK.sorted === Array("b", "a")) + } } -- cgit v1.2.3 From 3f96c6f27b08039fb7b8d295f5de2083544e979f Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 12 Jun 2013 17:20:05 -0700 Subject: Fixed jvmArgs in maven build. --- pom.xml | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index ce77ba37c6..c893ec755e 100644 --- a/pom.xml +++ b/pom.xml @@ -60,7 +60,7 @@ 4.1.2 1.2.17 - 0m + 64m 512m @@ -395,10 +395,8 @@ -Xms64m -Xmx1024m - -XX:PermSize - ${PermGen} - -XX:MaxPermSize - ${MaxPermGen} + -XX:PermSize=${PermGen} + -XX:MaxPermSize=${MaxPermGen} -source -- cgit v1.2.3 From 967a6a699da7da007f51e59d085a357da5ec14da Mon Sep 17 00:00:00 2001 From: Mingfei Date: Thu, 13 Jun 2013 14:36:07 +0800 Subject: modify sparklister function interface according to comments --- .../main/scala/spark/scheduler/DAGScheduler.scala | 15 ++-- .../src/main/scala/spark/scheduler/JobLogger.scala | 89 +++++++++++----------- .../main/scala/spark/scheduler/SparkListener.scala | 38 +++++---- .../scala/spark/scheduler/JobLoggerSuite.scala | 10 +-- 4 files changed, 79 insertions(+), 73 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 43dd7d6534..e281e5a8db 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -289,7 +289,7 @@ class DAGScheduler( val finalStage = newStage(finalRDD, None, runId) val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener, properties) clearCacheLocs() - sparkListeners.foreach(_.onJobStart(job, properties)) + sparkListeners.foreach(_.onJobStart(SparkListenerJobStart(job, properties))) logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length + " output partitions (allowLocal=" + allowLocal + ")") logInfo("Final stage: " + finalStage + " (" + finalStage.origin + ")") @@ -312,7 +312,7 @@ class DAGScheduler( handleExecutorLost(execId) case completion: CompletionEvent => - sparkListeners.foreach(_.onTaskEnd(completion)) + sparkListeners.foreach(_.onTaskEnd(SparkListenerTaskEnd(completion))) handleTaskCompletion(completion) case TaskSetFailed(taskSet, reason) => @@ -323,8 +323,8 @@ class DAGScheduler( for (job <- activeJobs) { val error = new SparkException("Job cancelled because SparkContext was shut down") job.listener.jobFailed(error) - val JobCancelEvent = new SparkListenerJobCancelled("SPARKCONTEXT_SHUTDOWN") - sparkListeners.foreach(_.onJobEnd(job, JobCancelEvent)) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobCancelled(job, + "SPARKCONTEXT_SHUTDOWN"))) } return true } @@ -472,7 +472,7 @@ class DAGScheduler( } } if (tasks.size > 0) { - sparkListeners.foreach(_.onStageSubmitted(stage, "TASKS_SIZE=" + tasks.size)) + sparkListeners.foreach(_.onStageSubmitted(SparkListenerStageSubmitted(stage, tasks.size))) logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks logDebug("New pending tasks: " + myPending) @@ -527,7 +527,7 @@ class DAGScheduler( activeJobs -= job resultStageToJob -= stage markStageAsFinished(stage) - sparkListeners.foreach(_.onJobEnd(job, SparkListenerJobSuccess)) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobSuccess(job))) } job.listener.taskSucceeded(rt.outputId, event.result) } @@ -671,8 +671,7 @@ class DAGScheduler( job.listener.jobFailed(new SparkException("Job failed: " + reason)) activeJobs -= job resultStageToJob -= resultStage - val jobFailedEvent = new SparkListenerJobFailed(failedStage) - sparkListeners.foreach(_.onJobEnd(job, jobFailedEvent)) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobFailed(job, failedStage))) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index 46b9fa974b..002c5826cb 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -15,13 +15,6 @@ import spark.scheduler.cluster.TaskInfo // used to record runtime information for each job, including RDD graph // tasks' start/stop shuffle information and information from outside -sealed trait JobLoggerEvent -case class JobLoggerOnJobStart(job: ActiveJob, properties: Properties) extends JobLoggerEvent -case class JobLoggerOnStageSubmitted(stage: Stage, info: String) extends JobLoggerEvent -case class JobLoggerOnStageCompleted(stageCompleted: StageCompleted) extends JobLoggerEvent -case class JobLoggerOnJobEnd(job: ActiveJob, event: SparkListenerEvents) extends JobLoggerEvent -case class JobLoggerOnTaskEnd(event: CompletionEvent) extends JobLoggerEvent - class JobLogger(val logDirName: String) extends SparkListener with Logging { private val logDir = if (System.getenv("SPARK_LOG_DIR") != null) @@ -32,7 +25,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { private val stageIDToJobID = new HashMap[Int, Int] private val jobIDToStages = new HashMap[Int, ListBuffer[Stage]] private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - private val eventQueue = new LinkedBlockingQueue[JobLoggerEvent] + private val eventQueue = new LinkedBlockingQueue[SparkListenerEvents] createLogDir() def this() = this(String.valueOf(System.currentTimeMillis())) @@ -50,15 +43,19 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { val event = eventQueue.take logDebug("Got event of type " + event.getClass.getName) event match { - case JobLoggerOnJobStart(job, info) => - processJobStartEvent(job, info) - case JobLoggerOnStageSubmitted(stage, info) => - processStageSubmittedEvent(stage, info) - case JobLoggerOnStageCompleted(stageCompleted) => - processStageCompletedEvent(stageCompleted) - case JobLoggerOnJobEnd(job, event) => - processJobEndEvent(job, event) - case JobLoggerOnTaskEnd(event) => + case SparkListenerJobStart(job, properties) => + processJobStartEvent(job, properties) + case SparkListenerStageSubmitted(stage, taskSize) => + processStageSubmittedEvent(stage, taskSize) + case StageCompleted(stageInfo) => + processStageCompletedEvent(stageInfo) + case SparkListenerJobSuccess(job) => + processJobEndEvent(job) + case SparkListenerJobFailed(job, failedStage) => + processJobEndEvent(job, failedStage) + case SparkListenerJobCancelled(job, reason) => + processJobEndEvent(job, reason) + case SparkListenerTaskEnd(event) => processTaskEndEvent(event) case _ => } @@ -225,26 +222,26 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { stageLogInfo(stageID, status + info + executorRunTime + readMetrics + writeMetrics) } - override def onStageSubmitted(stage: Stage, info: String = "") { - eventQueue.put(JobLoggerOnStageSubmitted(stage, info)) + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { + eventQueue.put(stageSubmitted) } - protected def processStageSubmittedEvent(stage: Stage, info: String) { - stageLogInfo(stage.id, "STAGE_ID=" + stage.id + " STATUS=SUBMITTED " + info) + protected def processStageSubmittedEvent(stage: Stage, taskSize: Int) { + stageLogInfo(stage.id, "STAGE_ID=" + stage.id + " STATUS=SUBMITTED" + " TASK_SIZE=" + taskSize) } override def onStageCompleted(stageCompleted: StageCompleted) { - eventQueue.put(JobLoggerOnStageCompleted(stageCompleted)) + eventQueue.put(stageCompleted) } - protected def processStageCompletedEvent(stageCompleted: StageCompleted) { - stageLogInfo(stageCompleted.stageInfo.stage.id, "STAGE_ID=" + - stageCompleted.stageInfo.stage.id + " STATUS=COMPLETED") + protected def processStageCompletedEvent(stageInfo: StageInfo) { + stageLogInfo(stageInfo.stage.id, "STAGE_ID=" + + stageInfo.stage.id + " STATUS=COMPLETED") } - override def onTaskEnd(event: CompletionEvent) { - eventQueue.put(JobLoggerOnTaskEnd(event)) + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + eventQueue.put(taskEnd) } protected def processTaskEndEvent(event: CompletionEvent) { @@ -273,24 +270,26 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { } } - override def onJobEnd(job: ActiveJob, event: SparkListenerEvents) { - eventQueue.put(JobLoggerOnJobEnd(job, event)) + override def onJobEnd(jobEnd: SparkListenerEvents) { + eventQueue.put(jobEnd) } - protected def processJobEndEvent(job: ActiveJob, event: SparkListenerEvents) { - var info = "JOB_ID=" + job.runId + " STATUS=" - var validEvent = true - event match { - case SparkListenerJobSuccess => info += "SUCCESS" - case SparkListenerJobFailed(failedStage) => - info += "FAILED REASON=STAGE_FAILED FAILED_STAGE_ID=" + failedStage.id - case SparkListenerJobCancelled(reason) => info += "CANCELLED REASON=" + reason - case _ => validEvent = false - } - if (validEvent) { - jobLogInfo(job.runId, info) - closeLogWriter(job.runId) - } + protected def processJobEndEvent(job: ActiveJob) { + val info = "JOB_ID=" + job.runId + " STATUS=SUCCESS" + jobLogInfo(job.runId, info) + closeLogWriter(job.runId) + } + + protected def processJobEndEvent(job: ActiveJob, failedStage: Stage) { + val info = "JOB_ID=" + job.runId + " STATUS=FAILED REASON=STAGE_FAILED FAILED_STAGE_ID=" + + failedStage.id + jobLogInfo(job.runId, info) + closeLogWriter(job.runId) + } + protected def processJobEndEvent(job: ActiveJob, reason: String) { + var info = "JOB_ID=" + job.runId + " STATUS=CANCELLED REASON=" + reason + jobLogInfo(job.runId, info) + closeLogWriter(job.runId) } protected def recordJobProperties(jobID: Int, properties: Properties) { @@ -300,8 +299,8 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { } } - override def onJobStart(job: ActiveJob, properties: Properties = null) { - eventQueue.put(JobLoggerOnJobStart(job, properties)) + override def onJobStart(jobStart: SparkListenerJobStart) { + eventQueue.put(jobStart) } protected def processJobStartEvent(job: ActiveJob, properties: Properties) { diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 9cf7f3ffc0..9265261dc1 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -6,6 +6,24 @@ import spark.util.Distribution import spark.{Utils, Logging, SparkContext, TaskEndReason} import spark.executor.TaskMetrics + +sealed trait SparkListenerEvents + +case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int) extends SparkListenerEvents + +case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents + +case class SparkListenerTaskEnd(event: CompletionEvent) extends SparkListenerEvents + +case class SparkListenerJobStart(job: ActiveJob, properties: Properties = null) + extends SparkListenerEvents + +case class SparkListenerJobSuccess(job: ActiveJob) extends SparkListenerEvents + +case class SparkListenerJobFailed(job: ActiveJob, failedStage: Stage) extends SparkListenerEvents + +case class SparkListenerJobCancelled(job: ActiveJob, reason: String) extends SparkListenerEvents + trait SparkListener { /** * called when a stage is completed, with information on the completed stage @@ -15,35 +33,25 @@ trait SparkListener { /** * called when a stage is submitted */ - def onStageSubmitted(stage: Stage, info: String = "") { } - + def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { } + /** * called when a task ends */ - def onTaskEnd(event: CompletionEvent) { } + def onTaskEnd(taskEnd: SparkListenerTaskEnd) { } /** * called when a job starts */ - def onJobStart(job: ActiveJob, properties: Properties = null) { } + def onJobStart(jobStart: SparkListenerJobStart) { } /** * called when a job ends */ - def onJobEnd(job: ActiveJob, event: SparkListenerEvents) { } + def onJobEnd(jobEnd: SparkListenerEvents) { } } -sealed trait SparkListenerEvents - -case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents - -case object SparkListenerJobSuccess extends SparkListenerEvents - -case class SparkListenerJobFailed(failedStage: Stage) extends SparkListenerEvents - -case class SparkListenerJobCancelled(reason: String) extends SparkListenerEvents - /** * Simple SparkListener that logs a few summary statistics when each stage completes */ diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala index 34fd8b995e..a654bf3ffd 100644 --- a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala @@ -40,7 +40,7 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers val shuffleMapStage = new Stage(1, parentRdd, Some(shuffleDep), Nil, jobID) val rootStage = new Stage(0, rootRdd, None, List(shuffleMapStage), jobID) - joblogger.onStageSubmitted(rootStage) + joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStage, 4)) joblogger.getEventQueue.size should be (1) joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getName) parentRdd.setName("MyRDD") @@ -86,11 +86,11 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers var onJobStartCount = 0 var onStageCompletedCount = 0 var onStageSubmittedCount = 0 - override def onTaskEnd(event: CompletionEvent) = onTaskEndCount += 1 - override def onJobEnd(job: ActiveJob, event: SparkListenerEvents) = onJobEndCount += 1 - override def onJobStart(job: ActiveJob, properties: Properties) = onJobStartCount += 1 + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = onTaskEndCount += 1 + override def onJobEnd(jobEnd: SparkListenerEvents) = onJobEndCount += 1 + override def onJobStart(jobStart: SparkListenerJobStart) = onJobStartCount += 1 override def onStageCompleted(stageCompleted: StageCompleted) = onStageCompletedCount += 1 - override def onStageSubmitted(stage: Stage, info: String = "") = onStageSubmittedCount += 1 + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = onStageSubmittedCount += 1 } sc.addSparkListener(joblogger) val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) } -- cgit v1.2.3 From b5b12823faf62766d880e497c90b44b21f5a433a Mon Sep 17 00:00:00 2001 From: Rohit Rai Date: Thu, 13 Jun 2013 14:05:46 +0530 Subject: Fixing the style as per feedback --- .../main/scala/spark/examples/CassandraTest.scala | 72 +++++++++++----------- 1 file changed, 37 insertions(+), 35 deletions(-) diff --git a/examples/src/main/scala/spark/examples/CassandraTest.scala b/examples/src/main/scala/spark/examples/CassandraTest.scala index 2cc62b9fe9..0fe1833e83 100644 --- a/examples/src/main/scala/spark/examples/CassandraTest.scala +++ b/examples/src/main/scala/spark/examples/CassandraTest.scala @@ -1,9 +1,11 @@ package spark.examples import org.apache.hadoop.mapreduce.Job -import org.apache.cassandra.hadoop.{ColumnFamilyOutputFormat, ConfigHelper, ColumnFamilyInputFormat} +import org.apache.cassandra.hadoop.ColumnFamilyOutputFormat +import org.apache.cassandra.hadoop.ConfigHelper +import org.apache.cassandra.hadoop.ColumnFamilyInputFormat import org.apache.cassandra.thrift._ -import spark.{RDD, SparkContext} +import spark.SparkContext import spark.SparkContext._ import java.nio.ByteBuffer import java.util.SortedMap @@ -12,9 +14,9 @@ import org.apache.cassandra.utils.ByteBufferUtil import scala.collection.JavaConversions._ - /* - * This example demonstrates using Spark with Cassandra with the New Hadoop API and Cassandra support for Hadoop. + * This example demonstrates using Spark with Cassandra with the New Hadoop API and Cassandra + * support for Hadoop. * * To run this example, run this file with the following command params - * @@ -26,32 +28,31 @@ import scala.collection.JavaConversions._ * 1. You have already created a keyspace called casDemo and it has a column family named Words * 2. There are column family has a column named "para" which has test content. * - * You can create the content by running the following script at the bottom of this file with cassandra-cli. + * You can create the content by running the following script at the bottom of this file with + * cassandra-cli. * */ object CassandraTest { + def main(args: Array[String]) { - //Get a SparkContext + // Get a SparkContext val sc = new SparkContext(args(0), "casDemo") - //Build the job configuration with ConfigHelper provided by Cassandra + // Build the job configuration with ConfigHelper provided by Cassandra val job = new Job() job.setInputFormatClass(classOf[ColumnFamilyInputFormat]) - ConfigHelper.setInputInitialAddress(job.getConfiguration(), args(1)) - - ConfigHelper.setInputRpcPort(job.getConfiguration(), args(2)) - - ConfigHelper.setOutputInitialAddress(job.getConfiguration(), args(1)) - - ConfigHelper.setOutputRpcPort(job.getConfiguration(), args(2)) + val host: String = args(1) + val port: String = args(2) + ConfigHelper.setInputInitialAddress(job.getConfiguration(), host) + ConfigHelper.setInputRpcPort(job.getConfiguration(), port) + ConfigHelper.setOutputInitialAddress(job.getConfiguration(), host) + ConfigHelper.setOutputRpcPort(job.getConfiguration(), port) ConfigHelper.setInputColumnFamily(job.getConfiguration(), "casDemo", "Words") - ConfigHelper.setOutputColumnFamily(job.getConfiguration(), "casDemo", "WordCount") - val predicate = new SlicePredicate() val sliceRange = new SliceRange() sliceRange.setStart(Array.empty[Byte]) @@ -60,11 +61,11 @@ object CassandraTest { ConfigHelper.setInputSlicePredicate(job.getConfiguration(), predicate) ConfigHelper.setInputPartitioner(job.getConfiguration(), "Murmur3Partitioner") - ConfigHelper.setOutputPartitioner(job.getConfiguration(), "Murmur3Partitioner") - //Make a new Hadoop RDD - val casRdd = sc.newAPIHadoopRDD(job.getConfiguration(), + // Make a new Hadoop RDD + val casRdd = sc.newAPIHadoopRDD( + job.getConfiguration(), classOf[ColumnFamilyInputFormat], classOf[ByteBuffer], classOf[SortedMap[ByteBuffer, IColumn]]) @@ -76,7 +77,7 @@ object CassandraTest { } } - //Lets get the word count in paras + // Lets get the word count in paras val counts = paraRdd.flatMap(p => p.split(" ")).map(word => (word, 1)).reduceByKey(_ + _) counts.collect().foreach { @@ -95,20 +96,17 @@ object CassandraTest { colCount.setValue(ByteBufferUtil.bytes(count.toLong)) colCount.setTimestamp(System.currentTimeMillis) - val outputkey = ByteBufferUtil.bytes(word + "-COUNT-" + System.currentTimeMillis) val mutations: java.util.List[Mutation] = new Mutation() :: new Mutation() :: Nil mutations.get(0).setColumn_or_supercolumn(new ColumnOrSuperColumn()) mutations.get(0).column_or_supercolumn.setColumn(colWord) - mutations.get(1).setColumn_or_supercolumn(new ColumnOrSuperColumn()) mutations.get(1).column_or_supercolumn.setColumn(colCount) (outputkey, mutations) } }.saveAsNewAPIHadoopFile("casDemo", classOf[ByteBuffer], classOf[List[Mutation]], classOf[ColumnFamilyOutputFormat], job.getConfiguration) - } } @@ -117,16 +115,20 @@ create keyspace casDemo; use casDemo; create column family WordCount with comparator = UTF8Type; -update column family WordCount with column_metadata = [{column_name: word, validation_class: UTF8Type}, {column_name: wcount, validation_class: LongType}]; +update column family WordCount with column_metadata = + [{column_name: word, validation_class: UTF8Type}, + {column_name: wcount, validation_class: LongType}]; create column family Words with comparator = UTF8Type; -update column family Words with column_metadata = [{column_name: book, validation_class: UTF8Type}, {column_name: para, validation_class: UTF8Type}]; +update column family Words with column_metadata = + [{column_name: book, validation_class: UTF8Type}, + {column_name: para, validation_class: UTF8Type}]; assume Words keys as utf8; set Words['3musk001']['book'] = 'The Three Musketeers'; -set Words['3musk001']['para'] = 'On the first Monday of the month of April, 1625, the market town of - Meung, in which the author of ROMANCE OF THE ROSE was born, appeared to +set Words['3musk001']['para'] = 'On the first Monday of the month of April, 1625, the market + town of Meung, in which the author of ROMANCE OF THE ROSE was born, appeared to be in as perfect a state of revolution as if the Huguenots had just made a second La Rochelle of it. Many citizens, seeing the women flying toward the High Street, leaving their children crying at the open doors, @@ -136,8 +138,8 @@ set Words['3musk001']['para'] = 'On the first Monday of the month of April, 1625 every minute, a compact group, vociferous and full of curiosity.'; set Words['3musk002']['book'] = 'The Three Musketeers'; -set Words['3musk002']['para'] = 'In those times panics were common, and few days passed without some city - or other registering in its archives an event of this kind. There were +set Words['3musk002']['para'] = 'In those times panics were common, and few days passed without + some city or other registering in its archives an event of this kind. There were nobles, who made war against each other; there was the king, who made war against the cardinal; there was Spain, which made war against the king. Then, in addition to these concealed or public, secret or open @@ -152,8 +154,8 @@ set Words['3musk002']['para'] = 'In those times panics were common, and few days cause of the hubbub was apparent to all'; set Words['3musk003']['book'] = 'The Three Musketeers'; -set Words['3musk003']['para'] = 'You ought, I say, then, to husband the means you have, however large - the sum may be; but you ought also to endeavor to perfect yourself in +set Words['3musk003']['para'] = 'You ought, I say, then, to husband the means you have, however + large the sum may be; but you ought also to endeavor to perfect yourself in the exercises becoming a gentleman. I will write a letter today to the Director of the Royal Academy, and tomorrow he will admit you without any expense to yourself. Do not refuse this little service. Our @@ -165,8 +167,8 @@ set Words['3musk003']['para'] = 'You ought, I say, then, to husband the means yo set Words['thelostworld001']['book'] = 'The Lost World'; -set Words['thelostworld001']['para'] = 'She sat with that proud, delicate profile of hers outlined against the - red curtain. How beautiful she was! And yet how aloof! We had been +set Words['thelostworld001']['para'] = 'She sat with that proud, delicate profile of hers outlined + against the red curtain. How beautiful she was! And yet how aloof! We had been friends, quite good friends; but never could I get beyond the same comradeship which I might have established with one of my fellow-reporters upon the Gazette,--perfectly frank, perfectly kindly, @@ -180,8 +182,8 @@ set Words['thelostworld001']['para'] = 'She sat with that proud, delicate profil as that--or had inherited it in that race memory which we call instinct.'; set Words['thelostworld002']['book'] = 'The Lost World'; -set Words['thelostworld002']['para'] = 'I always liked McArdle, the crabbed, old, round-backed, red-headed news - editor, and I rather hoped that he liked me. Of course, Beaumont was +set Words['thelostworld002']['para'] = 'I always liked McArdle, the crabbed, old, round-backed, + red-headed news editor, and I rather hoped that he liked me. Of course, Beaumont was the real boss; but he lived in the rarefied atmosphere of some Olympian height from which he could distinguish nothing smaller than an international crisis or a split in the Cabinet. Sometimes we saw him -- cgit v1.2.3 From 1d9f0df0652f455145d2dfed43a9407df6de6c43 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 13 Jun 2013 14:46:25 -0700 Subject: Fix some comments and style --- core/src/main/java/spark/network/netty/FileClient.java | 2 +- core/src/main/scala/spark/network/netty/ShuffleCopier.scala | 8 ++++---- core/src/main/scala/spark/storage/BlockFetcherIterator.scala | 6 +----- core/src/main/scala/spark/storage/DiskStore.scala | 3 +-- core/src/test/scala/spark/ShuffleSuite.scala | 3 +-- 5 files changed, 8 insertions(+), 14 deletions(-) diff --git a/core/src/main/java/spark/network/netty/FileClient.java b/core/src/main/java/spark/network/netty/FileClient.java index 517772202f..a4bb4bc701 100644 --- a/core/src/main/java/spark/network/netty/FileClient.java +++ b/core/src/main/java/spark/network/netty/FileClient.java @@ -30,7 +30,7 @@ class FileClient { .channel(OioSocketChannel.class) .option(ChannelOption.SO_KEEPALIVE, true) .option(ChannelOption.TCP_NODELAY, true) - .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, connectTimeout) // Disable connect timeout + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, connectTimeout) .handler(new FileClientChannelInitializer(handler)); } diff --git a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala index afb2cdbb3a..8d5194a737 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala @@ -18,8 +18,9 @@ private[spark] class ShuffleCopier extends Logging { resultCollectCallback: (String, Long, ByteBuf) => Unit) { val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) - val fc = new FileClient(handler, - System.getProperty("spark.shuffle.netty.connect.timeout", "60000").toInt) + val connectTimeout = System.getProperty("spark.shuffle.netty.connect.timeout", "60000").toInt + val fc = new FileClient(handler, connectTimeout) + try { fc.init() fc.connect(host, port) @@ -29,8 +30,7 @@ private[spark] class ShuffleCopier extends Logging { } catch { // Handle any socket-related exceptions in FileClient case e: Exception => { - logError("Shuffle copy of block " + blockId + " from " + host + ":" + port + - " failed", e) + logError("Shuffle copy of block " + blockId + " from " + host + ":" + port + " failed", e) handler.handleError(blockId) } } diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index bb78207c9f..bec876213e 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -322,11 +322,7 @@ object BlockFetcherIterator { override def next(): (String, Option[Iterator[Any]]) = { resultsGotten += 1 val result = results.take() - // if all the results has been retrieved, shutdown the copiers - // NO need to stop the copiers if we got all the blocks ? - // if (resultsGotten == _numBlocksToFetch && copiers != null) { - // stopCopiers() - // } + // If all the results has been retrieved, copiers will exit automatically (result.blockId, if (result.failed) None else Some(result.deserialize())) } } diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 0af6e4a359..15ab840155 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -212,10 +212,9 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) val file = getFile(blockId) if (!allowAppendExisting && file.exists()) { // NOTE(shivaram): Delete the file if it exists. This might happen if a ShuffleMap task - // was rescheduled on the same machine as the old task ? + // was rescheduled on the same machine as the old task. logWarning("File for block " + blockId + " already exists on disk: " + file + ". Deleting") file.delete() - // throw new Exception("File for block " + blockId + " already exists on disk: " + file) } file } diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 33b02fff80..1916885a73 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -376,8 +376,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { val a = sc.parallelize(1 to 4, NUM_BLOCKS) val b = a.map(x => (x, x*2)) - // NOTE: The default Java serializer doesn't create zero-sized blocks. - // So, use Kryo + // NOTE: The default Java serializer should create zero-sized blocks val c = new ShuffledRDD(b, new HashPartitioner(10)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId -- cgit v1.2.3 From 44b8dbaedeb88f12ea911968c524883805f7ad95 Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Thu, 13 Jun 2013 16:23:15 -0700 Subject: use Iterator.single(elem) instead of Iterator(elem) for improved performance based on scaladocs --- core/src/main/scala/spark/RDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index ec5e5e2433..bc9c17d507 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -734,7 +734,7 @@ abstract class RDD[T: ClassManifest]( val topK = mapPartitions { items => val queue = new BoundedPriorityQueue[T](num) queue ++= items - Iterator(queue) + Iterator.single(queue) }.reduce { (queue1, queue2) => queue1 ++= queue2 queue1 -- cgit v1.2.3 From 93b3f5e535c509a017a433b72249fc49c79d4a0f Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Thu, 13 Jun 2013 16:26:35 -0700 Subject: drop unneeded ClassManifest implicit --- core/src/main/scala/spark/util/BoundedPriorityQueue.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala index 53ee95a02e..ef01beaea5 100644 --- a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala @@ -8,7 +8,7 @@ import scala.collection.generic.Growable * add/offer methods such that only the top K elements are retained. The top * K elements are defined by an implicit Ordering[A]. */ -class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A], mf: ClassManifest[A]) +class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) extends JPriorityQueue[A](maxSize, ord) with Growable[A] { override def offer(a: A): Boolean = { -- cgit v1.2.3 From 6738178d0daf1bbe7441db7c0c773a29bb2ec388 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 13 Jun 2013 23:59:42 -0700 Subject: SPARK-772: groupByKey should disable map side combine. --- core/src/main/scala/spark/PairRDDFunctions.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 15593db0d9..fa4bbfc76f 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -19,7 +19,7 @@ import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.OutputFormat import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} -import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, Job => NewAPIHadoopJob, HadoopMapReduceUtil, TaskAttemptID, TaskAttemptContext} +import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, Job => NewAPIHadoopJob, HadoopMapReduceUtil} import spark.partial.BoundedDouble import spark.partial.PartialResult @@ -187,11 +187,13 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * partitioning of the resulting key-value pair RDD by passing a Partitioner. */ def groupByKey(partitioner: Partitioner): RDD[(K, Seq[V])] = { + // groupByKey shouldn't use map side combine because map side combine does not + // reduce the amount of data shuffled and requires all map side data be inserted + // into a hash table, leading to more objects in the old gen. def createCombiner(v: V) = ArrayBuffer(v) def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v - def mergeCombiners(b1: ArrayBuffer[V], b2: ArrayBuffer[V]) = b1 ++= b2 val bufs = combineByKey[ArrayBuffer[V]]( - createCombiner _, mergeValue _, mergeCombiners _, partitioner) + createCombiner _, mergeValue _, null, partitioner, mapSideCombine=false) bufs.asInstanceOf[RDD[(K, Seq[V])]] } -- cgit v1.2.3 From 2cc188fd546fa061812f9fd4f72cf936bd01a0e6 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 14 Jun 2013 00:10:54 -0700 Subject: SPARK-774: cogroup should also disable map side combine by default --- core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 7599ba1a02..8966f9f86e 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -6,7 +6,7 @@ import java.util.{HashMap => JHashMap} import scala.collection.JavaConversions import scala.collection.mutable.ArrayBuffer -import spark.{Aggregator, Logging, Partition, Partitioner, RDD, SparkEnv, TaskContext} +import spark.{Aggregator, Partition, Partitioner, RDD, SparkEnv, TaskContext} import spark.{Dependency, OneToOneDependency, ShuffleDependency} @@ -49,12 +49,16 @@ private[spark] class CoGroupAggregator * * @param rdds parent RDDs. * @param part partitioner used to partition the shuffle output. - * @param mapSideCombine flag indicating whether to merge values before shuffle step. + * @param mapSideCombine flag indicating whether to merge values before shuffle step. If the flag + * is on, Spark does an extra pass over the data on the map side to merge + * all values belonging to the same key together. This can reduce the amount + * of data shuffled if and only if the number of distinct keys is very small, + * and the ratio of key size to value size is also very small. */ class CoGroupedRDD[K]( @transient var rdds: Seq[RDD[(K, _)]], part: Partitioner, - val mapSideCombine: Boolean = true, + val mapSideCombine: Boolean = false, val serializerClass: String = null) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { -- cgit v1.2.3 From 53add598f2fe09759a0df1e08f87f70503f808c5 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Sat, 15 Jun 2013 01:34:17 +0800 Subject: Update LocalSchedulerSuite to avoid using sleep for task launch --- .../spark/scheduler/LocalSchedulerSuite.scala | 83 +++++++++++++++------- 1 file changed, 59 insertions(+), 24 deletions(-) diff --git a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala index 37d14ed113..8bd813fd14 100644 --- a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala @@ -9,9 +9,7 @@ import spark.scheduler.cluster._ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.{ConcurrentMap, HashMap} import java.util.concurrent.Semaphore -import java.util.concurrent.atomic.AtomicBoolean -import java.util.concurrent.atomic.AtomicInteger - +import java.util.concurrent.CountDownLatch import java.util.Properties class Lock() { @@ -35,9 +33,19 @@ class Lock() { object TaskThreadInfo { val threadToLock = HashMap[Int, Lock]() val threadToRunning = HashMap[Int, Boolean]() + val threadToStarted = HashMap[Int, CountDownLatch]() } - +/* + * 1. each thread contains one job. + * 2. each job contains one stage. + * 3. each stage only contains one task. + * 4. each task(launched) must be lanched orderly(using threadToStarted) to make sure + * it will get cpu core resource, and will wait to finished after user manually + * release "Lock" and then cluster will contain another free cpu cores. + * 5. each task(pending) must use "sleep" to make sure it has been added to taskSetManager queue, + * thus it will be scheduled later when cluster has free cpu cores. + */ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { def createThread(threadIndex: Int, poolName: String, sc: SparkContext, sem: Semaphore) { @@ -45,22 +53,23 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { TaskThreadInfo.threadToRunning(threadIndex) = false val nums = sc.parallelize(threadIndex to threadIndex, 1) TaskThreadInfo.threadToLock(threadIndex) = new Lock() + TaskThreadInfo.threadToStarted(threadIndex) = new CountDownLatch(1) new Thread { - if (poolName != null) { - sc.addLocalProperties("spark.scheduler.cluster.fair.pool",poolName) - } - override def run() { - val ans = nums.map(number => { - TaskThreadInfo.threadToRunning(number) = true - TaskThreadInfo.threadToLock(number).jobWait() - number - }).collect() - assert(ans.toList === List(threadIndex)) - sem.release() - TaskThreadInfo.threadToRunning(threadIndex) = false - } + if (poolName != null) { + sc.addLocalProperties("spark.scheduler.cluster.fair.pool",poolName) + } + override def run() { + val ans = nums.map(number => { + TaskThreadInfo.threadToRunning(number) = true + TaskThreadInfo.threadToStarted(number).countDown() + TaskThreadInfo.threadToLock(number).jobWait() + TaskThreadInfo.threadToRunning(number) = false + number + }).collect() + assert(ans.toList === List(threadIndex)) + sem.release() + } }.start() - Thread.sleep(2000) } test("Local FIFO scheduler end-to-end test") { @@ -69,11 +78,24 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { val sem = new Semaphore(0) createThread(1,null,sc,sem) + TaskThreadInfo.threadToStarted(1).await() createThread(2,null,sc,sem) + TaskThreadInfo.threadToStarted(2).await() createThread(3,null,sc,sem) + TaskThreadInfo.threadToStarted(3).await() createThread(4,null,sc,sem) + TaskThreadInfo.threadToStarted(4).await() + // thread 5 and 6 (stage pending)must meet following two points + // 1. stages (taskSetManager) of jobs in thread 5 and 6 should be add to taskSetManager + // queue before executing TaskThreadInfo.threadToLock(1).jobFinished() + // 2. priority of stage in thread 5 should be prior to priority of stage in thread 6 + // So I just use "sleep" 1s here for each thread. + // TODO: any better solution? createThread(5,null,sc,sem) + Thread.sleep(1000) createThread(6,null,sc,sem) + Thread.sleep(1000) + assert(TaskThreadInfo.threadToRunning(1) === true) assert(TaskThreadInfo.threadToRunning(2) === true) assert(TaskThreadInfo.threadToRunning(3) === true) @@ -82,8 +104,8 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { assert(TaskThreadInfo.threadToRunning(6) === false) TaskThreadInfo.threadToLock(1).jobFinished() - Thread.sleep(1000) - + TaskThreadInfo.threadToStarted(5).await() + assert(TaskThreadInfo.threadToRunning(1) === false) assert(TaskThreadInfo.threadToRunning(2) === true) assert(TaskThreadInfo.threadToRunning(3) === true) @@ -92,7 +114,7 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { assert(TaskThreadInfo.threadToRunning(6) === false) TaskThreadInfo.threadToLock(3).jobFinished() - Thread.sleep(1000) + TaskThreadInfo.threadToStarted(6).await() assert(TaskThreadInfo.threadToRunning(1) === false) assert(TaskThreadInfo.threadToRunning(2) === true) @@ -116,23 +138,31 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { System.setProperty("spark.fairscheduler.allocation.file", xmlPath) createThread(10,"1",sc,sem) + TaskThreadInfo.threadToStarted(10).await() createThread(20,"2",sc,sem) + TaskThreadInfo.threadToStarted(20).await() createThread(30,"3",sc,sem) + TaskThreadInfo.threadToStarted(30).await() assert(TaskThreadInfo.threadToRunning(10) === true) assert(TaskThreadInfo.threadToRunning(20) === true) assert(TaskThreadInfo.threadToRunning(30) === true) createThread(11,"1",sc,sem) + TaskThreadInfo.threadToStarted(11).await() createThread(21,"2",sc,sem) + TaskThreadInfo.threadToStarted(21).await() createThread(31,"3",sc,sem) + TaskThreadInfo.threadToStarted(31).await() assert(TaskThreadInfo.threadToRunning(11) === true) assert(TaskThreadInfo.threadToRunning(21) === true) assert(TaskThreadInfo.threadToRunning(31) === true) createThread(12,"1",sc,sem) + TaskThreadInfo.threadToStarted(12).await() createThread(22,"2",sc,sem) + TaskThreadInfo.threadToStarted(22).await() createThread(32,"3",sc,sem) assert(TaskThreadInfo.threadToRunning(12) === true) @@ -140,20 +170,25 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { assert(TaskThreadInfo.threadToRunning(32) === false) TaskThreadInfo.threadToLock(10).jobFinished() - Thread.sleep(1000) + TaskThreadInfo.threadToStarted(32).await() + assert(TaskThreadInfo.threadToRunning(32) === true) + //1. Similar with above scenario, sleep 1s for stage of 23 and 33 to be added to taskSetManager + // queue so that cluster will assign free cpu core to stage 23 after stage 11 finished. + //2. priority of 23 and 33 will be meaningless as using fair scheduler here. createThread(23,"2",sc,sem) createThread(33,"3",sc,sem) + Thread.sleep(1000) TaskThreadInfo.threadToLock(11).jobFinished() - Thread.sleep(1000) + TaskThreadInfo.threadToStarted(23).await() assert(TaskThreadInfo.threadToRunning(23) === true) assert(TaskThreadInfo.threadToRunning(33) === false) TaskThreadInfo.threadToLock(12).jobFinished() - Thread.sleep(1000) + TaskThreadInfo.threadToStarted(33).await() assert(TaskThreadInfo.threadToRunning(33) === true) -- cgit v1.2.3 From e8801d44900153eae6412963d2f3e2f19bfdc4e9 Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Fri, 14 Jun 2013 23:39:05 -0700 Subject: use delegation for BoundedPriorityQueue, add Java API --- core/src/main/scala/spark/RDD.scala | 9 ++---- core/src/main/scala/spark/api/java/JavaRDD.scala | 1 - .../main/scala/spark/api/java/JavaRDDLike.scala | 27 ++++++++++++++++- .../scala/spark/util/BoundedPriorityQueue.scala | 35 ++++++++++------------ 4 files changed, 44 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index bc9c17d507..4a4616c843 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -731,19 +731,14 @@ abstract class RDD[T: ClassManifest]( * @return an array of top elements */ def top(num: Int)(implicit ord: Ordering[T]): Array[T] = { - val topK = mapPartitions { items => + mapPartitions { items => val queue = new BoundedPriorityQueue[T](num) queue ++= items Iterator.single(queue) }.reduce { (queue1, queue2) => queue1 ++= queue2 queue1 - } - - val builder = Array.newBuilder[T] - builder.sizeHint(topK.size) - builder ++= topK - builder.result() + }.toArray } /** diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala index eb81ed64cd..626b499454 100644 --- a/core/src/main/scala/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaRDD.scala @@ -86,7 +86,6 @@ JavaRDDLike[T, JavaRDD[T]] { */ def subtract(other: JavaRDD[T], p: Partitioner): JavaRDD[T] = wrapRDD(rdd.subtract(other, p)) - } object JavaRDD { diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index 9b74d1226f..3e9c779d7b 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -1,6 +1,6 @@ package spark.api.java -import java.util.{List => JList} +import java.util.{List => JList, Comparator} import scala.Tuple2 import scala.collection.JavaConversions._ @@ -351,4 +351,29 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def toDebugString(): String = { rdd.toDebugString } + + /** + * Returns the top K elements from this RDD as defined by + * the specified Comparator[T]. + * @param num the number of top elements to return + * @param comp the comparator that defines the order + * @return an array of top elements + */ + def top(num: Int, comp: Comparator[T]): JList[T] = { + import scala.collection.JavaConversions._ + val topElems = rdd.top(num)(Ordering.comparatorToOrdering(comp)) + val arr: java.util.Collection[T] = topElems.toSeq + new java.util.ArrayList(arr) + } + + /** + * Returns the top K elements from this RDD using the + * natural ordering for T. + * @param num the number of top elements to return + * @return an array of top elements + */ + def top(num: Int): JList[T] = { + val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[T]] + top(num, comp) + } } diff --git a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala index ef01beaea5..4bc5db8bb7 100644 --- a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala @@ -1,30 +1,30 @@ package spark.util +import java.io.Serializable import java.util.{PriorityQueue => JPriorityQueue} import scala.collection.generic.Growable +import scala.collection.JavaConverters._ /** - * Bounded priority queue. This class modifies the original PriorityQueue's - * add/offer methods such that only the top K elements are retained. The top - * K elements are defined by an implicit Ordering[A]. + * Bounded priority queue. This class wraps the original PriorityQueue + * class and modifies it such that only the top K elements are retained. + * The top K elements are defined by an implicit Ordering[A]. */ class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) - extends JPriorityQueue[A](maxSize, ord) with Growable[A] { + extends Iterable[A] with Growable[A] with Serializable { - override def offer(a: A): Boolean = { - if (size < maxSize) super.offer(a) - else maybeReplaceLowest(a) - } + private val underlying = new JPriorityQueue[A](maxSize, ord) - override def add(a: A): Boolean = offer(a) + override def iterator: Iterator[A] = underlying.iterator.asScala override def ++=(xs: TraversableOnce[A]): this.type = { - xs.foreach(add) + xs.foreach { this += _ } this } override def +=(elem: A): this.type = { - add(elem) + if (size < maxSize) underlying.offer(elem) + else maybeReplaceLowest(elem) this } @@ -32,17 +32,14 @@ class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) this += elem1 += elem2 ++= elems } + override def clear() { underlying.clear() } + private def maybeReplaceLowest(a: A): Boolean = { - val head = peek() + val head = underlying.peek() if (head != null && ord.gt(a, head)) { - poll() - super.offer(a) + underlying.poll() + underlying.offer(a) } else false } } -object BoundedPriorityQueue { - import scala.collection.JavaConverters._ - implicit def asIterable[A](queue: BoundedPriorityQueue[A]): Iterable[A] = queue.asScala -} - -- cgit v1.2.3 From 479442a9b913b08a64da4bd5848111d950105336 Mon Sep 17 00:00:00 2001 From: Christopher Nguyen Date: Sat, 15 Jun 2013 17:35:55 -0700 Subject: Add zeroLengthPartitions() test to make sure, e.g., StatCounter.scala can handle empty partitions without incorrectly returning NaN --- core/src/test/scala/spark/JavaAPISuite.java | 22 ++++++++++++++++++++++ project/plugins.sbt | 2 ++ 2 files changed, 24 insertions(+) diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 93bb69b41c..3190a43e73 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -314,6 +314,28 @@ public class JavaAPISuite implements Serializable { List take = rdd.take(5); } + @Test + public void zeroLengthPartitions() { + // Create RDD with some consecutive empty partitions (including the "first" one) + JavaDoubleRDD rdd = sc + .parallelizeDoubles(Arrays.asList(-1.0, -1.0, -1.0, -1.0, 2.0, 4.0, -1.0, -1.0), 8) + .filter(new Function() { + @Override + public Boolean call(Double x) { + return x > 0.0; + } + }); + + // Run the partitions, including the consecutive empty ones, through StatCounter + StatCounter stats = rdd.stats(); + Assert.assertEquals(6.0, stats.sum(), 0.01); + Assert.assertEquals(6.0/2, rdd.mean(), 0.01); + Assert.assertEquals(1.0, rdd.variance(), 0.01); + Assert.assertEquals(1.0, rdd.stdev(), 0.01); + + // Add other tests here for classes that should be able to handle empty partitions correctly + } + @Test public void map() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); diff --git a/project/plugins.sbt b/project/plugins.sbt index d4f2442872..25b812a28d 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -16,3 +16,5 @@ addSbtPlugin("io.spray" %% "sbt-twirl" % "0.6.1") //resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) //addSbtPlugin("com.jsuereth" % "xsbt-gpg-plugin" % "0.6") + +libraryDependencies += "com.novocode" % "junit-interface" % "0.10-M4" % "test" -- cgit v1.2.3 From 5c886194e458c64fcf24066af351bde47dd8bf12 Mon Sep 17 00:00:00 2001 From: Christopher Nguyen Date: Sun, 16 Jun 2013 01:23:48 -0700 Subject: Move zero-length partition testing from JavaAPISuite.java to PartitioningSuite.scala --- core/src/test/scala/spark/JavaAPISuite.java | 22 ---------------------- core/src/test/scala/spark/PartitioningSuite.scala | 21 +++++++++++++++++++-- 2 files changed, 19 insertions(+), 24 deletions(-) diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 3190a43e73..93bb69b41c 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -314,28 +314,6 @@ public class JavaAPISuite implements Serializable { List take = rdd.take(5); } - @Test - public void zeroLengthPartitions() { - // Create RDD with some consecutive empty partitions (including the "first" one) - JavaDoubleRDD rdd = sc - .parallelizeDoubles(Arrays.asList(-1.0, -1.0, -1.0, -1.0, 2.0, 4.0, -1.0, -1.0), 8) - .filter(new Function() { - @Override - public Boolean call(Double x) { - return x > 0.0; - } - }); - - // Run the partitions, including the consecutive empty ones, through StatCounter - StatCounter stats = rdd.stats(); - Assert.assertEquals(6.0, stats.sum(), 0.01); - Assert.assertEquals(6.0/2, rdd.mean(), 0.01); - Assert.assertEquals(1.0, rdd.variance(), 0.01); - Assert.assertEquals(1.0, rdd.stdev(), 0.01); - - // Add other tests here for classes that should be able to handle empty partitions correctly - } - @Test public void map() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); diff --git a/core/src/test/scala/spark/PartitioningSuite.scala b/core/src/test/scala/spark/PartitioningSuite.scala index 60db759c25..e5745c81b3 100644 --- a/core/src/test/scala/spark/PartitioningSuite.scala +++ b/core/src/test/scala/spark/PartitioningSuite.scala @@ -1,10 +1,10 @@ package spark import org.scalatest.FunSuite - import scala.collection.mutable.ArrayBuffer - import SparkContext._ +import spark.util.StatCounter +import scala.math._ class PartitioningSuite extends FunSuite with LocalSparkContext { @@ -120,4 +120,21 @@ class PartitioningSuite extends FunSuite with LocalSparkContext { assert(intercept[SparkException]{ arrPairs.reduceByKeyLocally(_ + _) }.getMessage.contains("array")) assert(intercept[SparkException]{ arrPairs.reduceByKey(_ + _) }.getMessage.contains("array")) } + + test("Zero-length partitions should be correctly handled") { + // Create RDD with some consecutive empty partitions (including the "first" one) + sc = new SparkContext("local", "test") + val rdd: RDD[Double] = sc + .parallelize(Array(-1.0, -1.0, -1.0, -1.0, 2.0, 4.0, -1.0, -1.0), 8) + .filter(_ >= 0.0) + + // Run the partitions, including the consecutive empty ones, through StatCounter + val stats: StatCounter = rdd.stats(); + assert(abs(6.0 - stats.sum) < 0.01); + assert(abs(6.0/2 - rdd.mean) < 0.01); + assert(abs(1.0 - rdd.variance) < 0.01); + assert(abs(1.0 - rdd.stdev) < 0.01); + + // Add other tests here for classes that should be able to handle empty partitions correctly + } } -- cgit v1.2.3 From f91195cc150a3ead122046d14bd35b4fcf28c9cb Mon Sep 17 00:00:00 2001 From: Christopher Nguyen Date: Sun, 16 Jun 2013 01:29:53 -0700 Subject: Import just scala.math.abs rather than scala.math._ --- core/src/test/scala/spark/PartitioningSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/PartitioningSuite.scala b/core/src/test/scala/spark/PartitioningSuite.scala index e5745c81b3..16f93e71a3 100644 --- a/core/src/test/scala/spark/PartitioningSuite.scala +++ b/core/src/test/scala/spark/PartitioningSuite.scala @@ -4,7 +4,7 @@ import org.scalatest.FunSuite import scala.collection.mutable.ArrayBuffer import SparkContext._ import spark.util.StatCounter -import scala.math._ +import scala.math.abs class PartitioningSuite extends FunSuite with LocalSparkContext { -- cgit v1.2.3 From fb6d733fa88aa124deecf155af40cc095ecca5b3 Mon Sep 17 00:00:00 2001 From: Gavin Li Date: Sun, 16 Jun 2013 22:32:55 +0000 Subject: update according to comments --- core/src/main/scala/spark/RDD.scala | 71 ++------------------------- core/src/main/scala/spark/rdd/PipedRDD.scala | 29 +++++------ core/src/test/scala/spark/PipedRDDSuite.scala | 13 +++-- 3 files changed, 24 insertions(+), 89 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index a1c9604324..152f7be9bb 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -355,68 +355,6 @@ abstract class RDD[T: ClassManifest]( def pipe(command: String, env: Map[String, String]): RDD[String] = new PipedRDD(this, command, env) - /** - * Return an RDD created by piping elements to a forked external process. - * How each record in RDD is outputed to the process can be controled by providing a - * function trasnform(T, outputFunction: String => Unit). transform() will be called with - * the currnet record in RDD as the 1st parameter, and the function to output the record to - * the external process (like out.println()) as the 2nd parameter. - * Here's an example on how to pipe the RDD data of groupBy() in a streaming way, - * instead of constructing a huge String to concat all the records: - * def tranform(record:(String, Seq[String]), f:String=>Unit) = for (e <- record._2){f(e)} - * pipeContext can be used to transfer additional context data to the external process - * besides the RDD. pipeContext is a broadcast Seq[String], each line would be piped to - * external process with "^A" as the delimiter in the end of context data. Delimiter can also - * be customized by the last parameter delimiter. - */ - def pipe[U<: Seq[String]]( - command: String, - env: Map[String, String], - transform: (T,String => Unit) => Any, - pipeContext: Broadcast[U], - delimiter: String): RDD[String] = - new PipedRDD(this, command, env, transform, pipeContext, delimiter) - - /** - * Return an RDD created by piping elements to a forked external process. - * How each record in RDD is outputed to the process can be controled by providing a - * function trasnform(T, outputFunction: String => Unit). transform() will be called with - * the currnet record in RDD as the 1st parameter, and the function to output the record to - * the external process (like out.println()) as the 2nd parameter. - * Here's an example on how to pipe the RDD data of groupBy() in a streaming way, - * instead of constructing a huge String to concat all the records: - * def tranform(record:(String, Seq[String]), f:String=>Unit) = for (e <- record._2){f(e)} - * pipeContext can be used to transfer additional context data to the external process - * besides the RDD. pipeContext is a broadcast Seq[String], each line would be piped to - * external process with "^A" as the delimiter in the end of context data. Delimiter can also - * be customized by the last parameter delimiter. - */ - def pipe[U<: Seq[String]]( - command: String, - transform: (T,String => Unit) => Any, - pipeContext: Broadcast[U]): RDD[String] = - new PipedRDD(this, command, Map[String, String](), transform, pipeContext, "\u0001") - - /** - * Return an RDD created by piping elements to a forked external process. - * How each record in RDD is outputed to the process can be controled by providing a - * function trasnform(T, outputFunction: String => Unit). transform() will be called with - * the currnet record in RDD as the 1st parameter, and the function to output the record to - * the external process (like out.println()) as the 2nd parameter. - * Here's an example on how to pipe the RDD data of groupBy() in a streaming way, - * instead of constructing a huge String to concat all the records: - * def tranform(record:(String, Seq[String]), f:String=>Unit) = for (e <- record._2){f(e)} - * pipeContext can be used to transfer additional context data to the external process - * besides the RDD. pipeContext is a broadcast Seq[String], each line would be piped to - * external process with "^A" as the delimiter in the end of context data. Delimiter can also - * be customized by the last parameter delimiter. - */ - def pipe[U<: Seq[String]]( - command: String, - env: Map[String, String], - transform: (T,String => Unit) => Any, - pipeContext: Broadcast[U]): RDD[String] = - new PipedRDD(this, command, env, transform, pipeContext, "\u0001") /** * Return an RDD created by piping elements to a forked external process. @@ -432,13 +370,12 @@ abstract class RDD[T: ClassManifest]( * external process with "^A" as the delimiter in the end of context data. Delimiter can also * be customized by the last parameter delimiter. */ - def pipe[U<: Seq[String]]( + def pipe( command: Seq[String], env: Map[String, String] = Map(), - transform: (T,String => Unit) => Any = null, - pipeContext: Broadcast[U] = null, - delimiter: String = "\u0001"): RDD[String] = - new PipedRDD(this, command, env, transform, pipeContext, delimiter) + printPipeContext: (String => Unit) => Unit = null, + printRDDElement: (T, String => Unit) => Unit = null): RDD[String] = + new PipedRDD(this, command, env, if (printPipeContext ne null) sc.clean(printPipeContext) else null, printRDDElement) /** * Return a new RDD by applying a function to each partition of this RDD. diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala index d58aaae709..b2c07891ab 100644 --- a/core/src/main/scala/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/spark/rdd/PipedRDD.scala @@ -16,14 +16,12 @@ import spark.broadcast.Broadcast * An RDD that pipes the contents of each parent partition through an external command * (printing them one per line) and returns the output as a collection of strings. */ -class PipedRDD[T: ClassManifest, U <: Seq[String]]( +class PipedRDD[T: ClassManifest]( prev: RDD[T], command: Seq[String], envVars: Map[String, String], - transform: (T, String => Unit) => Any, - pipeContext: Broadcast[U], - delimiter: String - ) + printPipeContext: (String => Unit) => Unit, + printRDDElement: (T, String => Unit) => Unit) extends RDD[String](prev) { // Similar to Runtime.exec(), if we are given a single string, split it into words @@ -32,10 +30,9 @@ class PipedRDD[T: ClassManifest, U <: Seq[String]]( prev: RDD[T], command: String, envVars: Map[String, String] = Map(), - transform: (T, String => Unit) => Any = null, - pipeContext: Broadcast[U] = null, - delimiter: String = "\u0001") = - this(prev, PipedRDD.tokenize(command), envVars, transform, pipeContext, delimiter) + printPipeContext: (String => Unit) => Unit = null, + printRDDElement: (T, String => Unit) => Unit = null) = + this(prev, PipedRDD.tokenize(command), envVars, printPipeContext, printRDDElement) override def getPartitions: Array[Partition] = firstParent[T].partitions @@ -64,17 +61,13 @@ class PipedRDD[T: ClassManifest, U <: Seq[String]]( SparkEnv.set(env) val out = new PrintWriter(proc.getOutputStream) - // input the pipeContext firstly - if ( pipeContext != null) { - for (elem <- pipeContext.value) { - out.println(elem) - } - // delimiter\n as the marker of the end of the pipeContext - out.println(delimiter) + // input the pipe context firstly + if ( printPipeContext != null) { + printPipeContext(out.println(_)) } for (elem <- firstParent[T].iterator(split, context)) { - if (transform != null) { - transform(elem, out.println(_)) + if (printRDDElement != null) { + printRDDElement(elem, out.println(_)) } else { out.println(elem) } diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/spark/PipedRDDSuite.scala index d2852867de..ed075f93ec 100644 --- a/core/src/test/scala/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/spark/PipedRDDSuite.scala @@ -22,9 +22,12 @@ class PipedRDDSuite extends FunSuite with LocalSparkContext { test("advanced pipe") { sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + val bl = sc.broadcast(List("0")) - val piped = nums.pipe(Seq("cat"), Map[String, String](), - (i:Int, f: String=> Unit) => f(i + "_"), sc.broadcast(List("0"))) + val piped = nums.pipe(Seq("cat"), + Map[String, String](), + (f: String => Unit) => {bl.value.map(f(_));f("\u0001")}, + (i:Int, f: String=> Unit) => f(i + "_")) val c = piped.collect() @@ -40,8 +43,10 @@ class PipedRDDSuite extends FunSuite with LocalSparkContext { val nums1 = sc.makeRDD(Array("a\t1", "b\t2", "a\t3", "b\t4"), 2) val d = nums1.groupBy(str=>str.split("\t")(0)). - pipe(Seq("cat"), Map[String, String](), (i:Tuple2[String, Seq[String]], f: String=> Unit) => - {for (e <- i._2){ f(e + "_")}}, sc.broadcast(List("0"))).collect() + pipe(Seq("cat"), + Map[String, String](), + (f: String => Unit) => {bl.value.map(f(_));f("\u0001")}, + (i:Tuple2[String, Seq[String]], f: String=> Unit) => {for (e <- i._2){ f(e + "_")}}).collect() assert(d.size === 8) assert(d(0) === "0") assert(d(1) === "\u0001") -- cgit v1.2.3 From 4508089fc342802a2f37fea6893cd47abd81fdd7 Mon Sep 17 00:00:00 2001 From: Gavin Li Date: Mon, 17 Jun 2013 05:23:46 +0000 Subject: refine comments and add sc.clean --- core/src/main/scala/spark/RDD.scala | 30 ++++++++++++++++++------------ 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 05ff399a7b..223dcdc19d 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -361,24 +361,30 @@ abstract class RDD[T: ClassManifest]( /** * Return an RDD created by piping elements to a forked external process. - * How each record in RDD is outputed to the process can be controled by providing a - * function trasnform(T, outputFunction: String => Unit). transform() will be called with - * the currnet record in RDD as the 1st parameter, and the function to output the record to - * the external process (like out.println()) as the 2nd parameter. - * Here's an example on how to pipe the RDD data of groupBy() in a streaming way, - * instead of constructing a huge String to concat all the records: - * def tranform(record:(String, Seq[String]), f:String=>Unit) = for (e <- record._2){f(e)} - * pipeContext can be used to transfer additional context data to the external process - * besides the RDD. pipeContext is a broadcast Seq[String], each line would be piped to - * external process with "^A" as the delimiter in the end of context data. Delimiter can also - * be customized by the last parameter delimiter. + * The print behavior can be customized by providing two functions. + * + * @param command command to run in forked process. + * @param env environment variables to set. + * @param printPipeContext Before piping elements, this function is called as an oppotunity + * to pipe context data. Print line function (like out.println) will be + * passed as printPipeContext's parameter. + * @param printPipeContext Use this function to customize how to pipe elements. This function + * will be called with each RDD element as the 1st parameter, and the + * print line function (like out.println()) as the 2nd parameter. + * An example of pipe the RDD data of groupBy() in a streaming way, + * instead of constructing a huge String to concat all the elements: + * def printRDDElement(record:(String, Seq[String]), f:String=>Unit) = + * for (e <- record._2){f(e)} + * @return the result RDD */ def pipe( command: Seq[String], env: Map[String, String] = Map(), printPipeContext: (String => Unit) => Unit = null, printRDDElement: (T, String => Unit) => Unit = null): RDD[String] = - new PipedRDD(this, command, env, if (printPipeContext ne null) sc.clean(printPipeContext) else null, printRDDElement) + new PipedRDD(this, command, env, + if (printPipeContext ne null) sc.clean(printPipeContext) else null, + if (printRDDElement ne null) sc.clean(printRDDElement) else null) /** * Return a new RDD by applying a function to each partition of this RDD. -- cgit v1.2.3 From 1450296797e53f1a01166c885050091df9c96e2e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 17 Jun 2013 16:58:23 -0400 Subject: SPARK-781: Log the temp directory path when Spark says "Failed to create temp directory". --- core/src/main/scala/spark/Utils.scala | 4 +-- core/src/main/scala/spark/storage/DiskStore.scala | 34 +++++++++++------------ 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index ec15326014..fd7b8cc8d5 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -116,8 +116,8 @@ private object Utils extends Logging { while (dir == null) { attempts += 1 if (attempts > maxAttempts) { - throw new IOException("Failed to create a temp directory after " + maxAttempts + - " attempts!") + throw new IOException("Failed to create a temp directory under (" + root + ") after " + + maxAttempts + " attempts!") } try { dir = new File(root, "spark-" + UUID.randomUUID.toString) diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index c7281200e7..9914beec99 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -82,15 +82,15 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) override def size(): Long = lastValidPosition } - val MAX_DIR_CREATION_ATTEMPTS: Int = 10 - val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt + private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 + private val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt - var shuffleSender : ShuffleSender = null + private var shuffleSender : ShuffleSender = null // Create one local directory for each path mentioned in spark.local.dir; then, inside this // directory, create multiple subdirectories that we will hash files into, in order to avoid // having really large inodes at the top level. - val localDirs = createLocalDirs() - val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) + private val localDirs: Array[File] = createLocalDirs() + private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) addShutdownHook() @@ -99,7 +99,6 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) new DiskBlockObjectWriter(blockId, serializer, bufferSize) } - override def getSize(blockId: String): Long = { getFile(blockId).length() } @@ -232,8 +231,8 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) private def createLocalDirs(): Array[File] = { logDebug("Creating local directories at root dirs '" + rootDirs + "'") val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss") - rootDirs.split(",").map(rootDir => { - var foundLocalDir: Boolean = false + rootDirs.split(",").map { rootDir => + var foundLocalDir = false var localDir: File = null var localDirId: String = null var tries = 0 @@ -248,7 +247,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) } } catch { case e: Exception => - logWarning("Attempt " + tries + " to create local dir failed", e) + logWarning("Attempt " + tries + " to create local dir " + localDir + " failed", e) } } if (!foundLocalDir) { @@ -258,7 +257,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) } logInfo("Created local directory at " + localDir) localDir - }) + } } private def addShutdownHook() { @@ -266,15 +265,16 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { override def run() { logDebug("Shutdown hook called") - try { - localDirs.foreach { localDir => + localDirs.foreach { localDir => + try { if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) + } catch { + case t: Throwable => + logError("Exception while deleting local spark dir: " + localDir, t) } - if (shuffleSender != null) { - shuffleSender.stop - } - } catch { - case t: Throwable => logError("Exception while deleting local spark dirs", t) + } + if (shuffleSender != null) { + shuffleSender.stop } } }) -- cgit v1.2.3 From be3c406edf06d5ab9da98097c28ce3eebc958b8e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 17 Jun 2013 17:07:51 -0400 Subject: Fixed the typo pointed out by Matei. --- core/src/main/scala/spark/Utils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index fd7b8cc8d5..645c18541e 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -116,7 +116,7 @@ private object Utils extends Logging { while (dir == null) { attempts += 1 if (attempts > maxAttempts) { - throw new IOException("Failed to create a temp directory under (" + root + ") after " + + throw new IOException("Failed to create a temp directory (under " + root + ") after " + maxAttempts + " attempts!") } try { -- cgit v1.2.3 From 2ab311f4cee3f918dc28daaebd287b11c9f63429 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 18 Jun 2013 00:40:25 +0200 Subject: Removed second version of junit test plugin from plugins.sbt --- project/plugins.sbt | 2 -- 1 file changed, 2 deletions(-) diff --git a/project/plugins.sbt b/project/plugins.sbt index 25b812a28d..d4f2442872 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -16,5 +16,3 @@ addSbtPlugin("io.spray" %% "sbt-twirl" % "0.6.1") //resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) //addSbtPlugin("com.jsuereth" % "xsbt-gpg-plugin" % "0.6") - -libraryDependencies += "com.novocode" % "junit-interface" % "0.10-M4" % "test" -- cgit v1.2.3 From 1e9269c3eeeaa3a481b95521c703032ed84abd68 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 13 Jun 2013 10:46:22 +0800 Subject: reduce ZippedPartitionsRDD's getPreferredLocations complexity --- core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala index dd9f3c2680..b234428ab2 100644 --- a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala @@ -53,14 +53,10 @@ abstract class ZippedPartitionsBaseRDD[V: ClassManifest]( val exactMatchLocations = exactMatchPreferredLocations.reduce((x, y) => x.intersect(y)) // Remove exact match and then do host local match. - val otherNodePreferredLocations = rddSplitZip.map(x => { - x._1.preferredLocations(x._2).map(hostPort => { - val host = Utils.parseHostPort(hostPort)._1 - - if (exactMatchLocations.contains(host)) null else host - }).filter(_ != null) - }) - val otherNodeLocalLocations = otherNodePreferredLocations.reduce((x, y) => x.intersect(y)) + val exactMatchHosts = exactMatchLocations.map(Utils.parseHostPort(_)._1) + val matchPreferredHosts = exactMatchPreferredLocations.map(locs => locs.map(Utils.parseHostPort(_)._1)) + .reduce((x, y) => x.intersect(y)) + val otherNodeLocalLocations = matchPreferredHosts.filter { s => !exactMatchHosts.contains(s) } otherNodeLocalLocations ++ exactMatchLocations } -- cgit v1.2.3 From 0a2a9bce1e83e891334985c29176c6426b8b1751 Mon Sep 17 00:00:00 2001 From: Gavin Li Date: Tue, 18 Jun 2013 21:30:13 +0000 Subject: fix typo and coding style --- core/src/main/scala/spark/RDD.scala | 14 +++++++------- core/src/main/scala/spark/rdd/PipedRDD.scala | 2 +- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 223dcdc19d..709271d4eb 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -368,13 +368,13 @@ abstract class RDD[T: ClassManifest]( * @param printPipeContext Before piping elements, this function is called as an oppotunity * to pipe context data. Print line function (like out.println) will be * passed as printPipeContext's parameter. - * @param printPipeContext Use this function to customize how to pipe elements. This function - * will be called with each RDD element as the 1st parameter, and the - * print line function (like out.println()) as the 2nd parameter. - * An example of pipe the RDD data of groupBy() in a streaming way, - * instead of constructing a huge String to concat all the elements: - * def printRDDElement(record:(String, Seq[String]), f:String=>Unit) = - * for (e <- record._2){f(e)} + * @param printRDDElement Use this function to customize how to pipe elements. This function + * will be called with each RDD element as the 1st parameter, and the + * print line function (like out.println()) as the 2nd parameter. + * An example of pipe the RDD data of groupBy() in a streaming way, + * instead of constructing a huge String to concat all the elements: + * def printRDDElement(record:(String, Seq[String]), f:String=>Unit) = + * for (e <- record._2){f(e)} * @return the result RDD */ def pipe( diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala index b2c07891ab..c0baf43d43 100644 --- a/core/src/main/scala/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/spark/rdd/PipedRDD.scala @@ -62,7 +62,7 @@ class PipedRDD[T: ClassManifest]( val out = new PrintWriter(proc.getOutputStream) // input the pipe context firstly - if ( printPipeContext != null) { + if (printPipeContext != null) { printPipeContext(out.println(_)) } for (elem <- firstParent[T].iterator(split, context)) { -- cgit v1.2.3 From 7902baddc797f86f5bdbcc966f5cd60545638bf7 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 19 Jun 2013 13:34:30 +0200 Subject: Update ASM to version 4.0 --- core/pom.xml | 4 ++-- core/src/main/scala/spark/ClosureCleaner.scala | 11 +++++------ pom.xml | 6 +++--- project/SparkBuild.scala | 2 +- repl/src/main/scala/spark/repl/ExecutorClassLoader.scala | 3 +-- 5 files changed, 12 insertions(+), 14 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index d8687bf991..88f0ed70f3 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -32,8 +32,8 @@ compress-lzf - asm - asm-all + org.ow2.asm + asm com.google.protobuf diff --git a/core/src/main/scala/spark/ClosureCleaner.scala b/core/src/main/scala/spark/ClosureCleaner.scala index 50d6a1c5c9..d5e7132ff9 100644 --- a/core/src/main/scala/spark/ClosureCleaner.scala +++ b/core/src/main/scala/spark/ClosureCleaner.scala @@ -5,8 +5,7 @@ import java.lang.reflect.Field import scala.collection.mutable.Map import scala.collection.mutable.Set -import org.objectweb.asm.{ClassReader, MethodVisitor, Type} -import org.objectweb.asm.commons.EmptyVisitor +import org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} import org.objectweb.asm.Opcodes._ import java.io.{InputStream, IOException, ByteArrayOutputStream, ByteArrayInputStream, BufferedInputStream} @@ -162,10 +161,10 @@ private[spark] object ClosureCleaner extends Logging { } } -private[spark] class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends EmptyVisitor { +private[spark] class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor(ASM4) { override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { - return new EmptyVisitor { + return new MethodVisitor(ASM4) { override def visitFieldInsn(op: Int, owner: String, name: String, desc: String) { if (op == GETFIELD) { for (cl <- output.keys if cl.getName == owner.replace('/', '.')) { @@ -188,7 +187,7 @@ private[spark] class FieldAccessFinder(output: Map[Class[_], Set[String]]) exten } } -private[spark] class InnerClosureFinder(output: Set[Class[_]]) extends EmptyVisitor { +private[spark] class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM4) { var myName: String = null override def visit(version: Int, access: Int, name: String, sig: String, @@ -198,7 +197,7 @@ private[spark] class InnerClosureFinder(output: Set[Class[_]]) extends EmptyVisi override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { - return new EmptyVisitor { + return new MethodVisitor(ASM4) { override def visitMethodInsn(op: Int, owner: String, name: String, desc: String) { val argTypes = Type.getArgumentTypes(desc) diff --git a/pom.xml b/pom.xml index c893ec755e..3bcb2a3f34 100644 --- a/pom.xml +++ b/pom.xml @@ -190,9 +190,9 @@ 0.8.4 - asm - asm-all - 3.3.1 + org.ow2.asm + asm + 4.0 com.google.protobuf diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 824af821f9..b1f3f9a2ea 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -148,7 +148,7 @@ object SparkBuild extends Build { "org.slf4j" % "slf4j-log4j12" % slf4jVersion, "commons-daemon" % "commons-daemon" % "1.0.10", "com.ning" % "compress-lzf" % "0.8.4", - "asm" % "asm-all" % "3.3.1", + "org.ow2.asm" % "asm" % "4.0", "com.google.protobuf" % "protobuf-java" % "2.4.1", "de.javakaffee" % "kryo-serializers" % "0.22", "com.typesafe.akka" % "akka-actor" % "2.0.3" excludeAll(excludeNetty), diff --git a/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala index 13d81ec1cf..0e9aa863b5 100644 --- a/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala @@ -8,7 +8,6 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.objectweb.asm._ -import org.objectweb.asm.commons.EmptyVisitor import org.objectweb.asm.Opcodes._ @@ -83,7 +82,7 @@ extends ClassLoader(parent) { } class ConstructorCleaner(className: String, cv: ClassVisitor) -extends ClassAdapter(cv) { +extends ClassVisitor(ASM4, cv) { override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { val mv = cv.visitMethod(access, name, desc, sig, exceptions) -- cgit v1.2.3 From 75d78c7ac90ce717bf1009ec4d335fb4a6cfde24 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 19 Jun 2013 11:18:42 -0500 Subject: Add support for Spark on Yarn on a secure Hadoop cluster --- .../scala/spark/deploy/SparkHadoopUtil.scala | 13 +----- .../spark/deploy/yarn/ApplicationMaster.scala | 17 -------- .../scala/spark/deploy/yarn/Client.scala | 50 ++++++++++++++++++---- .../scala/spark/deploy/yarn/WorkerRunnable.scala | 33 ++++++++++++-- core/src/main/scala/spark/PairRDDFunctions.scala | 5 +++ core/src/main/scala/spark/SparkContext.scala | 9 ++++ .../scala/spark/scheduler/InputFormatInfo.scala | 9 ++++ .../spark/streaming/PairDStreamFunctions.scala | 7 ++- 8 files changed, 102 insertions(+), 41 deletions(-) diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala index ab1ab9d8a7..b96c047e10 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala @@ -27,18 +27,7 @@ object SparkHadoopUtil { } def runAsUser(func: (Product) => Unit, args: Product, user: String) { - - // println("running as user " + jobUserName) - - UserGroupInformation.setConfiguration(yarnConf) - val appMasterUgi: UserGroupInformation = UserGroupInformation.createRemoteUser(user) - appMasterUgi.doAs(new PrivilegedExceptionAction[AnyRef] { - def run: AnyRef = { - func(args) - // no return value ... - null - } - }) + func(args) } // Note that all params which start with SPARK are propagated all the way through, so if in yarn mode, this MUST be set to true. diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index aa72c1e5fe..f19648ec68 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -30,23 +30,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e def run() { - // Initialization - val jobUserName = Utils.getUserNameFromEnvironment() - logInfo("running as user " + jobUserName) - - // run as user ... - UserGroupInformation.setConfiguration(yarnConf) - val appMasterUgi: UserGroupInformation = UserGroupInformation.createRemoteUser(jobUserName) - appMasterUgi.doAs(new PrivilegedExceptionAction[AnyRef] { - def run: AnyRef = { - runImpl() - return null - } - }) - } - - private def runImpl() { - appAttemptId = getApplicationAttemptId() resourceManager = registerWithResourceManager() val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala index 7a881e26df..f20cc31c7c 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala @@ -1,9 +1,13 @@ package spark.deploy.yarn import java.net.{InetSocketAddress, URI} +import java.nio.ByteBuffer import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.mapred.Master import org.apache.hadoop.net.NetUtils +import org.apache.hadoop.io.DataOutputBuffer +import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.api.protocolrecords._ @@ -23,6 +27,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl var rpc: YarnRPC = YarnRPC.create(conf) val yarnConf: YarnConfiguration = new YarnConfiguration(conf) + val credentials = UserGroupInformation.getCurrentUser().getCredentials(); def run() { init(yarnConf) @@ -41,7 +46,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl appContext.setQueue(args.amQueue) appContext.setAMContainerSpec(amContainer) appContext.setUser(args.amUser) - + submitApp(appContext) monitorApplication(appId) @@ -62,14 +67,21 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl def verifyClusterResources(app: GetNewApplicationResponse) = { val maxMem = app.getMaximumResourceCapability().getMemory() - logInfo("Max mem capabililty of resources in this cluster " + maxMem) + logInfo("Max mem capabililty of a single resource in this cluster " + maxMem) - // If the cluster does not have enough memory resources, exit. - val requestedMem = (args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + args.numWorkers * args.workerMemory - if (requestedMem > maxMem) { - logError("Cluster cannot satisfy memory resource request of " + requestedMem) + // if we have requested more then the clusters max for a single resource then exit. + if (args.workerMemory > maxMem) { + logError("the worker size is to large to run on this cluster " + args.workerMemory); + System.exit(1) + } + val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD + if (amMem > maxMem) { + logError("AM size is to large to run on this cluster " + amMem) System.exit(1) } + + // We could add checks to make sure the entire cluster has enough resources but that involves getting + // all the node reports and computing ourselves } def createApplicationSubmissionContext(appId: ApplicationId): ApplicationSubmissionContext = { @@ -86,6 +98,15 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl // Upload Spark and the application JAR to the remote file system // Add them as local resources to the AM val fs = FileSystem.get(conf) + + val delegTokenRenewer = Master.getMasterPrincipal(conf); + if (UserGroupInformation.isSecurityEnabled()) { + if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { + logError("Can't get Master Kerberos principal for use as renewer") + System.exit(1) + } + } + Map("spark.jar" -> System.getenv("SPARK_JAR"), "app.jar" -> args.userJar, "log4j.properties" -> System.getenv("SPARK_LOG4J_CONF")) .foreach { case(destName, _localPath) => val localPath: String = if (_localPath != null) _localPath.trim() else "" @@ -97,6 +118,11 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl fs.copyFromLocalFile(false, true, src, dst) val destStatus = fs.getFileStatus(dst) + // get tokens for anything we upload to hdfs + if (UserGroupInformation.isSecurityEnabled()) { + fs.addDelegationTokens(delegTokenRenewer, credentials); + } + val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource] amJarRsrc.setType(LocalResourceType.FILE) amJarRsrc.setVisibility(LocalResourceVisibility.APPLICATION) @@ -106,6 +132,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl locaResources(destName) = amJarRsrc } } + UserGroupInformation.getCurrentUser().addCredentials(credentials); return locaResources } @@ -115,6 +142,8 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val env = new HashMap[String, String]() Apps.addToEnvironment(env, Environment.USER.name, args.amUser) + // set this so that UGI set to correct user in unsecure mode + Apps.addToEnvironment(env, "HADOOP_USER_NAME", args.amUser) // If log4j present, ensure ours overrides all others if (log4jConfLocalRes != null) Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./") @@ -195,7 +224,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } // Command for the ApplicationMaster - val commands = List[String]("java " + + val commands = List[String](Environment.JAVA_HOME.$() + "/bin/java " + " -server " + JAVA_OPTS + " spark.deploy.yarn.ApplicationMaster" + @@ -214,7 +243,12 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl // Memory for the ApplicationMaster capability.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD) amContainer.setResource(capability) - + + // Setup security tokens + val dob = new DataOutputBuffer() + credentials.writeTokenStorageToStream(dob) + amContainer.setContainerTokens(ByteBuffer.wrap(dob.getData())) + return amContainer } diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala index a2bf0af762..e22d256a84 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala @@ -1,9 +1,12 @@ package spark.deploy.yarn import java.net.URI +import java.nio.ByteBuffer +import java.security.PrivilegedExceptionAction import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} +import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.net.NetUtils import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api._ @@ -11,7 +14,7 @@ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC -import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} +import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils} import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import scala.collection.JavaConversions._ @@ -76,7 +79,13 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S */ ctx.setUser(UserGroupInformation.getCurrentUser().getShortUserName()) - val commands = List[String]("java " + + + val credentials = UserGroupInformation.getCurrentUser().getCredentials() + val dob = new DataOutputBuffer() + credentials.writeTokenStorageToStream(dob) + ctx.setContainerTokens(ByteBuffer.wrap(dob.getData())) + + val commands = List[String](Environment.JAVA_HOME.$() + "/bin/java " + " -server " + // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in an inconsistent state. @@ -145,6 +154,8 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S val env = new HashMap[String, String]() // should we add this ? Apps.addToEnvironment(env, Environment.USER.name, Utils.getUserNameFromEnvironment()) + // set this so that UGI set to correct user in unsecure mode + Apps.addToEnvironment(env, "HADOOP_USER_NAME", Utils.getUserNameFromEnvironment()) // If log4j present, ensure ours overrides all others if (System.getenv("SPARK_YARN_LOG4J_PATH") != null) { @@ -165,7 +176,23 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S val cmHostPortStr = container.getNodeId().getHost() + ":" + container.getNodeId().getPort() val cmAddress = NetUtils.createSocketAddr(cmHostPortStr) logInfo("Connecting to ContainerManager at " + cmHostPortStr) - return rpc.getProxy(classOf[ContainerManager], cmAddress, conf).asInstanceOf[ContainerManager] + + // use doAs and remoteUser here so we can add the container token and not + // pollute the current users credentials with all of the individual container tokens + val user = UserGroupInformation.createRemoteUser(container.getId().toString()); + val containerToken = container.getContainerToken(); + if (containerToken != null) { + user.addToken(ProtoUtils.convertFromProtoFormat(containerToken, cmAddress)) + } + + val proxy = user + .doAs(new PrivilegedExceptionAction[ContainerManager] { + def run: ContainerManager = { + return rpc.getProxy(classOf[ContainerManager], + cmAddress, conf).asInstanceOf[ContainerManager] + } + }); + return proxy; } } diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 2b0e697337..cc13f5d9c2 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -18,6 +18,7 @@ import org.apache.hadoop.mapred.OutputFormat import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, Job => NewAPIHadoopJob, HadoopMapReduceUtil, TaskAttemptID, TaskAttemptContext} +import org.apache.hadoop.security.UserGroupInformation import spark.partial.BoundedDouble import spark.partial.PartialResult @@ -584,6 +585,10 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( valueClass: Class[_], outputFormatClass: Class[_ <: OutputFormat[_, _]], conf: JobConf = new JobConf(self.context.hadoopConfiguration)) { + // make sure to propogate any credentials from the current user to the jobConf + // for Hadoop security + val jobCreds = conf.getCredentials(); + jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) conf.setOutputKeyClass(keyClass) conf.setOutputValueClass(valueClass) // conf.setOutputFormat(outputFormatClass) // Doesn't work in Scala 2.9 due to what may be a generics bug diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index bc05d08fd6..c3a56938b5 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -36,6 +36,7 @@ import org.apache.hadoop.mapred.TextInputFormat import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.{Job => NewHadoopJob} import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} +import org.apache.hadoop.security.UserGroupInformation import org.apache.mesos.MesosNativeLibrary @@ -294,6 +295,10 @@ class SparkContext( valueClass: Class[V], minSplits: Int = defaultMinSplits ): RDD[(K, V)] = { + // make sure to propogate any credentials from the current user to the jobConf + // for Hadoop security + val jobCreds = conf.getCredentials(); + jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minSplits) } @@ -306,6 +311,10 @@ class SparkContext( minSplits: Int = defaultMinSplits ) : RDD[(K, V)] = { val conf = new JobConf(hadoopConfiguration) + // make sure to propogate any credentials from the current user to the jobConf + // for Hadoop security + val jobCreds = conf.getCredentials(); + jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) FileInputFormat.setInputPaths(conf, path) new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minSplits) } diff --git a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala index 287f731787..30a56d7135 100644 --- a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala @@ -3,6 +3,7 @@ package spark.scheduler import spark.Logging import scala.collection.immutable.Set import org.apache.hadoop.mapred.{FileInputFormat, JobConf} +import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.util.ReflectionUtils import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.conf.Configuration @@ -70,6 +71,10 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl // This method does not expect failures, since validate has already passed ... private def prefLocsFromMapreduceInputFormat(): Set[SplitInfo] = { val conf = new JobConf(configuration) + // make sure to propogate any credentials from the current user to the jobConf + // for Hadoop security + val jobCreds = conf.getCredentials(); + jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) FileInputFormat.setInputPaths(conf, path) val instance: org.apache.hadoop.mapreduce.InputFormat[_, _] = @@ -89,6 +94,10 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl // This method does not expect failures, since validate has already passed ... private def prefLocsFromMapredInputFormat(): Set[SplitInfo] = { val jobConf = new JobConf(configuration) + // make sure to propogate any credentials from the current user to the jobConf + // for Hadoop security + val jobCreds = jobConf.getCredentials(); + jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) FileInputFormat.setInputPaths(jobConf, path) val instance: org.apache.hadoop.mapred.InputFormat[_, _] = diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala index 3ec922957d..20ee1d3c5d 100644 --- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala @@ -14,6 +14,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.mapred.{JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.hadoop.mapred.OutputFormat +import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.conf.Configuration class PairDStreamFunctions[K: ClassManifest, V: ClassManifest](self: DStream[(K,V)]) @@ -470,7 +471,11 @@ extends Serializable { valueClass: Class[_], outputFormatClass: Class[_ <: OutputFormat[_, _]], conf: JobConf = new JobConf - ) { + ) { + // make sure to propogate any credentials from the current user to the jobConf + // for Hadoop security + val jobCreds = conf.getCredentials(); + jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) val saveFunc = (rdd: RDD[(K, V)], time: Time) => { val file = rddToFileName(prefix, suffix, time) rdd.saveAsHadoopFile(file, keyClass, valueClass, outputFormatClass, conf) -- cgit v1.2.3 From ae7a5da6b31f5bf64f713b3d9bff6e441d8615b4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 20 Jun 2013 18:44:46 +0200 Subject: Fix some dependency issues in SBT build (same will be needed for Maven): - Exclude a version of ASM 3.x that comes from HBase - Don't use a special ASF repo for HBase - Update SLF4J version - Add sbt-dependency-graph plugin so we can easily find dependency trees --- project/SparkBuild.scala | 10 +++++----- project/plugins.sbt | 2 ++ 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b1f3f9a2ea..24c8b734d0 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -125,12 +125,13 @@ object SparkBuild extends Build { publishMavenStyle in MavenCompile := true, publishLocal in MavenCompile <<= publishTask(publishLocalConfiguration in MavenCompile, deliverLocal), publishLocalBoth <<= Seq(publishLocal in MavenCompile, publishLocal).dependOn - ) + ) ++ net.virtualvoid.sbt.graph.Plugin.graphSettings - val slf4jVersion = "1.6.1" + val slf4jVersion = "1.7.2" val excludeJackson = ExclusionRule(organization = "org.codehaus.jackson") val excludeNetty = ExclusionRule(organization = "org.jboss.netty") + val excludeAsm = ExclusionRule(organization = "asm") def coreSettings = sharedSettings ++ Seq( name := "spark-core", @@ -201,11 +202,10 @@ object SparkBuild extends Build { def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", - resolvers ++= Seq("Apache HBase" at "https://repository.apache.org/content/repositories/releases"), libraryDependencies ++= Seq( "com.twitter" % "algebird-core_2.9.2" % "0.1.11", - "org.apache.hbase" % "hbase" % "0.94.6" excludeAll(excludeNetty), + "org.apache.hbase" % "hbase" % "0.94.6" excludeAll(excludeNetty, excludeAsm), "org.apache.cassandra" % "cassandra-all" % "1.2.5" exclude("com.google.guava", "guava") @@ -224,7 +224,7 @@ object SparkBuild extends Build { name := "spark-streaming", libraryDependencies ++= Seq( "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty), - "com.github.sgroschupf" % "zkclient" % "0.1", + "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty), "com.typesafe.akka" % "akka-zeromq" % "2.0.3" excludeAll(excludeNetty) ) diff --git a/project/plugins.sbt b/project/plugins.sbt index d4f2442872..f806e66481 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -16,3 +16,5 @@ addSbtPlugin("io.spray" %% "sbt-twirl" % "0.6.1") //resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) //addSbtPlugin("com.jsuereth" % "xsbt-gpg-plugin" % "0.6") + +addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.7.3") -- cgit v1.2.3 From 071ff7efa17a30db57a54f517a3cfecf3463db9f Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Thu, 20 Jun 2013 17:53:23 -0700 Subject: Enable building a fat jar for the Spark REPL --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 0ea23b446f..e3892371e0 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -197,7 +197,7 @@ object SparkBuild extends Build { def replSettings = sharedSettings ++ Seq( name := "spark-repl", libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _) - ) + ) ++ assemblySettings ++ extraAssemblySettings ++ Twirl.settings def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", -- cgit v1.2.3 From 52407951541399e60a5292394b3a443a5e7ff281 Mon Sep 17 00:00:00 2001 From: Mingfei Date: Fri, 21 Jun 2013 17:38:23 +0800 Subject: edit according to comments --- core/src/main/scala/spark/RDD.scala | 6 +- core/src/main/scala/spark/Utils.scala | 10 +-- .../main/scala/spark/executor/TaskMetrics.scala | 2 +- .../main/scala/spark/scheduler/DAGScheduler.scala | 14 +++-- .../src/main/scala/spark/scheduler/JobLogger.scala | 72 ++++++++++------------ .../main/scala/spark/scheduler/SparkListener.scala | 25 ++++---- .../scala/spark/scheduler/JobLoggerSuite.scala | 2 +- 7 files changed, 62 insertions(+), 69 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 8c0b7ca417..b17398953b 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -114,10 +114,10 @@ abstract class RDD[T: ClassManifest]( this } - /**User-defined generator of this RDD*/ - var generator = Utils.getCallSiteInfo._4 + /** User-defined generator of this RDD*/ + var generator = Utils.getCallSiteInfo.firstUserClass - /**reset generator*/ + /** Reset generator*/ def setGenerator(_generator: String) = { generator = _generator } diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 1630b2b4b0..1cfaee79b1 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -522,13 +522,14 @@ private object Utils extends Logging { execute(command, new File(".")) } - + class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String, + val firstUserLine: Int, val firstUserClass: String) /** * When called inside a class in the spark package, returns the name of the user code class * (outside the spark package) that called into Spark, as well as which Spark method they called. * This is used, for example, to tell users where in their code each RDD got created. */ - def getCallSiteInfo = { + def getCallSiteInfo: CallSiteInfo = { val trace = Thread.currentThread.getStackTrace().filter( el => (!el.getMethodName.contains("getStackTrace"))) @@ -560,12 +561,13 @@ private object Utils extends Logging { } } } - (lastSparkMethod, firstUserFile, firstUserLine, firstUserClass) + new CallSiteInfo(lastSparkMethod, firstUserFile, firstUserLine, firstUserClass) } def formatSparkCallSite = { val callSiteInfo = getCallSiteInfo - "%s at %s:%s".format(callSiteInfo._1, callSiteInfo._2, callSiteInfo._3) + "%s at %s:%s".format(callSiteInfo.lastSparkMethod, callSiteInfo.firstUserFile, + callSiteInfo.firstUserLine) } /** * Try to find a free port to bind to on the local host. This should ideally never be needed, diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index 26e8029365..1dc13754f9 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -2,7 +2,7 @@ package spark.executor class TaskMetrics extends Serializable { /** - * host's name the task runs on + * Host's name the task runs on */ var hostname: String = _ diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index e281e5a8db..4336f2f36d 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -312,7 +312,8 @@ class DAGScheduler( handleExecutorLost(execId) case completion: CompletionEvent => - sparkListeners.foreach(_.onTaskEnd(SparkListenerTaskEnd(completion))) + sparkListeners.foreach(_.onTaskEnd(SparkListenerTaskEnd(completion.task, + completion.reason, completion.taskInfo, completion.taskMetrics))) handleTaskCompletion(completion) case TaskSetFailed(taskSet, reason) => @@ -323,8 +324,8 @@ class DAGScheduler( for (job <- activeJobs) { val error = new SparkException("Job cancelled because SparkContext was shut down") job.listener.jobFailed(error) - sparkListeners.foreach(_.onJobEnd(SparkListenerJobCancelled(job, - "SPARKCONTEXT_SHUTDOWN"))) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, + JobFailed(error)))) } return true } @@ -527,7 +528,7 @@ class DAGScheduler( activeJobs -= job resultStageToJob -= stage markStageAsFinished(stage) - sparkListeners.foreach(_.onJobEnd(SparkListenerJobSuccess(job))) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobSucceeded))) } job.listener.taskSucceeded(rt.outputId, event.result) } @@ -668,10 +669,11 @@ class DAGScheduler( val dependentStages = resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage)).toSeq for (resultStage <- dependentStages) { val job = resultStageToJob(resultStage) - job.listener.jobFailed(new SparkException("Job failed: " + reason)) + val error = new SparkException("Job failed: " + reason) + job.listener.jobFailed(error) activeJobs -= job resultStageToJob -= resultStage - sparkListeners.foreach(_.onJobEnd(SparkListenerJobFailed(job, failedStage))) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error)))) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index 002c5826cb..760a0252b7 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -12,7 +12,7 @@ import spark._ import spark.executor.TaskMetrics import spark.scheduler.cluster.TaskInfo -// used to record runtime information for each job, including RDD graph +// Used to record runtime information for each job, including RDD graph // tasks' start/stop shuffle information and information from outside class JobLogger(val logDirName: String) extends SparkListener with Logging { @@ -49,21 +49,17 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { processStageSubmittedEvent(stage, taskSize) case StageCompleted(stageInfo) => processStageCompletedEvent(stageInfo) - case SparkListenerJobSuccess(job) => - processJobEndEvent(job) - case SparkListenerJobFailed(job, failedStage) => - processJobEndEvent(job, failedStage) - case SparkListenerJobCancelled(job, reason) => - processJobEndEvent(job, reason) - case SparkListenerTaskEnd(event) => - processTaskEndEvent(event) + case SparkListenerJobEnd(job, result) => + processJobEndEvent(job, result) + case SparkListenerTaskEnd(task, reason, taskInfo, taskMetrics) => + processTaskEndEvent(task, reason, taskInfo, taskMetrics) case _ => } } } }.start() - //create a folder for log files, the folder's name is the creation time of the jobLogger + // Create a folder for log files, the folder's name is the creation time of the jobLogger protected def createLogDir() { val dir = new File(logDir + "/" + logDirName + "/") if (dir.exists()) { @@ -244,54 +240,50 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { eventQueue.put(taskEnd) } - protected def processTaskEndEvent(event: CompletionEvent) { + protected def processTaskEndEvent(task: Task[_], reason: TaskEndReason, + taskInfo: TaskInfo, taskMetrics: TaskMetrics) { var taskStatus = "" - event.task match { + task match { case resultTask: ResultTask[_, _] => taskStatus = "TASK_TYPE=RESULT_TASK" case shuffleMapTask: ShuffleMapTask => taskStatus = "TASK_TYPE=SHUFFLE_MAP_TASK" } - event.reason match { + reason match { case Success => taskStatus += " STATUS=SUCCESS" - recordTaskMetrics(event.task.stageId, taskStatus, event.taskInfo, event.taskMetrics) + recordTaskMetrics(task.stageId, taskStatus, taskInfo, taskMetrics) case Resubmitted => - taskStatus += " STATUS=RESUBMITTED TID=" + event.taskInfo.taskId + - " STAGE_ID=" + event.task.stageId - stageLogInfo(event.task.stageId, taskStatus) + taskStatus += " STATUS=RESUBMITTED TID=" + taskInfo.taskId + + " STAGE_ID=" + task.stageId + stageLogInfo(task.stageId, taskStatus) case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => - taskStatus += " STATUS=FETCHFAILED TID=" + event.taskInfo.taskId + " STAGE_ID=" + - event.task.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" + + taskStatus += " STATUS=FETCHFAILED TID=" + taskInfo.taskId + " STAGE_ID=" + + task.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" + mapId + " REDUCE_ID=" + reduceId - stageLogInfo(event.task.stageId, taskStatus) + stageLogInfo(task.stageId, taskStatus) case OtherFailure(message) => - taskStatus += " STATUS=FAILURE TID=" + event.taskInfo.taskId + - " STAGE_ID=" + event.task.stageId + " INFO=" + message - stageLogInfo(event.task.stageId, taskStatus) + taskStatus += " STATUS=FAILURE TID=" + taskInfo.taskId + + " STAGE_ID=" + task.stageId + " INFO=" + message + stageLogInfo(task.stageId, taskStatus) case _ => } } - override def onJobEnd(jobEnd: SparkListenerEvents) { + override def onJobEnd(jobEnd: SparkListenerJobEnd) { eventQueue.put(jobEnd) } - protected def processJobEndEvent(job: ActiveJob) { - val info = "JOB_ID=" + job.runId + " STATUS=SUCCESS" - jobLogInfo(job.runId, info) - closeLogWriter(job.runId) - } - - protected def processJobEndEvent(job: ActiveJob, failedStage: Stage) { - val info = "JOB_ID=" + job.runId + " STATUS=FAILED REASON=STAGE_FAILED FAILED_STAGE_ID=" - + failedStage.id - jobLogInfo(job.runId, info) - closeLogWriter(job.runId) - } - protected def processJobEndEvent(job: ActiveJob, reason: String) { - var info = "JOB_ID=" + job.runId + " STATUS=CANCELLED REASON=" + reason - jobLogInfo(job.runId, info) + protected def processJobEndEvent(job: ActiveJob, reason: JobResult) { + var info = "JOB_ID=" + job.runId + reason match { + case JobSucceeded => info += " STATUS=SUCCESS" + case JobFailed(exception) => + info += " STATUS=FAILED REASON=" + exception.getMessage.split("\\s+").foreach(info += _ + "_") + case _ => + } + jobLogInfo(job.runId, info.substring(0, info.length - 1).toUpperCase) closeLogWriter(job.runId) } - + protected def recordJobProperties(jobID: Int, properties: Properties) { if(properties != null) { val annotation = properties.getProperty("spark.job.annotation", "") diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 9265261dc1..bac984b5c9 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -3,52 +3,49 @@ package spark.scheduler import java.util.Properties import spark.scheduler.cluster.TaskInfo import spark.util.Distribution -import spark.{Utils, Logging, SparkContext, TaskEndReason} +import spark.{Logging, SparkContext, TaskEndReason, Utils} import spark.executor.TaskMetrics - sealed trait SparkListenerEvents case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int) extends SparkListenerEvents case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents -case class SparkListenerTaskEnd(event: CompletionEvent) extends SparkListenerEvents +case class SparkListenerTaskEnd(task: Task[_], reason: TaskEndReason, taskInfo: TaskInfo, + taskMetrics: TaskMetrics) extends SparkListenerEvents case class SparkListenerJobStart(job: ActiveJob, properties: Properties = null) extends SparkListenerEvents - -case class SparkListenerJobSuccess(job: ActiveJob) extends SparkListenerEvents - -case class SparkListenerJobFailed(job: ActiveJob, failedStage: Stage) extends SparkListenerEvents -case class SparkListenerJobCancelled(job: ActiveJob, reason: String) extends SparkListenerEvents +case class SparkListenerJobEnd(job: ActiveJob, jobResult: JobResult) + extends SparkListenerEvents trait SparkListener { /** - * called when a stage is completed, with information on the completed stage + * Called when a stage is completed, with information on the completed stage */ def onStageCompleted(stageCompleted: StageCompleted) { } /** - * called when a stage is submitted + * Called when a stage is submitted */ def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { } /** - * called when a task ends + * Called when a task ends */ def onTaskEnd(taskEnd: SparkListenerTaskEnd) { } /** - * called when a job starts + * Called when a job starts */ def onJobStart(jobStart: SparkListenerJobStart) { } /** - * called when a job ends + * Called when a job ends */ - def onJobEnd(jobEnd: SparkListenerEvents) { } + def onJobEnd(jobEnd: SparkListenerJobEnd) { } } diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala index a654bf3ffd..4000c4d520 100644 --- a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala @@ -87,7 +87,7 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers var onStageCompletedCount = 0 var onStageSubmittedCount = 0 override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = onTaskEndCount += 1 - override def onJobEnd(jobEnd: SparkListenerEvents) = onJobEndCount += 1 + override def onJobEnd(jobEnd: SparkListenerJobEnd) = onJobEndCount += 1 override def onJobStart(jobStart: SparkListenerJobStart) = onJobStartCount += 1 override def onStageCompleted(stageCompleted: StageCompleted) = onStageCompletedCount += 1 override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = onStageSubmittedCount += 1 -- cgit v1.2.3 From aa7aa587beff22e2db50d2afadd95097856a299a Mon Sep 17 00:00:00 2001 From: Mingfei Date: Fri, 21 Jun 2013 17:48:41 +0800 Subject: some format modification --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 5 ++--- core/src/main/scala/spark/scheduler/JobLogger.scala | 10 +++++----- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 4336f2f36d..e412baa803 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -324,8 +324,7 @@ class DAGScheduler( for (job <- activeJobs) { val error = new SparkException("Job cancelled because SparkContext was shut down") job.listener.jobFailed(error) - sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, - JobFailed(error)))) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error)))) } return true } @@ -671,9 +670,9 @@ class DAGScheduler( val job = resultStageToJob(resultStage) val error = new SparkException("Job failed: " + reason) job.listener.jobFailed(error) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error)))) activeJobs -= job resultStageToJob -= resultStage - sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error)))) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index 760a0252b7..178bfaba3d 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -70,7 +70,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { } } - // create a log file for one job, the file name is the jobID + // Create a log file for one job, the file name is the jobID protected def createLogWriter(jobID: Int) { try{ val fileWriter = new PrintWriter(logDir + "/" + logDirName + "/" + jobID) @@ -80,7 +80,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { } } - // close log file for one job, and clean the stage relationship in stageIDToJobID + // Close log file, and clean the stage relationship in stageIDToJobID protected def closeLogWriter(jobID: Int) = jobIDToPrintWriter.get(jobID).foreach { fileWriter => fileWriter.close() @@ -91,7 +91,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { jobIDToStages -= jobID } - // write log information to log file, withTime parameter controls whether to recored + // Write log information to log file, withTime parameter controls whether to recored // time stamp for the information protected def jobLogInfo(jobID: Int, info: String, withTime: Boolean = true) { var writeInfo = info @@ -145,7 +145,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { } } - // generate indents and convert to String + // Generate indents and convert to String protected def indentString(indent: Int) = { val sb = new StringBuilder() for (i <- 1 to indent) { @@ -190,7 +190,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { jobLogInfo(jobID, indentString(indent) + stageInfo + " JOB_ID=" + stage.priority, false) } - // record task metrics into job log files + // Record task metrics into job log files protected def recordTaskMetrics(stageID: Int, status: String, taskInfo: TaskInfo, taskMetrics: TaskMetrics) { val info = " TID=" + taskInfo.taskId + " STAGE_ID=" + stageID + -- cgit v1.2.3 From 4b9862ac9cf2d00c5245e9a8b0fcb05b82030c98 Mon Sep 17 00:00:00 2001 From: Mingfei Date: Fri, 21 Jun 2013 17:55:32 +0800 Subject: small format modification --- core/src/main/scala/spark/Utils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 1cfaee79b1..96d86647f8 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -522,8 +522,8 @@ private object Utils extends Logging { execute(command, new File(".")) } - class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String, - val firstUserLine: Int, val firstUserClass: String) + private[spark] class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String, + val firstUserLine: Int, val firstUserClass: String) /** * When called inside a class in the spark package, returns the name of the user code class * (outside the spark package) that called into Spark, as well as which Spark method they called. -- cgit v1.2.3 From 2fc794a6c7f1b86e5c0103a9c82af2be7fafb347 Mon Sep 17 00:00:00 2001 From: Mingfei Date: Fri, 21 Jun 2013 18:21:35 +0800 Subject: small modify in DAGScheduler --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index e412baa803..f7d60be5db 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -289,7 +289,6 @@ class DAGScheduler( val finalStage = newStage(finalRDD, None, runId) val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener, properties) clearCacheLocs() - sparkListeners.foreach(_.onJobStart(SparkListenerJobStart(job, properties))) logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length + " output partitions (allowLocal=" + allowLocal + ")") logInfo("Final stage: " + finalStage + " (" + finalStage.origin + ")") @@ -299,6 +298,7 @@ class DAGScheduler( // Compute very short actions like first() or take() with no parent stages locally. runLocally(job) } else { + sparkListeners.foreach(_.onJobStart(SparkListenerJobStart(job, properties))) idToActiveJob(runId) = job activeJobs += job resultStageToJob(finalStage) = job -- cgit v1.2.3 From 93a1643405d7c1a1fffe8210130341f34d64ea72 Mon Sep 17 00:00:00 2001 From: James Phillpotts Date: Fri, 21 Jun 2013 14:21:52 +0100 Subject: Allow other twitter authorizations than username/password --- .../src/main/scala/spark/streaming/StreamingContext.scala | 15 ++++++++++++++- .../spark/streaming/dstream/TwitterInputDStream.scala | 14 ++++++-------- 2 files changed, 20 insertions(+), 9 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index b8b60aab43..f97e47ada0 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.apache.hadoop.fs.Path import twitter4j.Status +import twitter4j.auth.{Authorization, BasicAuthorization} /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic @@ -372,8 +373,20 @@ class StreamingContext private ( password: String, filters: Seq[String] = Nil, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): DStream[Status] = twitterStream(new BasicAuthorization(username, password), filters, storageLevel) + + /** + * Create a input stream that returns tweets received from Twitter. + * @param twitterAuth Twitter4J authentication + * @param filters Set of filter strings to get only those tweets that match them + * @param storageLevel Storage level to use for storing the received objects + */ + def twitterStream( + twitterAuth: Authorization, + filters: Seq[String] = Nil, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 ): DStream[Status] = { - val inputStream = new TwitterInputDStream(this, username, password, filters, storageLevel) + val inputStream = new TwitterInputDStream(this, twitterAuth, filters, storageLevel) registerInputStream(inputStream) inputStream } diff --git a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala index c697498862..0b01091a52 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala @@ -6,6 +6,7 @@ import storage.StorageLevel import twitter4j._ import twitter4j.auth.BasicAuthorization +import twitter4j.auth.Authorization /* A stream of Twitter statuses, potentially filtered by one or more keywords. * @@ -16,21 +17,19 @@ import twitter4j.auth.BasicAuthorization private[streaming] class TwitterInputDStream( @transient ssc_ : StreamingContext, - username: String, - password: String, + twitterAuth: Authorization, filters: Seq[String], storageLevel: StorageLevel ) extends NetworkInputDStream[Status](ssc_) { - + override def getReceiver(): NetworkReceiver[Status] = { - new TwitterReceiver(username, password, filters, storageLevel) + new TwitterReceiver(twitterAuth, filters, storageLevel) } } private[streaming] class TwitterReceiver( - username: String, - password: String, + twitterAuth: Authorization, filters: Seq[String], storageLevel: StorageLevel ) extends NetworkReceiver[Status] { @@ -40,8 +39,7 @@ class TwitterReceiver( protected override def onStart() { blockGenerator.start() - twitterStream = new TwitterStreamFactory() - .getInstance(new BasicAuthorization(username, password)) + twitterStream = new TwitterStreamFactory().getInstance(twitterAuth) twitterStream.addListener(new StatusListener { def onStatus(status: Status) = { blockGenerator += status -- cgit v1.2.3 From 40afe0d2a5562738ef2ff37ed1d448ae2d0cc927 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Sun, 10 Mar 2013 13:54:46 -0700 Subject: Add Python timing instrumentation --- core/src/main/scala/spark/api/python/PythonRDD.scala | 12 ++++++++++++ python/pyspark/serializers.py | 4 ++++ python/pyspark/worker.py | 16 +++++++++++++++- 3 files changed, 31 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 807119ca8c..e9978d713f 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -47,6 +47,7 @@ private[spark] class PythonRDD[T: ClassManifest]( currentEnvVars.put(variable, value) } + val startTime = System.currentTimeMillis val proc = pb.start() val env = SparkEnv.get @@ -108,6 +109,17 @@ private[spark] class PythonRDD[T: ClassManifest]( val obj = new Array[Byte](length) stream.readFully(obj) obj + case -3 => + // Timing data from child + val bootTime = stream.readLong() + val initTime = stream.readLong() + val finishTime = stream.readLong() + val boot = bootTime - startTime + val init = initTime - bootTime + val finish = finishTime - initTime + val total = finishTime - startTime + logInfo("Times: total = %s, boot = %s, init = %s, finish = %s".format(total, boot, init, finish)) + read case -2 => // Signals that an exception has been thrown in python val exLength = stream.readInt() diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 115cf28cc2..5a95144983 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -46,6 +46,10 @@ def read_long(stream): return struct.unpack("!q", length)[0] +def write_long(value, stream): + stream.write(struct.pack("!q", value)) + + def read_int(stream): length = stream.read(4) if length == "": diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 812e7a9da5..4c33ae49dc 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -1,6 +1,8 @@ """ Worker that receives input from Piped RDD. """ +import time +preboot_time = time.time() import os import sys import traceback @@ -12,7 +14,7 @@ from pyspark.broadcast import Broadcast, _broadcastRegistry from pyspark.cloudpickle import CloudPickler from pyspark.files import SparkFiles from pyspark.serializers import write_with_length, read_with_length, write_int, \ - read_long, read_int, dump_pickle, load_pickle, read_from_pickle_file + read_long, write_long, read_int, dump_pickle, load_pickle, read_from_pickle_file # Redirect stdout to stderr so that users must return values from functions. @@ -24,7 +26,16 @@ def load_obj(): return load_pickle(standard_b64decode(sys.stdin.readline().strip())) +def report_times(preboot, boot, init, finish): + write_int(-3, old_stdout) + write_long(1000 * preboot, old_stdout) + write_long(1000 * boot, old_stdout) + write_long(1000 * init, old_stdout) + write_long(1000 * finish, old_stdout) + + def main(): + boot_time = time.time() split_index = read_int(sys.stdin) spark_files_dir = load_pickle(read_with_length(sys.stdin)) SparkFiles._root_directory = spark_files_dir @@ -41,6 +52,7 @@ def main(): dumps = lambda x: x else: dumps = dump_pickle + init_time = time.time() iterator = read_from_pickle_file(sys.stdin) try: for obj in func(split_index, iterator): @@ -49,6 +61,8 @@ def main(): write_int(-2, old_stdout) write_with_length(traceback.format_exc(), old_stdout) sys.exit(-1) + finish_time = time.time() + report_times(preboot_time, boot_time, init_time, finish_time) # Mark the beginning of the accumulators section of the output write_int(-1, old_stdout) for aid, accum in _accumulatorRegistry.items(): -- cgit v1.2.3 From c79a6078c34c207ad9f9910252f5849424828bf1 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Mon, 6 May 2013 16:34:30 -0700 Subject: Prefork Python worker processes --- core/src/main/scala/spark/SparkEnv.scala | 11 +++ .../main/scala/spark/api/python/PythonRDD.scala | 66 +++++-------- .../main/scala/spark/api/python/PythonWorker.scala | 89 +++++++++++++++++ python/pyspark/daemon.py | 109 +++++++++++++++++++++ python/pyspark/worker.py | 61 ++++++------ 5 files changed, 263 insertions(+), 73 deletions(-) create mode 100644 core/src/main/scala/spark/api/python/PythonWorker.scala create mode 100644 python/pyspark/daemon.py diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index be1a04d619..5691e24c32 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -1,5 +1,8 @@ package spark +import collection.mutable +import serializer.Serializer + import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem} import akka.remote.RemoteActorRefProvider @@ -9,6 +12,7 @@ import spark.storage.BlockManagerMaster import spark.network.ConnectionManager import spark.serializer.{Serializer, SerializerManager} import spark.util.AkkaUtils +import spark.api.python.PythonWorker /** @@ -37,6 +41,8 @@ class SparkEnv ( // If executorId is NOT found, return defaultHostPort var executorIdToHostPort: Option[(String, String) => String]) { + private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorker]() + def stop() { httpFileServer.stop() mapOutputTracker.stop() @@ -50,6 +56,11 @@ class SparkEnv ( actorSystem.awaitTermination() } + def getPythonWorker(pythonExec: String, envVars: Map[String, String]): PythonWorker = { + synchronized { + pythonWorkers.getOrElseUpdate((pythonExec, envVars), new PythonWorker(pythonExec, envVars)) + } + } def resolveExecutorIdToHostPort(executorId: String, defaultHostPort: String): String = { val env = SparkEnv.get diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index e9978d713f..e5acc54c01 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -2,10 +2,9 @@ package spark.api.python import java.io._ import java.net._ -import java.util.{List => JList, ArrayList => JArrayList, Collections} +import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections} import scala.collection.JavaConversions._ -import scala.io.Source import spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import spark.broadcast.Broadcast @@ -16,7 +15,7 @@ import spark.rdd.PipedRDD private[spark] class PythonRDD[T: ClassManifest]( parent: RDD[T], command: Seq[String], - envVars: java.util.Map[String, String], + envVars: JMap[String, String], preservePartitoning: Boolean, pythonExec: String, broadcastVars: JList[Broadcast[Array[Byte]]], @@ -25,7 +24,7 @@ private[spark] class PythonRDD[T: ClassManifest]( // Similar to Runtime.exec(), if we are given a single string, split it into words // using a standard StringTokenizer (i.e. by spaces) - def this(parent: RDD[T], command: String, envVars: java.util.Map[String, String], + def this(parent: RDD[T], command: String, envVars: JMap[String, String], preservePartitoning: Boolean, pythonExec: String, broadcastVars: JList[Broadcast[Array[Byte]]], accumulator: Accumulator[JList[Array[Byte]]]) = @@ -36,36 +35,18 @@ private[spark] class PythonRDD[T: ClassManifest]( override val partitioner = if (preservePartitoning) parent.partitioner else None - override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = { - val SPARK_HOME = new ProcessBuilder().environment().get("SPARK_HOME") - - val pb = new ProcessBuilder(Seq(pythonExec, SPARK_HOME + "/python/pyspark/worker.py")) - // Add the environmental variables to the process. - val currentEnvVars = pb.environment() - - for ((variable, value) <- envVars) { - currentEnvVars.put(variable, value) - } + override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = { val startTime = System.currentTimeMillis - val proc = pb.start() + val worker = SparkEnv.get.getPythonWorker(pythonExec, envVars.toMap).create val env = SparkEnv.get - // Start a thread to print the process's stderr to ours - new Thread("stderr reader for " + pythonExec) { - override def run() { - for (line <- Source.fromInputStream(proc.getErrorStream).getLines) { - System.err.println(line) - } - } - }.start() - // Start a thread to feed the process input from our parent's iterator new Thread("stdin writer for " + pythonExec) { override def run() { SparkEnv.set(env) - val out = new PrintWriter(proc.getOutputStream) - val dOut = new DataOutputStream(proc.getOutputStream) + val out = new PrintWriter(worker.getOutputStream) + val dOut = new DataOutputStream(worker.getOutputStream) // Partition index dOut.writeInt(split.index) // sparkFilesDir @@ -89,16 +70,21 @@ private[spark] class PythonRDD[T: ClassManifest]( } dOut.flush() out.flush() - proc.getOutputStream.close() + worker.shutdownOutput() } }.start() // Return an iterator that read lines from the process's stdout - val stream = new DataInputStream(proc.getInputStream) + val stream = new DataInputStream(worker.getInputStream) return new Iterator[Array[Byte]] { def next(): Array[Byte] = { val obj = _nextObj - _nextObj = read() + if (hasNext) { + // FIXME: can deadlock if worker is waiting for us to + // respond to current message (currently irrelevant because + // output is shutdown before we read any input) + _nextObj = read() + } obj } @@ -110,7 +96,7 @@ private[spark] class PythonRDD[T: ClassManifest]( stream.readFully(obj) obj case -3 => - // Timing data from child + // Timing data from worker val bootTime = stream.readLong() val initTime = stream.readLong() val finishTime = stream.readLong() @@ -127,23 +113,21 @@ private[spark] class PythonRDD[T: ClassManifest]( stream.readFully(obj) throw new PythonException(new String(obj)) case -1 => - // We've finished the data section of the output, but we can still read some - // accumulator updates; let's do that, breaking when we get EOFException - while (true) { - val len2 = stream.readInt() + // We've finished the data section of the output, but we can still + // read some accumulator updates; let's do that, breaking when we + // get a negative length record. + var len2 = stream.readInt + while (len2 >= 0) { val update = new Array[Byte](len2) stream.readFully(update) accumulator += Collections.singletonList(update) + len2 = stream.readInt } new Array[Byte](0) } } catch { case eof: EOFException => { - val exitStatus = proc.waitFor() - if (exitStatus != 0) { - throw new Exception("Subprocess exited with status " + exitStatus) - } - new Array[Byte](0) + throw new SparkException("Python worker exited unexpectedly (crashed)", eof) } case e => throw e } @@ -171,7 +155,7 @@ private class PairwiseRDD(prev: RDD[Array[Byte]]) extends override def compute(split: Partition, context: TaskContext) = prev.iterator(split, context).grouped(2).map { case Seq(a, b) => (a, b) - case x => throw new Exception("PairwiseRDD: unexpected value: " + x) + case x => throw new SparkException("PairwiseRDD: unexpected value: " + x) } val asJavaPairRDD : JavaPairRDD[Array[Byte], Array[Byte]] = JavaPairRDD.fromRDD(this) } @@ -227,7 +211,7 @@ private[spark] object PythonRDD { dOut.write(s) dOut.writeByte(Pickle.STOP) } else { - throw new Exception("Unexpected RDD type") + throw new SparkException("Unexpected RDD type") } } diff --git a/core/src/main/scala/spark/api/python/PythonWorker.scala b/core/src/main/scala/spark/api/python/PythonWorker.scala new file mode 100644 index 0000000000..8ee3c6884f --- /dev/null +++ b/core/src/main/scala/spark/api/python/PythonWorker.scala @@ -0,0 +1,89 @@ +package spark.api.python + +import java.io.DataInputStream +import java.net.{Socket, SocketException, InetAddress} + +import scala.collection.JavaConversions._ + +import spark._ + +private[spark] class PythonWorker(pythonExec: String, envVars: Map[String, String]) + extends Logging { + var daemon: Process = null + val daemonHost = InetAddress.getByAddress(Array(127, 0, 0, 1)) + var daemonPort: Int = 0 + + def create(): Socket = { + synchronized { + // Start the daemon if it hasn't been started + startDaemon + + // Attempt to connect, restart and retry once if it fails + try { + new Socket(daemonHost, daemonPort) + } catch { + case exc: SocketException => { + logWarning("Python daemon unexpectedly quit, attempting to restart") + stopDaemon + startDaemon + new Socket(daemonHost, daemonPort) + } + case e => throw e + } + } + } + + private def startDaemon() { + synchronized { + // Is it already running? + if (daemon != null) { + return + } + + try { + // Create and start the daemon + val sparkHome = new ProcessBuilder().environment().get("SPARK_HOME") + val pb = new ProcessBuilder(Seq(pythonExec, sparkHome + "/python/pyspark/daemon.py")) + val workerEnv = pb.environment() + workerEnv.putAll(envVars) + daemon = pb.start() + daemonPort = new DataInputStream(daemon.getInputStream).readInt + + // Redirect the stderr to ours + new Thread("stderr reader for " + pythonExec) { + override def run() { + // FIXME HACK: We copy the stream on the level of bytes to + // attempt to dodge encoding problems. + val in = daemon.getErrorStream + var buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + System.err.write(buf, 0, len) + len = in.read(buf) + } + } + }.start() + } catch { + case e => { + stopDaemon + throw e + } + } + + // Important: don't close daemon's stdin (daemon.getOutputStream) so it can correctly + // detect our disappearance. + } + } + + private def stopDaemon() { + synchronized { + // Request shutdown of existing daemon by sending SIGTERM + if (daemon != null) { + daemon.destroy + } + + daemon = null + daemonPort = 0 + } + } +} diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py new file mode 100644 index 0000000000..642f30b2b9 --- /dev/null +++ b/python/pyspark/daemon.py @@ -0,0 +1,109 @@ +import os +import sys +import multiprocessing +from errno import EINTR, ECHILD +from socket import socket, AF_INET, SOCK_STREAM, SOMAXCONN +from signal import signal, SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN +from pyspark.worker import main as worker_main +from pyspark.serializers import write_int + +try: + POOLSIZE = multiprocessing.cpu_count() +except NotImplementedError: + POOLSIZE = 4 + +should_exit = False + + +def worker(listen_sock): + # Redirect stdout to stderr + os.dup2(2, 1) + + # Manager sends SIGHUP to request termination of workers in the pool + def handle_sighup(signum, frame): + global should_exit + should_exit = True + signal(SIGHUP, handle_sighup) + + while not should_exit: + # Wait until a client arrives or we have to exit + sock = None + while not should_exit and sock is None: + try: + sock, addr = listen_sock.accept() + except EnvironmentError as err: + if err.errno != EINTR: + raise + + if sock is not None: + # Fork a child to handle the client + if os.fork() == 0: + # Leave the worker pool + signal(SIGHUP, SIG_DFL) + listen_sock.close() + # Handle the client then exit + sockfile = sock.makefile() + worker_main(sockfile, sockfile) + sockfile.close() + sock.close() + os._exit(0) + else: + sock.close() + + assert should_exit + os._exit(0) + + +def manager(): + # Create a new process group to corral our children + os.setpgid(0, 0) + + # Create a listening socket on the AF_INET loopback interface + listen_sock = socket(AF_INET, SOCK_STREAM) + listen_sock.bind(('127.0.0.1', 0)) + listen_sock.listen(max(1024, 2 * POOLSIZE, SOMAXCONN)) + listen_host, listen_port = listen_sock.getsockname() + write_int(listen_port, sys.stdout) + + # Launch initial worker pool + for idx in range(POOLSIZE): + if os.fork() == 0: + worker(listen_sock) + raise RuntimeError("worker() unexpectedly returned") + listen_sock.close() + + def shutdown(): + global should_exit + os.kill(0, SIGHUP) + should_exit = True + + # Gracefully exit on SIGTERM, don't die on SIGHUP + signal(SIGTERM, lambda signum, frame: shutdown()) + signal(SIGHUP, SIG_IGN) + + # Cleanup zombie children + def handle_sigchld(signum, frame): + try: + pid, status = os.waitpid(0, os.WNOHANG) + if (pid, status) != (0, 0) and not should_exit: + raise RuntimeError("pool member crashed: %s, %s" % (pid, status)) + except EnvironmentError as err: + if err.errno not in (ECHILD, EINTR): + raise + signal(SIGCHLD, handle_sigchld) + + # Initialization complete + sys.stdout.close() + while not should_exit: + try: + # Spark tells us to exit by closing stdin + if sys.stdin.read() == '': + shutdown() + except EnvironmentError as err: + if err.errno != EINTR: + shutdown() + raise + + +if __name__ == '__main__': + manager() diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 4c33ae49dc..94d612ea6e 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -1,10 +1,9 @@ """ Worker that receives input from Piped RDD. """ -import time -preboot_time = time.time() import os import sys +import time import traceback from base64 import standard_b64decode # CloudPickler needs to be imported so that depicklers are registered using the @@ -17,57 +16,55 @@ from pyspark.serializers import write_with_length, read_with_length, write_int, read_long, write_long, read_int, dump_pickle, load_pickle, read_from_pickle_file -# Redirect stdout to stderr so that users must return values from functions. -old_stdout = os.fdopen(os.dup(1), 'w') -os.dup2(2, 1) - - -def load_obj(): - return load_pickle(standard_b64decode(sys.stdin.readline().strip())) +def load_obj(infile): + return load_pickle(standard_b64decode(infile.readline().strip())) -def report_times(preboot, boot, init, finish): - write_int(-3, old_stdout) - write_long(1000 * preboot, old_stdout) - write_long(1000 * boot, old_stdout) - write_long(1000 * init, old_stdout) - write_long(1000 * finish, old_stdout) +def report_times(outfile, boot, init, finish): + write_int(-3, outfile) + write_long(1000 * boot, outfile) + write_long(1000 * init, outfile) + write_long(1000 * finish, outfile) -def main(): +def main(infile, outfile): boot_time = time.time() - split_index = read_int(sys.stdin) - spark_files_dir = load_pickle(read_with_length(sys.stdin)) + split_index = read_int(infile) + spark_files_dir = load_pickle(read_with_length(infile)) SparkFiles._root_directory = spark_files_dir SparkFiles._is_running_on_worker = True sys.path.append(spark_files_dir) - num_broadcast_variables = read_int(sys.stdin) + num_broadcast_variables = read_int(infile) for _ in range(num_broadcast_variables): - bid = read_long(sys.stdin) - value = read_with_length(sys.stdin) + bid = read_long(infile) + value = read_with_length(infile) _broadcastRegistry[bid] = Broadcast(bid, load_pickle(value)) - func = load_obj() - bypassSerializer = load_obj() + func = load_obj(infile) + bypassSerializer = load_obj(infile) if bypassSerializer: dumps = lambda x: x else: dumps = dump_pickle init_time = time.time() - iterator = read_from_pickle_file(sys.stdin) + iterator = read_from_pickle_file(infile) try: for obj in func(split_index, iterator): - write_with_length(dumps(obj), old_stdout) + write_with_length(dumps(obj), outfile) except Exception as e: - write_int(-2, old_stdout) - write_with_length(traceback.format_exc(), old_stdout) - sys.exit(-1) + write_int(-2, outfile) + write_with_length(traceback.format_exc(), outfile) + raise finish_time = time.time() - report_times(preboot_time, boot_time, init_time, finish_time) + report_times(outfile, boot_time, init_time, finish_time) # Mark the beginning of the accumulators section of the output - write_int(-1, old_stdout) + write_int(-1, outfile) for aid, accum in _accumulatorRegistry.items(): - write_with_length(dump_pickle((aid, accum._value)), old_stdout) + write_with_length(dump_pickle((aid, accum._value)), outfile) + write_int(-1, outfile) if __name__ == '__main__': - main() + # Redirect stdout to stderr so that users must return values from functions. + old_stdout = os.fdopen(os.dup(1), 'w') + os.dup2(2, 1) + main(sys.stdin, old_stdout) -- cgit v1.2.3 From 62c4781400dd908c2fccdcebf0dc816ff0cb8ed4 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Fri, 10 May 2013 15:48:48 -0700 Subject: Add tests and fixes for Python daemon shutdown --- core/src/main/scala/spark/SparkEnv.scala | 1 + .../main/scala/spark/api/python/PythonWorker.scala | 4 ++ python/pyspark/daemon.py | 46 +++++++++++----------- python/pyspark/tests.py | 43 ++++++++++++++++++++ python/pyspark/worker.py | 2 + 5 files changed, 74 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 5691e24c32..5b55d45212 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -44,6 +44,7 @@ class SparkEnv ( private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorker]() def stop() { + pythonWorkers.foreach { case(key, worker) => worker.stop() } httpFileServer.stop() mapOutputTracker.stop() shuffleFetcher.stop() diff --git a/core/src/main/scala/spark/api/python/PythonWorker.scala b/core/src/main/scala/spark/api/python/PythonWorker.scala index 8ee3c6884f..74c8c6d37a 100644 --- a/core/src/main/scala/spark/api/python/PythonWorker.scala +++ b/core/src/main/scala/spark/api/python/PythonWorker.scala @@ -33,6 +33,10 @@ private[spark] class PythonWorker(pythonExec: String, envVars: Map[String, Strin } } + def stop() { + stopDaemon + } + private def startDaemon() { synchronized { // Is it already running? diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 642f30b2b9..ab9c19df57 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -12,7 +12,7 @@ try: except NotImplementedError: POOLSIZE = 4 -should_exit = False +should_exit = multiprocessing.Event() def worker(listen_sock): @@ -21,14 +21,13 @@ def worker(listen_sock): # Manager sends SIGHUP to request termination of workers in the pool def handle_sighup(signum, frame): - global should_exit - should_exit = True + assert should_exit.is_set() signal(SIGHUP, handle_sighup) - while not should_exit: + while not should_exit.is_set(): # Wait until a client arrives or we have to exit sock = None - while not should_exit and sock is None: + while not should_exit.is_set() and sock is None: try: sock, addr = listen_sock.accept() except EnvironmentError as err: @@ -36,8 +35,8 @@ def worker(listen_sock): raise if sock is not None: - # Fork a child to handle the client - if os.fork() == 0: + # Fork to handle the client + if os.fork() != 0: # Leave the worker pool signal(SIGHUP, SIG_DFL) listen_sock.close() @@ -50,7 +49,7 @@ def worker(listen_sock): else: sock.close() - assert should_exit + assert should_exit.is_set() os._exit(0) @@ -73,9 +72,7 @@ def manager(): listen_sock.close() def shutdown(): - global should_exit - os.kill(0, SIGHUP) - should_exit = True + should_exit.set() # Gracefully exit on SIGTERM, don't die on SIGHUP signal(SIGTERM, lambda signum, frame: shutdown()) @@ -85,8 +82,8 @@ def manager(): def handle_sigchld(signum, frame): try: pid, status = os.waitpid(0, os.WNOHANG) - if (pid, status) != (0, 0) and not should_exit: - raise RuntimeError("pool member crashed: %s, %s" % (pid, status)) + if status != 0 and not should_exit.is_set(): + raise RuntimeError("worker crashed: %s, %s" % (pid, status)) except EnvironmentError as err: if err.errno not in (ECHILD, EINTR): raise @@ -94,15 +91,20 @@ def manager(): # Initialization complete sys.stdout.close() - while not should_exit: - try: - # Spark tells us to exit by closing stdin - if sys.stdin.read() == '': - shutdown() - except EnvironmentError as err: - if err.errno != EINTR: - shutdown() - raise + try: + while not should_exit.is_set(): + try: + # Spark tells us to exit by closing stdin + if os.read(0, 512) == '': + shutdown() + except EnvironmentError as err: + if err.errno != EINTR: + shutdown() + raise + finally: + should_exit.set() + # Send SIGHUP to notify workers of shutdown + os.kill(0, SIGHUP) if __name__ == '__main__': diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 6a1962d267..1e34d47365 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -12,6 +12,7 @@ import unittest from pyspark.context import SparkContext from pyspark.files import SparkFiles from pyspark.java_gateway import SPARK_HOME +from pyspark.serializers import read_int class PySparkTestCase(unittest.TestCase): @@ -117,5 +118,47 @@ class TestIO(PySparkTestCase): self.sc.parallelize([1]).foreach(func) +class TestDaemon(unittest.TestCase): + def connect(self, port): + from socket import socket, AF_INET, SOCK_STREAM + sock = socket(AF_INET, SOCK_STREAM) + sock.connect(('127.0.0.1', port)) + # send a split index of -1 to shutdown the worker + sock.send("\xFF\xFF\xFF\xFF") + sock.close() + return True + + def do_termination_test(self, terminator): + from subprocess import Popen, PIPE + from errno import ECONNREFUSED + + # start daemon + daemon_path = os.path.join(os.path.dirname(__file__), "daemon.py") + daemon = Popen([sys.executable, daemon_path], stdin=PIPE, stdout=PIPE) + + # read the port number + port = read_int(daemon.stdout) + + # daemon should accept connections + self.assertTrue(self.connect(port)) + + # request shutdown + terminator(daemon) + time.sleep(1) + + # daemon should no longer accept connections + with self.assertRaises(EnvironmentError) as trap: + self.connect(port) + self.assertEqual(trap.exception.errno, ECONNREFUSED) + + def test_termination_stdin(self): + """Ensure that daemon and workers terminate when stdin is closed.""" + self.do_termination_test(lambda daemon: daemon.stdin.close()) + + def test_termination_sigterm(self): + """Ensure that daemon and workers terminate on SIGTERM.""" + from signal import SIGTERM + self.do_termination_test(lambda daemon: os.kill(daemon.pid, SIGTERM)) + if __name__ == "__main__": unittest.main() diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 94d612ea6e..f76ee3c236 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -30,6 +30,8 @@ def report_times(outfile, boot, init, finish): def main(infile, outfile): boot_time = time.time() split_index = read_int(infile) + if split_index == -1: # for unit tests + return spark_files_dir = load_pickle(read_with_length(infile)) SparkFiles._root_directory = spark_files_dir SparkFiles._is_running_on_worker = True -- cgit v1.2.3 From edb18ca928c988a713b9228bb74af1737f2b614b Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Mon, 13 May 2013 08:53:47 -0700 Subject: Rename PythonWorker to PythonWorkerFactory --- core/src/main/scala/spark/SparkEnv.scala | 8 +- .../main/scala/spark/api/python/PythonRDD.scala | 2 +- .../main/scala/spark/api/python/PythonWorker.scala | 93 --------------------- .../spark/api/python/PythonWorkerFactory.scala | 95 ++++++++++++++++++++++ 4 files changed, 100 insertions(+), 98 deletions(-) delete mode 100644 core/src/main/scala/spark/api/python/PythonWorker.scala create mode 100644 core/src/main/scala/spark/api/python/PythonWorkerFactory.scala diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 5b55d45212..0a23c45658 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -12,7 +12,7 @@ import spark.storage.BlockManagerMaster import spark.network.ConnectionManager import spark.serializer.{Serializer, SerializerManager} import spark.util.AkkaUtils -import spark.api.python.PythonWorker +import spark.api.python.PythonWorkerFactory /** @@ -41,7 +41,7 @@ class SparkEnv ( // If executorId is NOT found, return defaultHostPort var executorIdToHostPort: Option[(String, String) => String]) { - private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorker]() + private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() def stop() { pythonWorkers.foreach { case(key, worker) => worker.stop() } @@ -57,9 +57,9 @@ class SparkEnv ( actorSystem.awaitTermination() } - def getPythonWorker(pythonExec: String, envVars: Map[String, String]): PythonWorker = { + def createPythonWorker(pythonExec: String, envVars: Map[String, String]): java.net.Socket = { synchronized { - pythonWorkers.getOrElseUpdate((pythonExec, envVars), new PythonWorker(pythonExec, envVars)) + pythonWorkers.getOrElseUpdate((pythonExec, envVars), new PythonWorkerFactory(pythonExec, envVars)).create } } diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index e5acc54c01..3c48071b3f 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -38,8 +38,8 @@ private[spark] class PythonRDD[T: ClassManifest]( override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = { val startTime = System.currentTimeMillis - val worker = SparkEnv.get.getPythonWorker(pythonExec, envVars.toMap).create val env = SparkEnv.get + val worker = env.createPythonWorker(pythonExec, envVars.toMap) // Start a thread to feed the process input from our parent's iterator new Thread("stdin writer for " + pythonExec) { diff --git a/core/src/main/scala/spark/api/python/PythonWorker.scala b/core/src/main/scala/spark/api/python/PythonWorker.scala deleted file mode 100644 index 74c8c6d37a..0000000000 --- a/core/src/main/scala/spark/api/python/PythonWorker.scala +++ /dev/null @@ -1,93 +0,0 @@ -package spark.api.python - -import java.io.DataInputStream -import java.net.{Socket, SocketException, InetAddress} - -import scala.collection.JavaConversions._ - -import spark._ - -private[spark] class PythonWorker(pythonExec: String, envVars: Map[String, String]) - extends Logging { - var daemon: Process = null - val daemonHost = InetAddress.getByAddress(Array(127, 0, 0, 1)) - var daemonPort: Int = 0 - - def create(): Socket = { - synchronized { - // Start the daemon if it hasn't been started - startDaemon - - // Attempt to connect, restart and retry once if it fails - try { - new Socket(daemonHost, daemonPort) - } catch { - case exc: SocketException => { - logWarning("Python daemon unexpectedly quit, attempting to restart") - stopDaemon - startDaemon - new Socket(daemonHost, daemonPort) - } - case e => throw e - } - } - } - - def stop() { - stopDaemon - } - - private def startDaemon() { - synchronized { - // Is it already running? - if (daemon != null) { - return - } - - try { - // Create and start the daemon - val sparkHome = new ProcessBuilder().environment().get("SPARK_HOME") - val pb = new ProcessBuilder(Seq(pythonExec, sparkHome + "/python/pyspark/daemon.py")) - val workerEnv = pb.environment() - workerEnv.putAll(envVars) - daemon = pb.start() - daemonPort = new DataInputStream(daemon.getInputStream).readInt - - // Redirect the stderr to ours - new Thread("stderr reader for " + pythonExec) { - override def run() { - // FIXME HACK: We copy the stream on the level of bytes to - // attempt to dodge encoding problems. - val in = daemon.getErrorStream - var buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - System.err.write(buf, 0, len) - len = in.read(buf) - } - } - }.start() - } catch { - case e => { - stopDaemon - throw e - } - } - - // Important: don't close daemon's stdin (daemon.getOutputStream) so it can correctly - // detect our disappearance. - } - } - - private def stopDaemon() { - synchronized { - // Request shutdown of existing daemon by sending SIGTERM - if (daemon != null) { - daemon.destroy - } - - daemon = null - daemonPort = 0 - } - } -} diff --git a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala new file mode 100644 index 0000000000..ebbd226b3e --- /dev/null +++ b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala @@ -0,0 +1,95 @@ +package spark.api.python + +import java.io.{DataInputStream, IOException} +import java.net.{Socket, SocketException, InetAddress} + +import scala.collection.JavaConversions._ + +import spark._ + +private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String, String]) + extends Logging { + var daemon: Process = null + val daemonHost = InetAddress.getByAddress(Array(127, 0, 0, 1)) + var daemonPort: Int = 0 + + def create(): Socket = { + synchronized { + // Start the daemon if it hasn't been started + startDaemon + + // Attempt to connect, restart and retry once if it fails + try { + new Socket(daemonHost, daemonPort) + } catch { + case exc: SocketException => { + logWarning("Python daemon unexpectedly quit, attempting to restart") + stopDaemon + startDaemon + new Socket(daemonHost, daemonPort) + } + case e => throw e + } + } + } + + def stop() { + stopDaemon + } + + private def startDaemon() { + synchronized { + // Is it already running? + if (daemon != null) { + return + } + + try { + // Create and start the daemon + val sparkHome = new ProcessBuilder().environment().get("SPARK_HOME") + val pb = new ProcessBuilder(Seq(pythonExec, sparkHome + "/python/pyspark/daemon.py")) + val workerEnv = pb.environment() + workerEnv.putAll(envVars) + daemon = pb.start() + daemonPort = new DataInputStream(daemon.getInputStream).readInt + + // Redirect the stderr to ours + new Thread("stderr reader for " + pythonExec) { + override def run() { + scala.util.control.Exception.ignoring(classOf[IOException]) { + // FIXME HACK: We copy the stream on the level of bytes to + // attempt to dodge encoding problems. + val in = daemon.getErrorStream + var buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + System.err.write(buf, 0, len) + len = in.read(buf) + } + } + } + }.start() + } catch { + case e => { + stopDaemon + throw e + } + } + + // Important: don't close daemon's stdin (daemon.getOutputStream) so it can correctly + // detect our disappearance. + } + } + + private def stopDaemon() { + synchronized { + // Request shutdown of existing daemon by sending SIGTERM + if (daemon != null) { + daemon.destroy + } + + daemon = null + daemonPort = 0 + } + } +} -- cgit v1.2.3 From 7c5ff733ee1d3729b4b26f7c5542ca00c4d64139 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Thu, 23 May 2013 11:50:24 -0700 Subject: PySpark daemon: fix deadlock, improve error handling --- python/pyspark/daemon.py | 67 ++++++++++++++++++++++++++++++++++++------------ 1 file changed, 50 insertions(+), 17 deletions(-) diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index ab9c19df57..2b5e9b3581 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -1,6 +1,7 @@ import os import sys import multiprocessing +from ctypes import c_bool from errno import EINTR, ECHILD from socket import socket, AF_INET, SOCK_STREAM, SOMAXCONN from signal import signal, SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN @@ -12,7 +13,12 @@ try: except NotImplementedError: POOLSIZE = 4 -should_exit = multiprocessing.Event() +exit_flag = multiprocessing.Value(c_bool, False) + + +def should_exit(): + global exit_flag + return exit_flag.value def worker(listen_sock): @@ -20,14 +26,29 @@ def worker(listen_sock): os.dup2(2, 1) # Manager sends SIGHUP to request termination of workers in the pool - def handle_sighup(signum, frame): - assert should_exit.is_set() + def handle_sighup(*args): + assert should_exit() signal(SIGHUP, handle_sighup) - while not should_exit.is_set(): + # Cleanup zombie children + def handle_sigchld(*args): + pid = status = None + try: + while (pid, status) != (0, 0): + pid, status = os.waitpid(0, os.WNOHANG) + except EnvironmentError as err: + if err.errno == EINTR: + # retry + handle_sigchld() + elif err.errno != ECHILD: + raise + signal(SIGCHLD, handle_sigchld) + + # Handle clients + while not should_exit(): # Wait until a client arrives or we have to exit sock = None - while not should_exit.is_set() and sock is None: + while not should_exit() and sock is None: try: sock, addr = listen_sock.accept() except EnvironmentError as err: @@ -35,8 +56,10 @@ def worker(listen_sock): raise if sock is not None: - # Fork to handle the client - if os.fork() != 0: + # Fork a child to handle the client. + # The client is handled in the child so that the manager + # never receives SIGCHLD unless a worker crashes. + if os.fork() == 0: # Leave the worker pool signal(SIGHUP, SIG_DFL) listen_sock.close() @@ -49,8 +72,18 @@ def worker(listen_sock): else: sock.close() - assert should_exit.is_set() - os._exit(0) + +def launch_worker(listen_sock): + if os.fork() == 0: + try: + worker(listen_sock) + except Exception as err: + import traceback + traceback.print_exc() + os._exit(1) + else: + assert should_exit() + os._exit(0) def manager(): @@ -66,23 +99,22 @@ def manager(): # Launch initial worker pool for idx in range(POOLSIZE): - if os.fork() == 0: - worker(listen_sock) - raise RuntimeError("worker() unexpectedly returned") + launch_worker(listen_sock) listen_sock.close() def shutdown(): - should_exit.set() + global exit_flag + exit_flag.value = True # Gracefully exit on SIGTERM, don't die on SIGHUP signal(SIGTERM, lambda signum, frame: shutdown()) signal(SIGHUP, SIG_IGN) # Cleanup zombie children - def handle_sigchld(signum, frame): + def handle_sigchld(*args): try: pid, status = os.waitpid(0, os.WNOHANG) - if status != 0 and not should_exit.is_set(): + if status != 0 and not should_exit(): raise RuntimeError("worker crashed: %s, %s" % (pid, status)) except EnvironmentError as err: if err.errno not in (ECHILD, EINTR): @@ -92,7 +124,7 @@ def manager(): # Initialization complete sys.stdout.close() try: - while not should_exit.is_set(): + while not should_exit(): try: # Spark tells us to exit by closing stdin if os.read(0, 512) == '': @@ -102,7 +134,8 @@ def manager(): shutdown() raise finally: - should_exit.set() + signal(SIGTERM, SIG_DFL) + exit_flag.value = True # Send SIGHUP to notify workers of shutdown os.kill(0, SIGHUP) -- cgit v1.2.3 From 1ba3c173034c37ef99fc312c84943d2ab8885670 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Thu, 20 Jun 2013 12:49:10 -0400 Subject: use parens when calling method with side-effects --- core/src/main/scala/spark/SparkEnv.scala | 2 +- core/src/main/scala/spark/api/python/PythonRDD.scala | 4 ++-- .../main/scala/spark/api/python/PythonWorkerFactory.scala | 14 +++++++------- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 0a23c45658..7ccde2e818 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -59,7 +59,7 @@ class SparkEnv ( def createPythonWorker(pythonExec: String, envVars: Map[String, String]): java.net.Socket = { synchronized { - pythonWorkers.getOrElseUpdate((pythonExec, envVars), new PythonWorkerFactory(pythonExec, envVars)).create + pythonWorkers.getOrElseUpdate((pythonExec, envVars), new PythonWorkerFactory(pythonExec, envVars)).create() } } diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 3c48071b3f..63140cf37f 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -116,12 +116,12 @@ private[spark] class PythonRDD[T: ClassManifest]( // We've finished the data section of the output, but we can still // read some accumulator updates; let's do that, breaking when we // get a negative length record. - var len2 = stream.readInt + var len2 = stream.readInt() while (len2 >= 0) { val update = new Array[Byte](len2) stream.readFully(update) accumulator += Collections.singletonList(update) - len2 = stream.readInt + len2 = stream.readInt() } new Array[Byte](0) } diff --git a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala index ebbd226b3e..8844411d73 100644 --- a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala @@ -16,7 +16,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String def create(): Socket = { synchronized { // Start the daemon if it hasn't been started - startDaemon + startDaemon() // Attempt to connect, restart and retry once if it fails try { @@ -24,8 +24,8 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } catch { case exc: SocketException => { logWarning("Python daemon unexpectedly quit, attempting to restart") - stopDaemon - startDaemon + stopDaemon() + startDaemon() new Socket(daemonHost, daemonPort) } case e => throw e @@ -34,7 +34,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } def stop() { - stopDaemon + stopDaemon() } private def startDaemon() { @@ -51,7 +51,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String val workerEnv = pb.environment() workerEnv.putAll(envVars) daemon = pb.start() - daemonPort = new DataInputStream(daemon.getInputStream).readInt + daemonPort = new DataInputStream(daemon.getInputStream).readInt() // Redirect the stderr to ours new Thread("stderr reader for " + pythonExec) { @@ -71,7 +71,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String }.start() } catch { case e => { - stopDaemon + stopDaemon() throw e } } @@ -85,7 +85,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String synchronized { // Request shutdown of existing daemon by sending SIGTERM if (daemon != null) { - daemon.destroy + daemon.destroy() } daemon = null -- cgit v1.2.3 From c75bed0eebb1f937db02eb98deecd380724f747d Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Fri, 21 Jun 2013 12:13:48 -0400 Subject: Fix reporting of PySpark exceptions --- python/pyspark/daemon.py | 22 ++++++++++++++++++---- python/pyspark/worker.py | 2 +- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 2b5e9b3581..78a2da1e18 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -21,6 +21,15 @@ def should_exit(): return exit_flag.value +def compute_real_exit_code(exit_code): + # SystemExit's code can be integer or string, but os._exit only accepts integers + import numbers + if isinstance(exit_code, numbers.Integral): + return exit_code + else: + return 1 + + def worker(listen_sock): # Redirect stdout to stderr os.dup2(2, 1) @@ -65,10 +74,15 @@ def worker(listen_sock): listen_sock.close() # Handle the client then exit sockfile = sock.makefile() - worker_main(sockfile, sockfile) - sockfile.close() - sock.close() - os._exit(0) + exit_code = 0 + try: + worker_main(sockfile, sockfile) + except SystemExit as exc: + exit_code = exc.code + finally: + sockfile.close() + sock.close() + os._exit(compute_real_exit_code(exit_code)) else: sock.close() diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index f76ee3c236..379bbfd4c2 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -55,7 +55,7 @@ def main(infile, outfile): except Exception as e: write_int(-2, outfile) write_with_length(traceback.format_exc(), outfile) - raise + sys.exit(-1) finish_time = time.time() report_times(outfile, boot_time, init_time, finish_time) # Mark the beginning of the accumulators section of the output -- cgit v1.2.3 From b350f34703d4f29bbd0e603df852f7aae230b2a2 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 07:48:20 -0700 Subject: Increase memory for tests to prevent a crash on JDK 7 --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 24c8b734d0..faf6e2ae8e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -54,7 +54,7 @@ object SparkBuild extends Build { // Fork new JVMs for tests and set Java options for those fork := true, - javaOptions += "-Xmx2g", + javaOptions += "-Xmx2500m", // Only allow one test at a time, even across projects, since they run in the same JVM concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), -- cgit v1.2.3 From d92d3f7938dec954ea31de232f50cafd4b644065 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 10:24:19 -0700 Subject: Fix resolution of example code with Maven builds --- run | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/run b/run index c0065c53f1..e656e38ccf 100755 --- a/run +++ b/run @@ -132,10 +132,14 @@ if [ -e "$FWDIR/lib_managed" ]; then CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*" fi CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*" +# Add the shaded JAR for Maven builds if [ -e $REPL_BIN_DIR/target ]; then for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do CLASSPATH="$CLASSPATH:$jar" done + # The shaded JAR doesn't contain examples, so include those separately + EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` + CLASSPATH+=":$EXAMPLES_JAR" fi CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do @@ -148,9 +152,9 @@ if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; # Use the JAR from the SBT build export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` fi -if [ -e "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar ]; then +if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then # Use the JAR from the Maven build - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar` + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` fi # Add hadoop conf dir - else FileSystem.*, etc fail ! -- cgit v1.2.3 From 60fbf7e461ff23978ba2b56cf494aad70ffc0bf5 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 16 Jun 2013 20:08:25 -0700 Subject: Partially working checkpoint --- .../main/scala/spark/deploy/master/Master.scala | 2 + .../main/scala/spark/deploy/worker/Worker.scala | 2 + .../main/scala/spark/storage/BlockManagerUI.scala | 45 ++++++++++++++++++++-- core/src/main/scala/spark/util/AkkaUtils.scala | 36 ++++++++--------- 4 files changed, 63 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 3e965e82ac..edaacd87b8 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -57,6 +57,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act def startWebUi() { val webUi = new MasterWebUI(context.system, self) + /* try { AkkaUtils.startSprayServer(context.system, "0.0.0.0", webUiPort, webUi.handler) } catch { @@ -64,6 +65,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act logError("Failed to create web UI", e) System.exit(1) } + */ } override def receive = { diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 3dc2207170..f8fdab927e 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -89,6 +89,7 @@ private[spark] class Worker( def startWebUi() { val webUi = new WorkerWebUI(context.system, self, workDir) + /* try { AkkaUtils.startSprayServer(context.system, "0.0.0.0", webUiPort, webUi.handler) } catch { @@ -96,6 +97,7 @@ private[spark] class Worker( logError("Failed to create web UI", e) System.exit(1) } + */ } override def receive = { diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index e02281344a..f33855fad2 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -8,6 +8,12 @@ import cc.spray.Directives import spark.{Logging, SparkContext} import spark.util.AkkaUtils import spark.Utils +import spark.util.WebUI +import org.eclipse.jetty.server.handler.{HandlerList, ContextHandler, ResourceHandler} +import org.eclipse.jetty.server.Handler +import xml.Elem +import xml.Node +import java.net.URLClassLoader /** @@ -32,7 +38,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, /** Start a HTTP server to run the Web interface */ def start() { try { - AkkaUtils.startSprayServer(actorSystem, "0.0.0.0", port, handler, "BlockManagerHTTPServer") + AkkaUtils.startJettyServer("0.0.0.0", port, handlers) logInfo("Started BlockManager web UI at http://%s:%d".format(host, port)) } catch { case e: Exception => @@ -41,6 +47,40 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, } } + val staticHandler = new ResourceHandler + println("RESOURCE: ") + val resource = getClass.getClassLoader.getResource(STATIC_RESOURCE_DIR) + staticHandler.setResourceBase(resource.toString) + + + val handlers = Array[(String, Handler)]( + ("/static", staticHandler), + ("*", WebUI.makeHandler(WebUI.makePage(indexContent, "Spark Storage"))) + ) + + def indexContent: Seq[Node] = { + val storageStatusList = sc.getExecutorStorageStatus + // Calculate macro-level statistics + val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) + .reduceOption(_+_).getOrElse(0L) + val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) + +
    +
    +
      +
    • Memory: + {Utils.memoryBytesToString(maxMem - remainingMem)} Used + ({Utils.memoryBytesToString(remainingMem)} Available)
    • +
    • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
    • +
    +
    +
    +
    + } + + /* val handler = { get { path("") { @@ -73,7 +113,6 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, getFromResourceDirectory(STATIC_RESOURCE_DIR) } } - } - + */ private[spark] def appUIAddress = "http://" + host + ":" + port } diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index e93cc3b485..12581f535f 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -13,6 +13,9 @@ import cc.spray.io.pipelines.MessageHandlerDispatch.SingletonHandler import akka.dispatch.Await import spark.{Utils, SparkException} import java.util.concurrent.TimeoutException +import org.eclipse.jetty.server.Server +import org.eclipse.jetty.server.Handler +import org.eclipse.jetty.server.handler.{HandlerList, ContextHandler} /** * Various utility classes for working with Akka. @@ -66,26 +69,21 @@ private[spark] object AkkaUtils { * handle requests. Returns the bound port or throws a SparkException on failure. * TODO: Not changing ip to host here - is it required ? */ - def startSprayServer(actorSystem: ActorSystem, ip: String, port: Int, route: Route, - name: String = "HttpServer"): ActorRef = { - val ioWorker = new IoWorker(actorSystem).start() - val httpService = actorSystem.actorOf(Props(new HttpService(route))) - val rootService = actorSystem.actorOf(Props(new SprayCanRootService(httpService))) - val server = actorSystem.actorOf( - Props(new HttpServer(ioWorker, SingletonHandler(rootService))), name = name) - actorSystem.registerOnTermination { ioWorker.stop() } - val timeout = 3.seconds - val future = server.ask(HttpServer.Bind(ip, port))(timeout) - try { - Await.result(future, timeout) match { - case bound: HttpServer.Bound => - return server - case other: Any => - throw new SparkException("Failed to bind web UI to port " + port + ": " + other) + def startJettyServer(ip: String, port: Int, handlers: Array[(String, Handler)]) = { + val handlersToRegister = handlers.map { case(path, handler) => + if (path == "*") { + handler + } else { + val contextHandler = new ContextHandler(path) + contextHandler.setHandler(handler) + contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] } - } catch { - case e: TimeoutException => - throw new SparkException("Failed to bind web UI to port " + port) } + + val handlerList = new HandlerList + handlerList.setHandlers(handlersToRegister) + val server = new Server(port) + server.setHandler(handlerList) + server.start() } } -- cgit v1.2.3 From e66f57019424bc5c7a1ca55cd335c34877b6d9cf Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 17 Jun 2013 13:53:13 -0700 Subject: Completely hacked version of block manager UI in jetty --- .../main/scala/spark/storage/BlockManagerUI.scala | 173 ++++++++++++++++----- core/src/main/scala/spark/util/AkkaUtils.scala | 23 --- 2 files changed, 134 insertions(+), 62 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index f33855fad2..e577d1b597 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -11,9 +11,11 @@ import spark.Utils import spark.util.WebUI import org.eclipse.jetty.server.handler.{HandlerList, ContextHandler, ResourceHandler} import org.eclipse.jetty.server.Handler +import javax.servlet.http.{HttpServletResponse, HttpServletRequest} import xml.Elem import xml.Node import java.net.URLClassLoader +import spark.util.WebUI._ /** @@ -38,7 +40,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, /** Start a HTTP server to run the Web interface */ def start() { try { - AkkaUtils.startJettyServer("0.0.0.0", port, handlers) + WebUI.startJettyServer("0.0.0.0", port, handlers) logInfo("Started BlockManager web UI at http://%s:%d".format(host, port)) } catch { case e: Exception => @@ -55,10 +57,109 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, val handlers = Array[(String, Handler)]( ("/static", staticHandler), - ("*", WebUI.makeHandler(WebUI.makePage(indexContent, "Spark Storage"))) + ("/rdd", (request: HttpServletRequest) => rddPage(request)), + ("*", (request: HttpServletRequest) => WebUI.makePage(indexPage, "Spark Storage")) ) - def indexContent: Seq[Node] = { + def rddPage(request: HttpServletRequest): Seq[Node] = { + val id = request.getParameter("id") + val prefix = "rdd_" + id.toString + val storageStatusList = sc.getExecutorStorageStatus + val filteredStorageStatusList = StorageUtils. + filterStorageStatusByPrefix(storageStatusList, prefix) + val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head + spark.storage.html.rdd.render(rddInfo, filteredStorageStatusList) + + val content = +
    +
    +
      +
    • + Storage Level: + {rddInfo.storageLevel.description} +
    • +
    • + Cached Partitions: + {rddInfo.numCachedPartitions} +
    • +
    • + Total Partitions: + {rddInfo.numPartitions} +
    • +
    • + Memory Size: + {Utils.memoryBytesToString(rddInfo.memSize)} +
    • +
    • + Disk Size: + {Utils.memoryBytesToString(rddInfo.diskSize)} +
    • +
    +
    +
    +
    +
    +
    +

    RDD Summary

    +
    + + + + + + + + + + + {storageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1).map { case (k,v) => + + + + + + + } + } + +
    Block NameStorage LevelSize in MemorySize on Disk
    {k} + {v.storageLevel.description} + {Utils.memoryBytesToString(v.memSize)}{Utils.memoryBytesToString(v.diskSize)}
    +
    +
    +
    +
    +
    +

    Worker Summary

    +
    + + + + + + + + + + {for(status <- storageStatusList) { + + + + + + }} + +
    HostMemory UsageDisk Usage
    {status.blockManagerId.host + ":" + status.blockManagerId.port} + {Utils.memoryBytesToString(status.memUsed(prefix))} + ({Utils.memoryBytesToString(status.memRemaining)} Total Available) + {Utils.memoryBytesToString(status.diskUsed(prefix))}
    +
    +
    ; + + WebUI.makePage(content, "RDD Info: " + id) + } + + def indexPage: Seq[Node] = { val storageStatusList = sc.getExecutorStorageStatus // Calculate macro-level statistics val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) @@ -77,42 +178,36 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, -
    - } - - /* - val handler = { - get { - path("") { - completeWith { - // Request the current storage status from the Master - val storageStatusList = sc.getExecutorStorageStatus - // Calculate macro-level statistics - val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) - .reduceOption(_+_).getOrElse(0L) - val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) - spark.storage.html.index. - render(maxMem, remainingMem, diskSpaceUsed, rdds, storageStatusList) - } - } ~ - path("rdd") { - parameter("id") { id => - completeWith { - val prefix = "rdd_" + id.toString - val storageStatusList = sc.getExecutorStorageStatus - val filteredStorageStatusList = StorageUtils. - filterStorageStatusByPrefix(storageStatusList, prefix) - val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head - spark.storage.html.rdd.render(rddInfo, filteredStorageStatusList) +
    + + + + + + + + + + + + + {for (rdd <- rdds) yield + + + + + + + + } - } - } ~ - pathPrefix("static") { - getFromResourceDirectory(STATIC_RESOURCE_DIR) - } - } - */ + +
    RDD NameStorage LevelCached PartitionsFraction Partitions CachedSize in MemorySize on Disk
    + + {rdd.name} + + {rdd.storageLevel.description} + {rdd.numCachedPartitions}{rdd.numCachedPartitions / rdd.numPartitions.toDouble}{Utils.memoryBytesToString(rdd.memSize)}{Utils.memoryBytesToString(rdd.diskSize)}
    + } private[spark] def appUIAddress = "http://" + host + ":" + port } diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index 12581f535f..bd2d637ae7 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -63,27 +63,4 @@ private[spark] object AkkaUtils { val boundPort = provider.asInstanceOf[RemoteActorRefProvider].transport.address.port.get return (actorSystem, boundPort) } - - /** - * Creates a Spray HTTP server bound to a given IP and port with a given Spray Route object to - * handle requests. Returns the bound port or throws a SparkException on failure. - * TODO: Not changing ip to host here - is it required ? - */ - def startJettyServer(ip: String, port: Int, handlers: Array[(String, Handler)]) = { - val handlersToRegister = handlers.map { case(path, handler) => - if (path == "*") { - handler - } else { - val contextHandler = new ContextHandler(path) - contextHandler.setHandler(handler) - contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] - } - } - - val handlerList = new HandlerList - handlerList.setHandlers(handlersToRegister) - val server = new Server(port) - server.setHandler(handlerList) - server.start() - } } -- cgit v1.2.3 From 7cd70dc2c19a5c95a214786f9b0712a42de6cba5 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 17 Jun 2013 13:58:35 -0700 Subject: Minor cleanup --- .../main/scala/spark/storage/BlockManagerUI.scala | 112 ++++++++++----------- 1 file changed, 55 insertions(+), 57 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index e577d1b597..9e1220c92a 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -49,16 +49,10 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, } } - val staticHandler = new ResourceHandler - println("RESOURCE: ") - val resource = getClass.getClassLoader.getResource(STATIC_RESOURCE_DIR) - staticHandler.setResourceBase(resource.toString) - - val handlers = Array[(String, Handler)]( - ("/static", staticHandler), + ("/static", createStaticHandler(STATIC_RESOURCE_DIR)), ("/rdd", (request: HttpServletRequest) => rddPage(request)), - ("*", (request: HttpServletRequest) => WebUI.makePage(indexPage, "Spark Storage")) + ("*", (request: HttpServletRequest) => indexPage) ) def rddPage(request: HttpServletRequest): Seq[Node] = { @@ -112,16 +106,17 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, - {storageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1).map { case (k,v) => - - {k} - - {v.storageLevel.description} - - {Utils.memoryBytesToString(v.memSize)} - {Utils.memoryBytesToString(v.diskSize)} - - } + {filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1).map { + case (k,v) => + + {k} + + {v.storageLevel.description} + + {Utils.memoryBytesToString(v.memSize)} + {Utils.memoryBytesToString(v.diskSize)} + + } } @@ -168,46 +163,49 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, .reduceOption(_+_).getOrElse(0L) val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) -
    -
    -
      -
    • Memory: - {Utils.memoryBytesToString(maxMem - remainingMem)} Used - ({Utils.memoryBytesToString(remainingMem)} Available)
    • -
    • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
    • -
    + val content = +
    +
    +
      +
    • Memory: + {Utils.memoryBytesToString(maxMem - remainingMem)} Used + ({Utils.memoryBytesToString(remainingMem)} Available)
    • +
    • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
    • +
    +
    -
    -
    - - - - - - - - - - - - - {for (rdd <- rdds) yield - - - - - - - - - } - -
    RDD NameStorage LevelCached PartitionsFraction Partitions CachedSize in MemorySize on Disk
    - - {rdd.name} - - {rdd.storageLevel.description} - {rdd.numCachedPartitions}{rdd.numCachedPartitions / rdd.numPartitions.toDouble}{Utils.memoryBytesToString(rdd.memSize)}{Utils.memoryBytesToString(rdd.diskSize)}
    +
    + + + + + + + + + + + + + {for (rdd <- rdds) yield + + + + + + + + + } + +
    RDD NameStorage LevelCached PartitionsFraction Partitions CachedSize in MemorySize on Disk
    + + {rdd.name} + + {rdd.storageLevel.description} + {rdd.numCachedPartitions}{rdd.numCachedPartitions / rdd.numPartitions.toDouble}{Utils.memoryBytesToString(rdd.memSize)}{Utils.memoryBytesToString(rdd.diskSize)}
    ; + + WebUI.makePage(content, "Spark Storage") } private[spark] def appUIAddress = "http://" + host + ":" + port } -- cgit v1.2.3 From 950f83535a8393a131f97a1a64454a446c5ea31c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 17 Jun 2013 14:27:11 -0700 Subject: Adding deterministic port --- .../main/scala/spark/storage/BlockManagerUI.scala | 22 ++++++++++------------ 1 file changed, 10 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index 9e1220c92a..cea5c0907e 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -25,23 +25,16 @@ private[spark] class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, sc: SparkContext) extends Directives with Logging { - val STATIC_RESOURCE_DIR = "spark/deploy/static" - implicit val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") val host = Utils.localHostName() - val port = if (System.getProperty("spark.ui.port") != null) { - System.getProperty("spark.ui.port").toInt - } else { - // TODO: Unfortunately, it's not possible to pass port 0 to spray and figure out which - // random port it bound to, so we have to try to find a local one by creating a socket. - Utils.findFreePort() - } + val port = Option(System.getProperty("spark.ui.port")) + .getOrElse(BlockManagerUI.DEFAULT_PORT).toInt /** Start a HTTP server to run the Web interface */ def start() { try { - WebUI.startJettyServer("0.0.0.0", port, handlers) - logInfo("Started BlockManager web UI at http://%s:%d".format(host, port)) + val (server, boundPort) = WebUI.startJettyServer("0.0.0.0", port, handlers) + logInfo("Started BlockManager web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => logError("Failed to create BlockManager WebUI", e) @@ -50,7 +43,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, } val handlers = Array[(String, Handler)]( - ("/static", createStaticHandler(STATIC_RESOURCE_DIR)), + ("/static", createStaticHandler(BlockManagerUI.STATIC_RESOURCE_DIR)), ("/rdd", (request: HttpServletRequest) => rddPage(request)), ("*", (request: HttpServletRequest) => indexPage) ) @@ -209,3 +202,8 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, } private[spark] def appUIAddress = "http://" + host + ":" + port } + +object BlockManagerUI { + val STATIC_RESOURCE_DIR = "spark/deploy/static" + val DEFAULT_PORT = "33000" +} -- cgit v1.2.3 From 7e6977b6c5f7c889b189285a7fa765db5ff2a285 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 17 Jun 2013 14:59:57 -0700 Subject: Fix in storage status page --- .../main/scala/spark/storage/BlockManagerUI.scala | 22 ++++++++++++---------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index cea5c0907e..6ac4398de7 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -129,16 +129,18 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, - {for(status <- storageStatusList) { - - {status.blockManagerId.host + ":" + status.blockManagerId.port} - - {Utils.memoryBytesToString(status.memUsed(prefix))} - ({Utils.memoryBytesToString(status.memRemaining)} Total Available) - - {Utils.memoryBytesToString(status.diskUsed(prefix))} - - }} + {filteredStorageStatusList.map { + status => + + {status.blockManagerId.host + ":" + status.blockManagerId.port} + + {Utils.memoryBytesToString(status.memUsed(prefix))} + ({Utils.memoryBytesToString(status.memRemaining)} Total Available) + + {Utils.memoryBytesToString(status.diskUsed(prefix))} + + } + }
    -- cgit v1.2.3 From 2c36a514aac2c15822c67f536c9f50450c8d5761 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 17 Jun 2013 21:44:26 -0700 Subject: Spray refactoring for master web UI --- .../main/scala/spark/deploy/master/Master.scala | 12 +- .../scala/spark/deploy/master/MasterWebUI.scala | 205 ++++++++++++++++++++- 2 files changed, 201 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index edaacd87b8..6a7bbdfcbf 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -56,16 +56,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } def startWebUi() { - val webUi = new MasterWebUI(context.system, self) - /* - try { - AkkaUtils.startSprayServer(context.system, "0.0.0.0", webUiPort, webUi.handler) - } catch { - case e: Exception => - logError("Failed to create web UI", e) - System.exit(1) - } - */ + val webUi = new MasterWebUI(self) + webUi.start() } override def receive = { diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index a4e21c8130..6393f2244b 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -10,20 +10,207 @@ import cc.spray.directives._ import cc.spray.typeconversion.TwirlSupport._ import cc.spray.http.MediaTypes import cc.spray.typeconversion.SprayJsonSupport._ +import scala.xml.Node +import spark.{Logging, Utils} +import spark.util.{WebUI => UtilsWebUI} import spark.deploy._ import spark.deploy.JsonProtocol._ +import concurrent.Future +import org.eclipse.jetty.server.Handler +import spark.util.WebUI._ +import spark.deploy.MasterState +import javax.servlet.http.HttpServletRequest +import java.util.concurrent.TimeUnit /** * Web UI server for the standalone master. */ private[spark] -class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Directives { - val RESOURCE_DIR = "spark/deploy/master/webui" - val STATIC_RESOURCE_DIR = "spark/deploy/static" - - implicit val timeout = Timeout(Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")) - +class MasterWebUI(master: ActorRef) extends Logging { + + implicit val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + val host = Utils.localHostName() + val port = Option(System.getProperty("spark.ui.port")) + .getOrElse(MasterWebUI.DEFAULT_PORT).toInt + + def start() { + try { + val (server, boundPort) = UtilsWebUI.startJettyServer("0.0.0.0", port, handlers) + logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) + } catch { + case e: Exception => + logError("Failed to create Master WebUI", e) + System.exit(1) + } + } + + val handlers = Array[(String, Handler)]( + ("/static", createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR)), + ("*", (request: HttpServletRequest) => index) + ) + + def appDetail(request: HttpServletRequest): Seq[Node] = { + val appId = request.getParameter("appId") + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val state = Await.result(stateFuture, 3 seconds) + val app = state.activeApps.find(_.id == appId).getOrElse({ + state.completedApps.find(_.id == appId).getOrElse(null) + }) + val content = +
    +
    +
      +
    • ID: app.id
    • +
    • Description: app.desc.name
    • +
    • User: app.desc.user
    • +
    • Cores: + {if(app.desc.maxCores == Integer.MAX_VALUE) { + "Unlimited %s granted".format(app.coresGranted) + } else { + "%s (%s granted, %s left)".format( + app.desc.maxCores, + app.coresGranted, + app.coresLeft) } + } + } +
    • +
    • Memory per Slave: @app.desc.memoryPerSlave
    • +
    • Submit Date: @app.submitDate
    • +
    • State: @app.state
    • +
    • Application Detail UI
    • +
    +
    +
    + +
    + + +
    +
    +

    Executor Summary

    +
    + @executors_table(app.executors.values.toList) +
    +
    + + } + + def index: Seq[Node] = { + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val state = Await.result(stateFuture, 3 seconds) + + val content = +
    +
    +
      +
    • URL:{state.uri}
    • +
    • Workers:{state.workers.size}
    • +
    • Cores: {state.workers.map(_.cores).sum}Total, + {state.workers.map(_.coresUsed).sum} Used
    • +
    • Memory: {Utils.memoryMegabytesToString(state.workers.map(_.memory).sum)} Total, + {Utils.memoryMegabytesToString(state.workers.map(_.memoryUsed).sum)} Used
    • +
    • Applications: {state.activeApps.size} Running, {state.completedApps.size} Completed
    • +
    +
    +
    + +
    +
    +

    Workers

    +
    + {workerTable(state.workers.sortBy(_.id))} +
    +
    + +
    + +
    +
    +

    Running Applications

    +
    + {appTable(state.activeApps.sortBy(_.startTime).reverse)} +
    +
    + +
    + +
    +
    +

    Completed Applications

    +
    + {appTable(state.completedApps.sortBy(_.endTime).reverse)} +
    +
    ; + UtilsWebUI.makePage(content, "Spark Master: " + state.uri) + } + + def workerTable(workers: Seq[spark.deploy.master.WorkerInfo]) = { + + + + + + + + + + + + {workers.map{ worker => + + + + + + + + } + } + +
    IDAddressStateCoresMemory
    + {worker.id} + {worker.host}:{worker.port}{worker.state}{worker.cores} ({worker.coresUsed} Used){Utils.memoryMegabytesToString(worker.memory)} + ({Utils.memoryMegabytesToString(worker.memoryUsed)} Used)
    + } + + def appTable(apps: Seq[spark.deploy.master.ApplicationInfo]) = { + + + + + + + + + + + + + + + {apps.map{app => + + + + + + + + + + + } + } + +
    IDDescriptionCoresMemory per NodeSubmit TimeUserStateDuration
    + {app.id} + {app.desc.name} + {app.coresGranted} + {Utils.memoryMegabytesToString(app.desc.memoryPerSlave)}{WebUI.formatDate(app.submitDate)}{app.desc.user}{app.state.toString}{WebUI.formatDuration(app.duration)}
    + } + + /* val handler = { get { (path("") & parameters('format ?)) { @@ -71,4 +258,10 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct getFromResourceDirectory(RESOURCE_DIR) } } + */ } + +object MasterWebUI { + val STATIC_RESOURCE_DIR = "spark/deploy/static" + val DEFAULT_PORT = "34000" +} \ No newline at end of file -- cgit v1.2.3 From f85fd7a793e6b4c023075f962ffcb0893d622fc3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 17 Jun 2013 21:58:17 -0700 Subject: Commenting unfinished part --- core/src/main/scala/spark/deploy/master/MasterWebUI.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 6393f2244b..c76b5b1e0e 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -78,7 +78,7 @@ class MasterWebUI(master: ActorRef) extends Logging {
  • Memory per Slave: @app.desc.memoryPerSlave
  • Submit Date: @app.submitDate
  • State: @app.state
  • -
  • Application Detail UI
  • +
  • Application Detail UI
  • @@ -90,7 +90,7 @@ class MasterWebUI(master: ActorRef) extends Logging {

    Executor Summary


    - @executors_table(app.executors.values.toList) + @executors_table(app.executors.values.toList)
    -- cgit v1.2.3 From e55cf0245fa7d352f948312bb81cf2c09fe63d6f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 17 Jun 2013 21:59:25 -0700 Subject: Adding WebUI file --- core/src/main/scala/spark/util/WebUI.scala | 104 +++++++++++++++++++++++++++++ 1 file changed, 104 insertions(+) create mode 100644 core/src/main/scala/spark/util/WebUI.scala diff --git a/core/src/main/scala/spark/util/WebUI.scala b/core/src/main/scala/spark/util/WebUI.scala new file mode 100644 index 0000000000..eaacd95691 --- /dev/null +++ b/core/src/main/scala/spark/util/WebUI.scala @@ -0,0 +1,104 @@ +package spark.util + +import xml.Elem +import xml.Node +import util.parsing.json.{JSONFormat, JSONObject} +import org.eclipse.jetty.server.{Server, Request, Handler} +import javax.servlet.http.{HttpServletResponse, HttpServletRequest} +import org.eclipse.jetty.util.component.LifeCycle.Listener +import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler} +import util.Try +import util.Success +import util.Failure +import spark.Logging +import annotation.tailrec + +object WebUI extends Logging { + type Responder[T] = HttpServletRequest => T + + implicit def jsonResponderToHandler(responder: Responder[JSONObject]): Handler = { + createHandler(responder, "text/json") + } + + implicit def htmlResponderToHandler(responder: Responder[Seq[Node]]): Handler = { + createHandler(responder, "text/html") + } + + def createHandler[T <% AnyRef](responder: Responder[T], contentType: String): Handler = { + new AbstractHandler { + def handle(target: String, + baseRequest: Request, + request: HttpServletRequest, + response: HttpServletResponse) { + response.setContentType("%s;charset=utf-8".format(contentType)) + response.setStatus(HttpServletResponse.SC_OK) + baseRequest.setHandled(true) + response.getWriter().println( + responder(request).toString + ) + } + } + } + + def createStaticHandler(resourceBase: String): ResourceHandler = { + val staticHandler = new ResourceHandler + val resource = getClass.getClassLoader.getResource(resourceBase) + staticHandler.setResourceBase(resource.toString) + staticHandler + } + + def startJettyServer(ip: String, port: Int, handlers: Array[(String, Handler)]): (Server, Int) = { + val handlersToRegister = handlers.map { case(path, handler) => + if (path == "*") { + handler + } else { + val contextHandler = new ContextHandler(path) + contextHandler.setHandler(handler) + contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] + } + } + + val handlerList = new HandlerList + handlerList.setHandlers(handlersToRegister) + + @tailrec + def connect(currentPort: Int): (Server, Int) = { + val server = new Server(port) + server.setHandler(handlerList) + Try { server.start() } match { + case s: Success[_] => (server, currentPort) + case f: Failure[_] => + logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) + connect((currentPort + 1) % 65536) + } + } + + connect(port) + } + + def makePage(content: => Seq[Node], title: String): Seq[Node] = { + + + + + + + {title} + + +
    +
    +
    + +

    + {title} +

    +
    +
    + {content} +
    + + + } +} -- cgit v1.2.3 From 9161db5478c3e8c1c124070bacdf571b744cee22 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 09:50:24 -0700 Subject: Cleaning up master web UI --- .../scala/spark/deploy/master/MasterWebUI.scala | 201 ++++++++++----------- 1 file changed, 93 insertions(+), 108 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index c76b5b1e0e..0b7d163980 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -1,27 +1,19 @@ package spark.deploy.master -import akka.actor.{ActorRef, ActorSystem} +import akka.actor.{ActorRef} import akka.dispatch.Await import akka.pattern.ask -import akka.util.{Duration, Timeout} +import akka.util.{Duration} import akka.util.duration._ -import cc.spray.Directives -import cc.spray.directives._ -import cc.spray.typeconversion.TwirlSupport._ -import cc.spray.http.MediaTypes -import cc.spray.typeconversion.SprayJsonSupport._ import scala.xml.Node import spark.{Logging, Utils} import spark.util.{WebUI => UtilsWebUI} import spark.deploy._ -import spark.deploy.JsonProtocol._ -import concurrent.Future import org.eclipse.jetty.server.Handler import spark.util.WebUI._ import spark.deploy.MasterState import javax.servlet.http.HttpServletRequest -import java.util.concurrent.TimeUnit /** * Web UI server for the standalone master. @@ -47,9 +39,11 @@ class MasterWebUI(master: ActorRef) extends Logging { val handlers = Array[(String, Handler)]( ("/static", createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR)), + ("/app", (request: HttpServletRequest) => appDetail(request)), ("*", (request: HttpServletRequest) => index) ) + /** Executor details for a particular application */ def appDetail(request: HttpServletRequest): Seq[Node] = { val appId = request.getParameter("appId") val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] @@ -65,37 +59,73 @@ class MasterWebUI(master: ActorRef) extends Logging {
  • Description: app.desc.name
  • User: app.desc.user
  • Cores: - {if(app.desc.maxCores == Integer.MAX_VALUE) { - "Unlimited %s granted".format(app.coresGranted) - } else { - "%s (%s granted, %s left)".format( - app.desc.maxCores, - app.coresGranted, - app.coresLeft) } - } - } -
  • -
  • Memory per Slave: @app.desc.memoryPerSlave
  • -
  • Submit Date: @app.submitDate
  • -
  • State: @app.state
  • -
  • Application Detail UI
  • - - - + { + if (app.desc.maxCores == Integer.MAX_VALUE) { + "Unlimited %s granted".format(app.coresGranted) + } else { + "%s (%s granted, %s left)".format( + app.desc.maxCores, app.coresGranted, app.coresLeft) + } + } + +
  • Memory per Slave: {app.desc.memoryPerSlave}
  • +
  • Submit Date: {app.submitDate}
  • +
  • State: {app.state}
  • +
  • Application Detail UI
  • + + +
    - +

    Executor Summary


    - @executors_table(app.executors.values.toList) + {executorsTable(app.executors.values.toList)}
    + UtilsWebUI.makePage(content, "Application Info: " + app.desc.name) + } + + def executorsTable(executors: Seq[ExecutorInfo]): Seq[Node] = { + + + + + + + + + + + + + {executors.map(executorRow)} + +
    ExecutorIDWorkerCoresMemoryStateLogs
    + } + def executorRow(executor: ExecutorInfo): Seq[Node] = { + + {executor.id} + + {executor.worker.id} + + {executor.cores} + {executor.memory} + {executor.state} + + stdout + stdout + + } + /** Index view listing applications and executors */ def index: Seq[Node] = { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] val state = Await.result(stateFuture, 3 seconds) @@ -108,9 +138,12 @@ class MasterWebUI(master: ActorRef) extends Logging {
  • Workers:{state.workers.size}
  • Cores: {state.workers.map(_.cores).sum}Total, {state.workers.map(_.coresUsed).sum} Used
  • -
  • Memory: {Utils.memoryMegabytesToString(state.workers.map(_.memory).sum)} Total, +
  • Memory: + {Utils.memoryMegabytesToString(state.workers.map(_.memory).sum)} Total, {Utils.memoryMegabytesToString(state.workers.map(_.memoryUsed).sum)} Used
  • -
  • Applications: {state.activeApps.size} Running, {state.completedApps.size} Completed
  • +
  • Applications: + {state.activeApps.size} Running, + {state.completedApps.size} Completed
  • @@ -157,17 +190,18 @@ class MasterWebUI(master: ActorRef) extends Logging { - {workers.map{ worker => - - - {worker.id} - - {worker.host}:{worker.port} - {worker.state} - {worker.cores} ({worker.coresUsed} Used) - {Utils.memoryMegabytesToString(worker.memory)} - ({Utils.memoryMegabytesToString(worker.memoryUsed)} Used) - + { + workers.map{ worker => + + + {worker.id} + + {worker.host}:{worker.port} + {worker.state} + {worker.cores} ({worker.coresUsed} Used) + {Utils.memoryMegabytesToString(worker.memory)} + ({Utils.memoryMegabytesToString(worker.memoryUsed)} Used) + } } @@ -189,76 +223,27 @@ class MasterWebUI(master: ActorRef) extends Logging { - {apps.map{app => - - - {app.id} - - {app.desc.name} - - {app.coresGranted} - - {Utils.memoryMegabytesToString(app.desc.memoryPerSlave)} - {WebUI.formatDate(app.submitDate)} - {app.desc.user} - {app.state.toString} - {WebUI.formatDuration(app.duration)} - + { + apps.map{ app => + + + {app.id} + + {app.desc.name} + + {app.coresGranted} + + {Utils.memoryMegabytesToString(app.desc.memoryPerSlave)} + {WebUI.formatDate(app.submitDate)} + {app.desc.user} + {app.state.toString} + {WebUI.formatDuration(app.duration)} + + } } - } } - - /* - val handler = { - get { - (path("") & parameters('format ?)) { - case Some(js) if js.equalsIgnoreCase("json") => - val future = master ? RequestMasterState - respondWithMediaType(MediaTypes.`application/json`) { ctx => - ctx.complete(future.mapTo[MasterState]) - } - case _ => - completeWith { - val future = master ? RequestMasterState - future.map { - masterState => spark.deploy.master.html.index.render(masterState.asInstanceOf[MasterState]) - } - } - } ~ - path("app") { - parameters("appId", 'format ?) { - case (appId, Some(js)) if (js.equalsIgnoreCase("json")) => - val future = master ? RequestMasterState - val appInfo = for (masterState <- future.mapTo[MasterState]) yield { - masterState.activeApps.find(_.id == appId).getOrElse({ - masterState.completedApps.find(_.id == appId).getOrElse(null) - }) - } - respondWithMediaType(MediaTypes.`application/json`) { ctx => - ctx.complete(appInfo.mapTo[ApplicationInfo]) - } - case (appId, _) => - completeWith { - val future = master ? RequestMasterState - future.map { state => - val masterState = state.asInstanceOf[MasterState] - val app = masterState.activeApps.find(_.id == appId).getOrElse({ - masterState.completedApps.find(_.id == appId).getOrElse(null) - }) - spark.deploy.master.html.app_details.render(app) - } - } - } - } ~ - pathPrefix("static") { - getFromResourceDirectory(STATIC_RESOURCE_DIR) - } ~ - getFromResourceDirectory(RESOURCE_DIR) - } - } - */ } object MasterWebUI { -- cgit v1.2.3 From ee73c09ac965076c9239587721ffa76741b99483 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 10:10:48 -0700 Subject: Some comments --- core/src/main/scala/spark/deploy/master/MasterWebUI.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 0b7d163980..34f50fd5e4 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -78,14 +78,13 @@ class MasterWebUI(master: ActorRef) extends Logging {
    - -
    +

    Executor Summary


    {executorsTable(app.executors.values.toList)}
    -
    +
    ; UtilsWebUI.makePage(content, "Application Info: " + app.desc.name) } -- cgit v1.2.3 From fc94576ece99f2b224a951b32f0f6360701b7cd3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 13:11:57 -0700 Subject: Adding worker version of UI --- .../scala/spark/deploy/master/MasterWebUI.scala | 16 +-- .../main/scala/spark/deploy/worker/Worker.scala | 12 +- .../scala/spark/deploy/worker/WorkerWebUI.scala | 135 +++++++++++++++++++-- core/src/main/scala/spark/util/WebUI.scala | 31 +++-- 4 files changed, 162 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 34f50fd5e4..a2e9dfd762 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -20,10 +20,10 @@ import javax.servlet.http.HttpServletRequest */ private[spark] class MasterWebUI(master: ActorRef) extends Logging { - - implicit val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + implicit val timeout = Duration.create( + System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") val host = Utils.localHostName() - val port = Option(System.getProperty("spark.ui.port")) + val port = Option(System.getProperty("master.ui.port")) .getOrElse(MasterWebUI.DEFAULT_PORT).toInt def start() { @@ -82,13 +82,13 @@ class MasterWebUI(master: ActorRef) extends Logging {

    Executor Summary


    - {executorsTable(app.executors.values.toList)} + {executorTable(app.executors.values.toList)}
    ; UtilsWebUI.makePage(content, "Application Info: " + app.desc.name) } - def executorsTable(executors: Seq[ExecutorInfo]): Seq[Node] = { + def executorTable(executors: Seq[ExecutorInfo]): Seq[Node] = { @@ -119,7 +119,7 @@ class MasterWebUI(master: ActorRef) extends Logging { stdoutstdout + .format(executor.worker.webUiAddress, executor.application.id, executor.id)}>stderr } @@ -135,7 +135,7 @@ class MasterWebUI(master: ActorRef) extends Logging {
    • URL:{state.uri}
    • Workers:{state.workers.size}
    • -
    • Cores: {state.workers.map(_.cores).sum}Total, +
    • Cores: {state.workers.map(_.cores).sum} Total, {state.workers.map(_.coresUsed).sum} Used
    • Memory: {Utils.memoryMegabytesToString(state.workers.map(_.memory).sum)} Total, @@ -247,5 +247,5 @@ class MasterWebUI(master: ActorRef) extends Logging { object MasterWebUI { val STATIC_RESOURCE_DIR = "spark/deploy/static" - val DEFAULT_PORT = "34000" + val DEFAULT_PORT = "8080" } \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index f8fdab927e..3878fe3f7b 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -88,16 +88,8 @@ private[spark] class Worker( } def startWebUi() { - val webUi = new WorkerWebUI(context.system, self, workDir) - /* - try { - AkkaUtils.startSprayServer(context.system, "0.0.0.0", webUiPort, webUi.handler) - } catch { - case e: Exception => - logError("Failed to create web UI", e) - System.exit(1) - } - */ + val webUi = new WorkerWebUI(self, workDir) + webUi.start() } override def receive = { diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index 3235c50d1b..b8b4b89738 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -8,22 +8,137 @@ import akka.util.duration._ import cc.spray.Directives import cc.spray.typeconversion.TwirlSupport._ import cc.spray.http.MediaTypes -import cc.spray.typeconversion.SprayJsonSupport._ import spark.deploy.{WorkerState, RequestWorkerState} import spark.deploy.JsonProtocol._ import java.io.File +import spark.util.{WebUI => UtilsWebUI} +import spark.{Utils, Logging} +import org.eclipse.jetty.server.Handler +import spark.util.WebUI._ +import spark.deploy.WorkerState +import scala.io.Source +import javax.servlet.http.HttpServletRequest +import xml.Node /** * Web UI server for the standalone worker. */ private[spark] -class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef, workDir: File) extends Directives { - val RESOURCE_DIR = "spark/deploy/worker/webui" - val STATIC_RESOURCE_DIR = "spark/deploy/static" - - implicit val timeout = Timeout(Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")) - +class WorkerWebUI(worker: ActorRef, workDir: File) extends Logging { + implicit val timeout = Timeout( + Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")) + val host = Utils.localHostName() + val port = Option(System.getProperty("wroker.ui.port")) + .getOrElse(WorkerWebUI.DEFAULT_PORT).toInt + + val handlers = Array[(String, Handler)]( + ("/static", createStaticHandler(WorkerWebUI.STATIC_RESOURCE_DIR)), + ("/log", (request: HttpServletRequest) => log(request)), + ("*", (request: HttpServletRequest) => index) + ) + + def start() { + try { + val (server, boundPort) = UtilsWebUI.startJettyServer("0.0.0.0", port, handlers) + logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) + } catch { + case e: Exception => + logError("Failed to create Worker WebUI", e) + System.exit(1) + } + } + + def index(): Seq[Node] = { + val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] + val workerState = Await.result(stateFuture, 3 seconds) + val content = +
      +
      +
        +
      • ID: {workerState.workerId}
      • +
      • + Master URL: {workerState.masterUrl} +
      • +
      • Cores: {workerState.cores} ({workerState.coresUsed} Used)
      • +
      • Memory: {Utils.memoryMegabytesToString(workerState.memory)} + ({Utils.memoryMegabytesToString(workerState.memoryUsed)} Used)
      • +
      +

      Back to Master

      +
      +
      +
      + +
      +
      +

      Running Executors {workerState.executors.size}

      +
      + {executorTable(workerState.executors)} +
      +
      +
      + +
      +
      +

      Finished Executors

      +
      + {executorTable(workerState.finishedExecutors)} +
      +
      ; + + UtilsWebUI.makePage(content, "Spark Worker on %s:%s".format(workerState.host, workerState.port)) + } + + def executorTable(executors: Seq[ExecutorRunner]): Seq[Node] = { +
    + + + + + + + + + + + {executors.map(executorRow)} + +
    ExecutorIDCoresMemoryJob DetailsLogs
    + } + + def executorRow(executor: ExecutorRunner): Seq[Node] = { + + {executor.execId} + {executor.cores} + {Utils.memoryMegabytesToString(executor.memory)} + +
      +
    • ID: {executor.appId}
    • +
    • Name: {executor.appDesc.name}
    • +
    • User: {executor.appDesc.user}
    • +
    + + + stdout + stderr + + + } + + def log(request: HttpServletRequest): String = { + val appId = request.getParameter("appId") + val executorId = request.getParameter("executorId") + val logType = request.getParameter("logType") + val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) + val source = Source.fromFile(path) + val lines = source.mkString + source.close() + lines + } + + /* val handler = { get { (path("") & parameters('format ?)) { @@ -54,4 +169,10 @@ class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef, workDir: File) getFromResourceDirectory(RESOURCE_DIR) } } + */ +} + +object WorkerWebUI { + val STATIC_RESOURCE_DIR = "spark/deploy/static" + val DEFAULT_PORT="8081" } diff --git a/core/src/main/scala/spark/util/WebUI.scala b/core/src/main/scala/spark/util/WebUI.scala index eaacd95691..34b776f1d8 100644 --- a/core/src/main/scala/spark/util/WebUI.scala +++ b/core/src/main/scala/spark/util/WebUI.scala @@ -16,13 +16,14 @@ import annotation.tailrec object WebUI extends Logging { type Responder[T] = HttpServletRequest => T - implicit def jsonResponderToHandler(responder: Responder[JSONObject]): Handler = { + implicit def jsonResponderToHandler(responder: Responder[JSONObject]): Handler = createHandler(responder, "text/json") - } - implicit def htmlResponderToHandler(responder: Responder[Seq[Node]]): Handler = { + implicit def htmlResponderToHandler(responder: Responder[Seq[Node]]): Handler = createHandler(responder, "text/html") - } + + implicit def textResponderToHandler(responder: Responder[String]): Handler = + createHandler(responder, "text/plain") def createHandler[T <% AnyRef](responder: Responder[T], contentType: String): Handler = { new AbstractHandler { @@ -40,12 +41,28 @@ object WebUI extends Logging { } } + /** Create and return a staticHandler if resourceBase can be located */ def createStaticHandler(resourceBase: String): ResourceHandler = { val staticHandler = new ResourceHandler - val resource = getClass.getClassLoader.getResource(resourceBase) - staticHandler.setResourceBase(resource.toString) - staticHandler + Option(getClass.getClassLoader.getResource(resourceBase)) match { + case Some(res) => + staticHandler.setResourceBase (res.toString) + staticHandler + } + } + + /* + /** Create and return a staticHandler if resourceBase can be located */ + def createStaticHandler(resourceBase: String): Option[ResourceHandler] = { + val staticHandler = new ResourceHandler + Option(getClass.getClassLoader.getResource(resourceBase)) match { + case Some(res) => + staticHandler.setResourceBase (res.toString) + Some(staticHandler) + case None => None + } } + */ def startJettyServer(ip: String, port: Int, handlers: Array[(String, Handler)]): (Server, Int) = { val handlersToRegister = handlers.map { case(path, handler) => -- cgit v1.2.3 From 91ec5a1a04339983d57a72d8df8f1d769d8d855a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 15:13:12 -0700 Subject: Changing JSON protocol and removing spray code --- .../src/main/scala/spark/deploy/JsonProtocol.scala | 114 +++++++++------------ .../scala/spark/deploy/master/MasterWebUI.scala | 6 +- .../scala/spark/deploy/worker/WorkerWebUI.scala | 6 +- .../main/scala/spark/storage/BlockManagerUI.scala | 11 +- core/src/main/scala/spark/util/AkkaUtils.scala | 15 +-- core/src/main/scala/spark/util/WebUI.scala | 12 +-- project/SparkBuild.scala | 7 +- project/plugins.sbt | 2 - 8 files changed, 66 insertions(+), 107 deletions(-) diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index ea832101d2..b4365d31e9 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -1,79 +1,65 @@ package spark.deploy import master.{ApplicationInfo, WorkerInfo} +import net.liftweb.json.JsonDSL._ import worker.ExecutorRunner -import cc.spray.json._ -/** - * spray-json helper class containing implicit conversion to json for marshalling responses - */ -private[spark] object JsonProtocol extends DefaultJsonProtocol { - implicit object WorkerInfoJsonFormat extends RootJsonWriter[WorkerInfo] { - def write(obj: WorkerInfo) = JsObject( - "id" -> JsString(obj.id), - "host" -> JsString(obj.host), - "port" -> JsNumber(obj.port), - "webuiaddress" -> JsString(obj.webUiAddress), - "cores" -> JsNumber(obj.cores), - "coresused" -> JsNumber(obj.coresUsed), - "memory" -> JsNumber(obj.memory), - "memoryused" -> JsNumber(obj.memoryUsed) - ) - } +object JsonProtocol { + def writeWorkerInfo(obj: WorkerInfo) = { + ("id" -> obj.id) ~ + ("host" -> obj.host) ~ + ("port" -> obj.port) ~ + ("webuiaddress" -> obj.webUiAddress) ~ + ("cores" -> obj.cores) ~ + ("coresused" -> obj.coresUsed) ~ + ("memory" -> obj.memory) ~ + ("memoryused" -> obj.memoryUsed) + } - implicit object AppInfoJsonFormat extends RootJsonWriter[ApplicationInfo] { - def write(obj: ApplicationInfo) = JsObject( - "starttime" -> JsNumber(obj.startTime), - "id" -> JsString(obj.id), - "name" -> JsString(obj.desc.name), - "cores" -> JsNumber(obj.desc.maxCores), - "user" -> JsString(obj.desc.user), - "memoryperslave" -> JsNumber(obj.desc.memoryPerSlave), - "submitdate" -> JsString(obj.submitDate.toString)) + def writeApplicationInfo(obj: ApplicationInfo) = { + ("starttime" -> obj.startTime) ~ + ("id" -> obj.id) ~ + ("name" -> obj.desc.name) ~ + ("cores" -> obj.desc.maxCores) ~ + ("user" -> obj.desc.user) ~ + ("memoryperslave" -> obj.desc.memoryPerSlave) ~ + ("submitdate" -> obj.submitDate.toString) } - implicit object AppDescriptionJsonFormat extends RootJsonWriter[ApplicationDescription] { - def write(obj: ApplicationDescription) = JsObject( - "name" -> JsString(obj.name), - "cores" -> JsNumber(obj.maxCores), - "memoryperslave" -> JsNumber(obj.memoryPerSlave), - "user" -> JsString(obj.user) - ) + def writeApplicationDescription(obj: ApplicationDescription) = { + ("name" -> obj.name) ~ + ("cores" -> obj.maxCores) ~ + ("memoryperslave" -> obj.memoryPerSlave) ~ + ("user" -> obj.user) } - implicit object ExecutorRunnerJsonFormat extends RootJsonWriter[ExecutorRunner] { - def write(obj: ExecutorRunner) = JsObject( - "id" -> JsNumber(obj.execId), - "memory" -> JsNumber(obj.memory), - "appid" -> JsString(obj.appId), - "appdesc" -> obj.appDesc.toJson.asJsObject - ) + def writeExecutorRunner(obj: ExecutorRunner) = { + ("id" -> obj.execId) ~ + ("memory" -> obj.memory) ~ + ("appid" -> obj.appId) ~ + ("appdesc" -> writeApplicationDescription(obj.appDesc)) } - implicit object MasterStateJsonFormat extends RootJsonWriter[MasterState] { - def write(obj: MasterState) = JsObject( - "url" -> JsString("spark://" + obj.uri), - "workers" -> JsArray(obj.workers.toList.map(_.toJson)), - "cores" -> JsNumber(obj.workers.map(_.cores).sum), - "coresused" -> JsNumber(obj.workers.map(_.coresUsed).sum), - "memory" -> JsNumber(obj.workers.map(_.memory).sum), - "memoryused" -> JsNumber(obj.workers.map(_.memoryUsed).sum), - "activeapps" -> JsArray(obj.activeApps.toList.map(_.toJson)), - "completedapps" -> JsArray(obj.completedApps.toList.map(_.toJson)) - ) + def writeMasterState(obj: MasterState) = { + ("url" -> ("spark://" + obj.uri)) ~ + ("workers" -> obj.workers.toList.map(writeWorkerInfo)) ~ + ("cores" -> obj.workers.map(_.cores).sum) ~ + ("coresused" -> obj.workers.map(_.coresUsed).sum) ~ + ("memory" -> obj.workers.map(_.memory).sum) ~ + ("memoryused" -> obj.workers.map(_.memoryUsed).sum) ~ + ("activeapps" -> obj.activeApps.toList.map(writeApplicationInfo)) ~ + ("completedapps" -> obj.completedApps.toList.map(writeApplicationInfo)) } - implicit object WorkerStateJsonFormat extends RootJsonWriter[WorkerState] { - def write(obj: WorkerState) = JsObject( - "id" -> JsString(obj.workerId), - "masterurl" -> JsString(obj.masterUrl), - "masterwebuiurl" -> JsString(obj.masterWebUiUrl), - "cores" -> JsNumber(obj.cores), - "coresused" -> JsNumber(obj.coresUsed), - "memory" -> JsNumber(obj.memory), - "memoryused" -> JsNumber(obj.memoryUsed), - "executors" -> JsArray(obj.executors.toList.map(_.toJson)), - "finishedexecutors" -> JsArray(obj.finishedExecutors.toList.map(_.toJson)) - ) + def writeWorkerState(obj: WorkerState) = { + ("id" -> obj.workerId) ~ + ("masterurl" -> obj.masterUrl) ~ + ("masterwebuiurl" -> obj.masterWebUiUrl) ~ + ("cores" -> obj.cores) ~ + ("coresused" -> obj.coresUsed) ~ + ("memory" -> obj.memory) ~ + ("memoryused" -> obj.memoryUsed) ~ + ("executors" -> obj.executors.toList.map(writeExecutorRunner)) ~ + ("finishedexecutors" -> obj.finishedExecutors.toList.map(writeExecutorRunner)) } -} +} \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index a2e9dfd762..6623142d69 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -55,9 +55,9 @@ class MasterWebUI(master: ActorRef) extends Logging {
      -
    • ID: app.id
    • -
    • Description: app.desc.name
    • -
    • User: app.desc.user
    • +
    • ID: {app.id}
    • +
    • Description: {app.desc.name}
    • +
    • User: {app.desc.user}
    • Cores: { if (app.desc.maxCores == Integer.MAX_VALUE) { diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index b8b4b89738..0af9eb8efa 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -1,16 +1,12 @@ package spark.deploy.worker -import akka.actor.{ActorRef, ActorSystem} +import akka.actor.ActorRef import akka.dispatch.Await import akka.pattern.ask import akka.util.{Duration, Timeout} import akka.util.duration._ -import cc.spray.Directives -import cc.spray.typeconversion.TwirlSupport._ -import cc.spray.http.MediaTypes import spark.deploy.{WorkerState, RequestWorkerState} -import spark.deploy.JsonProtocol._ import java.io.File import spark.util.{WebUI => UtilsWebUI} import spark.{Utils, Logging} diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index 6ac4398de7..e9c362fce7 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -3,18 +3,12 @@ package spark.storage import akka.actor.{ActorRef, ActorSystem} import akka.util.Duration import akka.util.duration._ -import cc.spray.typeconversion.TwirlSupport._ -import cc.spray.Directives import spark.{Logging, SparkContext} -import spark.util.AkkaUtils import spark.Utils import spark.util.WebUI -import org.eclipse.jetty.server.handler.{HandlerList, ContextHandler, ResourceHandler} import org.eclipse.jetty.server.Handler -import javax.servlet.http.{HttpServletResponse, HttpServletRequest} -import xml.Elem +import javax.servlet.http.HttpServletRequest import xml.Node -import java.net.URLClassLoader import spark.util.WebUI._ @@ -23,7 +17,7 @@ import spark.util.WebUI._ */ private[spark] class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, sc: SparkContext) - extends Directives with Logging { + extends Logging { implicit val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") val host = Utils.localHostName() @@ -55,7 +49,6 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, val filteredStorageStatusList = StorageUtils. filterStorageStatusByPrefix(storageStatusList, prefix) val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head - spark.storage.html.rdd.render(rddInfo, filteredStorageStatusList) val content =
      diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index bd2d637ae7..134c912c46 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -1,21 +1,10 @@ package spark.util -import akka.actor.{ActorRef, Props, ActorSystemImpl, ActorSystem} +import akka.actor.{ActorSystemImpl, ActorSystem} import com.typesafe.config.ConfigFactory import akka.util.duration._ -import akka.pattern.ask import akka.remote.RemoteActorRefProvider -import cc.spray.Route -import cc.spray.io.IoWorker -import cc.spray.{SprayCanRootService, HttpService} -import cc.spray.can.server.HttpServer -import cc.spray.io.pipelines.MessageHandlerDispatch.SingletonHandler -import akka.dispatch.Await -import spark.{Utils, SparkException} -import java.util.concurrent.TimeoutException -import org.eclipse.jetty.server.Server -import org.eclipse.jetty.server.Handler -import org.eclipse.jetty.server.handler.{HandlerList, ContextHandler} + /** * Various utility classes for working with Akka. diff --git a/core/src/main/scala/spark/util/WebUI.scala b/core/src/main/scala/spark/util/WebUI.scala index 34b776f1d8..e6b39b15eb 100644 --- a/core/src/main/scala/spark/util/WebUI.scala +++ b/core/src/main/scala/spark/util/WebUI.scala @@ -1,23 +1,22 @@ package spark.util -import xml.Elem import xml.Node -import util.parsing.json.{JSONFormat, JSONObject} import org.eclipse.jetty.server.{Server, Request, Handler} import javax.servlet.http.{HttpServletResponse, HttpServletRequest} -import org.eclipse.jetty.util.component.LifeCycle.Listener import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler} import util.Try import util.Success import util.Failure import spark.Logging import annotation.tailrec +import net.liftweb.json.JsonAST.JValue +import net.liftweb.json._ object WebUI extends Logging { type Responder[T] = HttpServletRequest => T - implicit def jsonResponderToHandler(responder: Responder[JSONObject]): Handler = - createHandler(responder, "text/json") + implicit def jsonResponderToHandler(responder: Responder[JValue]): Handler = + createHandler(responder, "text/json", (in: JValue) => pretty(render(in))) implicit def htmlResponderToHandler(responder: Responder[Seq[Node]]): Handler = createHandler(responder, "text/html") @@ -25,7 +24,8 @@ object WebUI extends Logging { implicit def textResponderToHandler(responder: Responder[String]): Handler = createHandler(responder, "text/plain") - def createHandler[T <% AnyRef](responder: Responder[T], contentType: String): Handler = { + def createHandler[T <% AnyRef](responder: Responder[T], contentType: String, + extractFn: T => String = (in: Any) => in.toString): Handler = { new AbstractHandler { def handle(target: String, baseRequest: Request, diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index faf6e2ae8e..ec26b2a229 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -4,7 +4,6 @@ import sbt.Classpaths.publishTask import Keys._ import sbtassembly.Plugin._ import AssemblyKeys._ -import twirl.sbt.TwirlPlugin._ // For Sonatype publishing //import com.jsuereth.pgp.sbtplugin.PgpKeys._ @@ -157,9 +156,7 @@ object SparkBuild extends Build { "com.typesafe.akka" % "akka-slf4j" % "2.0.3" excludeAll(excludeNetty), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", - "cc.spray" % "spray-can" % "1.0-M2.1" excludeAll(excludeNetty), - "cc.spray" % "spray-server" % "1.0-M2.1" excludeAll(excludeNetty), - "cc.spray" % "spray-json_2.9.2" % "1.1.1" excludeAll(excludeNetty), + "net.liftweb" % "lift-json_2.9.2" % "2.5", "org.apache.mesos" % "mesos" % "0.9.0-incubating", "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test" @@ -189,7 +186,7 @@ object SparkBuild extends Build { "src/hadoop" + HADOOP_MAJOR_VERSION + "/scala" } ) } - ) ++ assemblySettings ++ extraAssemblySettings ++ Twirl.settings + ) ++ assemblySettings ++ extraAssemblySettings def rootSettings = sharedSettings ++ Seq( publish := {} diff --git a/project/plugins.sbt b/project/plugins.sbt index f806e66481..1b0f879b94 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -10,8 +10,6 @@ addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.1") addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.2.0") -addSbtPlugin("io.spray" %% "sbt-twirl" % "0.6.1") - // For Sonatype publishing //resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) -- cgit v1.2.3 From d6fde4ffe402b406ac9d9ea6d5c3ee0fb682f628 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 15:34:54 -0700 Subject: Some JSON cleanup --- core/src/main/scala/spark/deploy/master/MasterWebUI.scala | 13 +++++++++++++ core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala | 11 +++++++++-- core/src/main/scala/spark/util/WebUI.scala | 5 ++--- 3 files changed, 24 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 6623142d69..49300ea742 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -14,6 +14,7 @@ import org.eclipse.jetty.server.Handler import spark.util.WebUI._ import spark.deploy.MasterState import javax.servlet.http.HttpServletRequest +import net.liftweb.json.JsonAST.JValue /** * Web UI server for the standalone master. @@ -39,10 +40,22 @@ class MasterWebUI(master: ActorRef) extends Logging { val handlers = Array[(String, Handler)]( ("/static", createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR)), + ("/app/json", (request: HttpServletRequest) => appDetailJson(request)), ("/app", (request: HttpServletRequest) => appDetail(request)), ("*", (request: HttpServletRequest) => index) ) + /** Executor details for a particular application */ + def appDetailJson(request: HttpServletRequest): JValue = { + val appId = request.getParameter("appId") + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val state = Await.result(stateFuture, 3 seconds) + val app = state.activeApps.find(_.id == appId).getOrElse({ + state.completedApps.find(_.id == appId).getOrElse(null) + }) + JsonProtocol.writeApplicationInfo(app) + } + /** Executor details for a particular application */ def appDetail(request: HttpServletRequest): Seq[Node] = { val appId = request.getParameter("appId") diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index 0af9eb8efa..3257f0f513 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -6,16 +6,16 @@ import akka.pattern.ask import akka.util.{Duration, Timeout} import akka.util.duration._ -import spark.deploy.{WorkerState, RequestWorkerState} +import spark.deploy.{JsonProtocol, WorkerState, RequestWorkerState} import java.io.File import spark.util.{WebUI => UtilsWebUI} import spark.{Utils, Logging} import org.eclipse.jetty.server.Handler import spark.util.WebUI._ -import spark.deploy.WorkerState import scala.io.Source import javax.servlet.http.HttpServletRequest import xml.Node +import net.liftweb.json.JsonAST.JValue /** * Web UI server for the standalone worker. @@ -31,6 +31,7 @@ class WorkerWebUI(worker: ActorRef, workDir: File) extends Logging { val handlers = Array[(String, Handler)]( ("/static", createStaticHandler(WorkerWebUI.STATIC_RESOURCE_DIR)), ("/log", (request: HttpServletRequest) => log(request)), + ("/json", (request: HttpServletRequest) => indexJson), ("*", (request: HttpServletRequest) => index) ) @@ -45,6 +46,12 @@ class WorkerWebUI(worker: ActorRef, workDir: File) extends Logging { } } + def indexJson(): JValue = { + val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] + val workerState = Await.result(stateFuture, 3 seconds) + JsonProtocol.writeWorkerState(workerState) + } + def index(): Seq[Node] = { val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] val workerState = Await.result(stateFuture, 3 seconds) diff --git a/core/src/main/scala/spark/util/WebUI.scala b/core/src/main/scala/spark/util/WebUI.scala index e6b39b15eb..4d9df55a4f 100644 --- a/core/src/main/scala/spark/util/WebUI.scala +++ b/core/src/main/scala/spark/util/WebUI.scala @@ -34,9 +34,8 @@ object WebUI extends Logging { response.setContentType("%s;charset=utf-8".format(contentType)) response.setStatus(HttpServletResponse.SC_OK) baseRequest.setHandled(true) - response.getWriter().println( - responder(request).toString - ) + val result = responder(request) + response.getWriter().println(extractFn(result)) } } } -- cgit v1.2.3 From 4e1f202481844d950ef2a80679bdc9cee85f0b75 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 15:37:27 -0700 Subject: Removing dead code --- core/src/main/scala/spark/util/WebUI.scala | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/core/src/main/scala/spark/util/WebUI.scala b/core/src/main/scala/spark/util/WebUI.scala index 4d9df55a4f..ae69c159e7 100644 --- a/core/src/main/scala/spark/util/WebUI.scala +++ b/core/src/main/scala/spark/util/WebUI.scala @@ -40,7 +40,6 @@ object WebUI extends Logging { } } - /** Create and return a staticHandler if resourceBase can be located */ def createStaticHandler(resourceBase: String): ResourceHandler = { val staticHandler = new ResourceHandler Option(getClass.getClassLoader.getResource(resourceBase)) match { @@ -50,19 +49,6 @@ object WebUI extends Logging { } } - /* - /** Create and return a staticHandler if resourceBase can be located */ - def createStaticHandler(resourceBase: String): Option[ResourceHandler] = { - val staticHandler = new ResourceHandler - Option(getClass.getClassLoader.getResource(resourceBase)) match { - case Some(res) => - staticHandler.setResourceBase (res.toString) - Some(staticHandler) - case None => None - } - } - */ - def startJettyServer(ip: String, port: Int, handlers: Array[(String, Handler)]): (Server, Int) = { val handlersToRegister = handlers.map { case(path, handler) => if (path == "*") { -- cgit v1.2.3 From 17f145f3bcdee12d2d965af3bc2ae3f9aa2d8ff3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 16:50:23 -0700 Subject: Updating Maven build --- core/pom.xml | 24 ++++-------------------- pom.xml | 51 +++++---------------------------------------------- 2 files changed, 9 insertions(+), 66 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 88f0ed70f3..5edafb3706 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -55,6 +55,10 @@ com.typesafe.akka akka-slf4j + + net.liftweb + lift-json_2.9.2 + it.unimi.dsi fastutil @@ -63,22 +67,6 @@ colt colt - - cc.spray - spray-can - - - cc.spray - spray-server - - - cc.spray - spray-json_2.9.2 - - - org.tomdz.twirl - twirl-api - com.github.scala-incubator.io scala-io-file_2.9.2 @@ -171,10 +159,6 @@ - - org.tomdz.twirl - twirl-maven-plugin - diff --git a/pom.xml b/pom.xml index 3bcb2a3f34..81043e3b6c 100644 --- a/pom.xml +++ b/pom.xml @@ -54,8 +54,6 @@ 2.9.3 0.9.0-incubating 2.0.3 - 1.0-M2.1 - 1.1.1 1.6.1 4.1.2 1.2.17 @@ -98,17 +96,6 @@ false - - spray-repo - Spray Repository - http://repo.spray.cc/ - - true - - - false - - twitter4j-repo Twitter4J Repository @@ -229,26 +216,6 @@ colt 1.2.0 - - cc.spray - spray-can - ${spray.version} - - - cc.spray - spray-server - ${spray.version} - - - cc.spray - spray-json_2.9.2 - ${spray.json.version} - - - org.tomdz.twirl - twirl-api - 1.0.2 - com.github.scala-incubator.io scala-io-file_2.9.2 @@ -270,7 +237,11 @@ 10.4.2.0 test - + + net.liftweb + lift-json_2.9.2 + 2.5 + org.scala-lang scala-compiler @@ -346,18 +317,6 @@ build-helper-maven-plugin 1.7 - - org.tomdz.twirl - twirl-maven-plugin - 1.0.1 - - - - generate - - - - net.alchim31.maven scala-maven-plugin -- cgit v1.2.3 From 32a45d01b104f79d9e5f10c333c44cbf910949ef Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 18:45:14 -0700 Subject: Removing twirl files --- core/src/main/scala/spark/util/WebUI.scala | 12 ++-- core/src/main/twirl/spark/common/layout.scala.html | 35 ---------- .../spark/deploy/master/app_details.scala.html | 38 ---------- .../twirl/spark/deploy/master/app_row.scala.html | 20 ------ .../twirl/spark/deploy/master/app_table.scala.html | 21 ------ .../spark/deploy/master/executor_row.scala.html | 15 ---- .../spark/deploy/master/executors_table.scala.html | 19 ----- .../twirl/spark/deploy/master/index.scala.html | 53 -------------- .../spark/deploy/master/worker_row.scala.html | 14 ---- .../spark/deploy/master/worker_table.scala.html | 18 ----- .../spark/deploy/worker/executor_row.scala.html | 20 ------ .../spark/deploy/worker/executors_table.scala.html | 18 ----- .../twirl/spark/deploy/worker/index.scala.html | 44 ------------ core/src/main/twirl/spark/storage/index.scala.html | 40 ----------- core/src/main/twirl/spark/storage/rdd.scala.html | 81 ---------------------- .../main/twirl/spark/storage/rdd_table.scala.html | 32 --------- .../twirl/spark/storage/worker_table.scala.html | 24 ------- 17 files changed, 8 insertions(+), 496 deletions(-) delete mode 100644 core/src/main/twirl/spark/common/layout.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/master/app_details.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/master/app_row.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/master/app_table.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/master/executor_row.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/master/executors_table.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/master/index.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/master/worker_row.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/master/worker_table.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/worker/executor_row.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/worker/executors_table.scala.html delete mode 100644 core/src/main/twirl/spark/deploy/worker/index.scala.html delete mode 100644 core/src/main/twirl/spark/storage/index.scala.html delete mode 100644 core/src/main/twirl/spark/storage/rdd.scala.html delete mode 100644 core/src/main/twirl/spark/storage/rdd_table.scala.html delete mode 100644 core/src/main/twirl/spark/storage/worker_table.scala.html diff --git a/core/src/main/scala/spark/util/WebUI.scala b/core/src/main/scala/spark/util/WebUI.scala index ae69c159e7..b9ddfdb40f 100644 --- a/core/src/main/scala/spark/util/WebUI.scala +++ b/core/src/main/scala/spark/util/WebUI.scala @@ -86,6 +86,9 @@ object WebUI extends Logging { {title} +
      @@ -98,9 +101,10 @@ object WebUI extends Logging {
      - {content} -
    - - +
    + {content} +
    + + } } diff --git a/core/src/main/twirl/spark/common/layout.scala.html b/core/src/main/twirl/spark/common/layout.scala.html deleted file mode 100644 index b9192060aa..0000000000 --- a/core/src/main/twirl/spark/common/layout.scala.html +++ /dev/null @@ -1,35 +0,0 @@ -@(title: String)(content: Html) - - - - - - - - - - @title - - - - -
    - - -
    -
    - -

    @title

    -
    -
    - -
    - - @content - -
    - - - \ No newline at end of file diff --git a/core/src/main/twirl/spark/deploy/master/app_details.scala.html b/core/src/main/twirl/spark/deploy/master/app_details.scala.html deleted file mode 100644 index 5e5e5de551..0000000000 --- a/core/src/main/twirl/spark/deploy/master/app_details.scala.html +++ /dev/null @@ -1,38 +0,0 @@ -@(app: spark.deploy.master.ApplicationInfo) - -@spark.common.html.layout(title = "Application Details") { - - -
    -
    -
      -
    • ID: @app.id
    • -
    • Description: @app.desc.name
    • -
    • User: @app.desc.user
    • -
    • Cores: - @if(app.desc.maxCores == Integer.MAX_VALUE) { - Unlimited (@app.coresGranted granted) - } else { - @app.desc.maxCores (@app.coresGranted granted, @app.coresLeft left) - } -
    • -
    • Memory per Slave: @app.desc.memoryPerSlave
    • -
    • Submit Date: @app.submitDate
    • -
    • State: @app.state
    • -
    • Application Detail UI
    • -
    -
    -
    - -
    - - -
    -
    -

    Executor Summary

    -
    - @executors_table(app.executors.values.toList) -
    -
    - -} diff --git a/core/src/main/twirl/spark/deploy/master/app_row.scala.html b/core/src/main/twirl/spark/deploy/master/app_row.scala.html deleted file mode 100644 index feb306f35c..0000000000 --- a/core/src/main/twirl/spark/deploy/master/app_row.scala.html +++ /dev/null @@ -1,20 +0,0 @@ -@(app: spark.deploy.master.ApplicationInfo) - -@import spark.Utils -@import spark.deploy.WebUI.formatDate -@import spark.deploy.WebUI.formatDuration - - - - @app.id - - @app.desc.name - - @app.coresGranted - - @Utils.memoryMegabytesToString(app.desc.memoryPerSlave) - @formatDate(app.submitDate) - @app.desc.user - @app.state.toString() - @formatDuration(app.duration) - diff --git a/core/src/main/twirl/spark/deploy/master/app_table.scala.html b/core/src/main/twirl/spark/deploy/master/app_table.scala.html deleted file mode 100644 index f789cee0f1..0000000000 --- a/core/src/main/twirl/spark/deploy/master/app_table.scala.html +++ /dev/null @@ -1,21 +0,0 @@ -@(apps: Array[spark.deploy.master.ApplicationInfo]) - - - - - - - - - - - - - - - - @for(j <- apps) { - @app_row(j) - } - -
    IDDescriptionCoresMemory per NodeSubmit TimeUserStateDuration
    diff --git a/core/src/main/twirl/spark/deploy/master/executor_row.scala.html b/core/src/main/twirl/spark/deploy/master/executor_row.scala.html deleted file mode 100644 index 21e72c7aab..0000000000 --- a/core/src/main/twirl/spark/deploy/master/executor_row.scala.html +++ /dev/null @@ -1,15 +0,0 @@ -@(executor: spark.deploy.master.ExecutorInfo) - - - @executor.id - - @executor.worker.id - - @executor.cores - @executor.memory - @executor.state - - stdout - stderr - - diff --git a/core/src/main/twirl/spark/deploy/master/executors_table.scala.html b/core/src/main/twirl/spark/deploy/master/executors_table.scala.html deleted file mode 100644 index cafc42c80e..0000000000 --- a/core/src/main/twirl/spark/deploy/master/executors_table.scala.html +++ /dev/null @@ -1,19 +0,0 @@ -@(executors: List[spark.deploy.master.ExecutorInfo]) - - - - - - - - - - - - - - @for(e <- executors) { - @executor_row(e) - } - -
    ExecutorIDWorkerCoresMemoryStateLogs
    \ No newline at end of file diff --git a/core/src/main/twirl/spark/deploy/master/index.scala.html b/core/src/main/twirl/spark/deploy/master/index.scala.html deleted file mode 100644 index b9b9f08810..0000000000 --- a/core/src/main/twirl/spark/deploy/master/index.scala.html +++ /dev/null @@ -1,53 +0,0 @@ -@(state: spark.deploy.MasterState) -@import spark.deploy.master._ -@import spark.Utils - -@spark.common.html.layout(title = "Spark Master on " + state.host + ":" + state.port) { - - -
    -
    -
      -
    • URL: @(state.uri)
    • -
    • Workers: @state.workers.size
    • -
    • Cores: @{state.workers.map(_.cores).sum} Total, - @{state.workers.map(_.coresUsed).sum} Used
    • -
    • Memory: @{Utils.memoryMegabytesToString(state.workers.map(_.memory).sum)} Total, - @{Utils.memoryMegabytesToString(state.workers.map(_.memoryUsed).sum)} Used
    • -
    • Applications: @state.activeApps.size Running, @state.completedApps.size Completed
    • -
    -
    -
    - - -
    -
    -

    Workers

    -
    - @worker_table(state.workers.sortBy(_.id)) -
    -
    - -
    - - -
    -
    -

    Running Applications

    -
    - @app_table(state.activeApps.sortBy(_.startTime).reverse) -
    -
    - -
    - - -
    -
    -

    Completed Applications

    -
    - @app_table(state.completedApps.sortBy(_.endTime).reverse) -
    -
    - -} diff --git a/core/src/main/twirl/spark/deploy/master/worker_row.scala.html b/core/src/main/twirl/spark/deploy/master/worker_row.scala.html deleted file mode 100644 index 46277ca421..0000000000 --- a/core/src/main/twirl/spark/deploy/master/worker_row.scala.html +++ /dev/null @@ -1,14 +0,0 @@ -@(worker: spark.deploy.master.WorkerInfo) - -@import spark.Utils - - - - @worker.id - - @{worker.host}:@{worker.port} - @worker.state - @worker.cores (@worker.coresUsed Used) - @{Utils.memoryMegabytesToString(worker.memory)} - (@{Utils.memoryMegabytesToString(worker.memoryUsed)} Used) - diff --git a/core/src/main/twirl/spark/deploy/master/worker_table.scala.html b/core/src/main/twirl/spark/deploy/master/worker_table.scala.html deleted file mode 100644 index b249411a62..0000000000 --- a/core/src/main/twirl/spark/deploy/master/worker_table.scala.html +++ /dev/null @@ -1,18 +0,0 @@ -@(workers: Array[spark.deploy.master.WorkerInfo]) - - - - - - - - - - - - - @for(w <- workers) { - @worker_row(w) - } - -
    IDAddressStateCoresMemory
    diff --git a/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html b/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html deleted file mode 100644 index dad0a89080..0000000000 --- a/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html +++ /dev/null @@ -1,20 +0,0 @@ -@(executor: spark.deploy.worker.ExecutorRunner) - -@import spark.Utils - - - @executor.execId - @executor.cores - @Utils.memoryMegabytesToString(executor.memory) - -
      -
    • ID: @executor.appId
    • -
    • Name: @executor.appDesc.name
    • -
    • User: @executor.appDesc.user
    • -
    - - - stdout - stderr - - diff --git a/core/src/main/twirl/spark/deploy/worker/executors_table.scala.html b/core/src/main/twirl/spark/deploy/worker/executors_table.scala.html deleted file mode 100644 index 327a2399c7..0000000000 --- a/core/src/main/twirl/spark/deploy/worker/executors_table.scala.html +++ /dev/null @@ -1,18 +0,0 @@ -@(executors: List[spark.deploy.worker.ExecutorRunner]) - - - - - - - - - - - - - @for(e <- executors) { - @executor_row(e) - } - -
    ExecutorIDCoresMemoryJob DetailsLogs
    \ No newline at end of file diff --git a/core/src/main/twirl/spark/deploy/worker/index.scala.html b/core/src/main/twirl/spark/deploy/worker/index.scala.html deleted file mode 100644 index 0e66af9284..0000000000 --- a/core/src/main/twirl/spark/deploy/worker/index.scala.html +++ /dev/null @@ -1,44 +0,0 @@ -@(worker: spark.deploy.WorkerState) -@import spark.Utils - -@spark.common.html.layout(title = "Spark Worker on " + worker.host + ":" + worker.port) { - - -
    -
    -
      -
    • ID: @worker.workerId
    • -
    • - Master URL: @worker.masterUrl -
    • -
    • Cores: @worker.cores (@worker.coresUsed Used)
    • -
    • Memory: @{Utils.memoryMegabytesToString(worker.memory)} - (@{Utils.memoryMegabytesToString(worker.memoryUsed)} Used)
    • -
    -

    Back to Master

    -
    -
    - -
    - - -
    -
    -

    Running Executors

    -
    - @executors_table(worker.executors) -
    -
    - -
    - - -
    -
    -

    Finished Executors

    -
    - @executors_table(worker.finishedExecutors) -
    -
    - -} diff --git a/core/src/main/twirl/spark/storage/index.scala.html b/core/src/main/twirl/spark/storage/index.scala.html deleted file mode 100644 index 2b337f6133..0000000000 --- a/core/src/main/twirl/spark/storage/index.scala.html +++ /dev/null @@ -1,40 +0,0 @@ -@(maxMem: Long, remainingMem: Long, diskSpaceUsed: Long, rdds: Array[spark.storage.RDDInfo], storageStatusList: Array[spark.storage.StorageStatus]) -@import spark.Utils - -@spark.common.html.layout(title = "Storage Dashboard") { - - -
    -
    -
      -
    • Memory: - @{Utils.memoryBytesToString(maxMem - remainingMem)} Used - (@{Utils.memoryBytesToString(remainingMem)} Available)
    • -
    • Disk: @{Utils.memoryBytesToString(diskSpaceUsed)} Used
    • -
    -
    -
    - -
    - - -
    -
    -

    RDD Summary

    -
    - @rdd_table(rdds) -
    -
    - -
    - - -
    -
    -

    Worker Summary

    -
    - @worker_table(storageStatusList) -
    -
    - -} \ No newline at end of file diff --git a/core/src/main/twirl/spark/storage/rdd.scala.html b/core/src/main/twirl/spark/storage/rdd.scala.html deleted file mode 100644 index d85addeb17..0000000000 --- a/core/src/main/twirl/spark/storage/rdd.scala.html +++ /dev/null @@ -1,81 +0,0 @@ -@(rddInfo: spark.storage.RDDInfo, storageStatusList: Array[spark.storage.StorageStatus]) -@import spark.Utils - -@spark.common.html.layout(title = "RDD Info ") { - - -
    -
    -
      -
    • - Storage Level: - @(rddInfo.storageLevel.description) -
    • - Cached Partitions: - @(rddInfo.numCachedPartitions) -
    • -
    • - Total Partitions: - @(rddInfo.numPartitions) -
    • -
    • - Memory Size: - @{Utils.memoryBytesToString(rddInfo.memSize)} -
    • -
    • - Disk Size: - @{Utils.memoryBytesToString(rddInfo.diskSize)} -
    • -
    -
    -
    - -
    - - -
    -
    -

    RDD Summary

    -
    - - - - - - - - - - - - - - @storageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1).map { case (k,v) => - - - - - - - } - -
    Block NameStorage LevelSize in MemorySize on Disk
    @k - @(v.storageLevel.description) - @{Utils.memoryBytesToString(v.memSize)}@{Utils.memoryBytesToString(v.diskSize)}
    - - -
    -
    - -
    - - -
    -
    -

    Worker Summary

    -
    - @worker_table(storageStatusList, "rdd_" + rddInfo.id ) -
    -
    - -} \ No newline at end of file diff --git a/core/src/main/twirl/spark/storage/rdd_table.scala.html b/core/src/main/twirl/spark/storage/rdd_table.scala.html deleted file mode 100644 index a51e64aed0..0000000000 --- a/core/src/main/twirl/spark/storage/rdd_table.scala.html +++ /dev/null @@ -1,32 +0,0 @@ -@(rdds: Array[spark.storage.RDDInfo]) -@import spark.Utils - - - - - - - - - - - - - - @for(rdd <- rdds) { - - - - - - - - - } - -
    RDD NameStorage LevelCached PartitionsFraction Partitions CachedSize in MemorySize on Disk
    - - @rdd.name - - @(rdd.storageLevel.description) - @rdd.numCachedPartitions@(rdd.numCachedPartitions / rdd.numPartitions.toDouble)@{Utils.memoryBytesToString(rdd.memSize)}@{Utils.memoryBytesToString(rdd.diskSize)}
    \ No newline at end of file diff --git a/core/src/main/twirl/spark/storage/worker_table.scala.html b/core/src/main/twirl/spark/storage/worker_table.scala.html deleted file mode 100644 index cd72a688c1..0000000000 --- a/core/src/main/twirl/spark/storage/worker_table.scala.html +++ /dev/null @@ -1,24 +0,0 @@ -@(workersStatusList: Array[spark.storage.StorageStatus], prefix: String = "") -@import spark.Utils - - - - - - - - - - - @for(status <- workersStatusList) { - - - - - - } - -
    HostMemory UsageDisk Usage
    @(status.blockManagerId.host + ":" + status.blockManagerId.port) - @(Utils.memoryBytesToString(status.memUsed(prefix))) - (@(Utils.memoryBytesToString(status.memRemaining)) Total Available) - @(Utils.memoryBytesToString(status.diskUsed(prefix)))
    \ No newline at end of file -- cgit v1.2.3 From 8b5c7e71c4cbf8b8ad9ab9c81f3b21ee3da49e24 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 19:02:20 -0700 Subject: Import cleanup --- core/src/main/scala/spark/deploy/WebUI.scala | 2 +- .../scala/spark/deploy/master/MasterWebUI.scala | 25 +++++++++++----------- .../scala/spark/deploy/worker/WorkerWebUI.scala | 13 ++++++----- .../main/scala/spark/storage/BlockManagerUI.scala | 8 +++---- core/src/main/scala/spark/util/WebUI.scala | 13 +++++------ 5 files changed, 27 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/spark/deploy/WebUI.scala b/core/src/main/scala/spark/deploy/WebUI.scala index ad1a1092b2..844c4142c7 100644 --- a/core/src/main/scala/spark/deploy/WebUI.scala +++ b/core/src/main/scala/spark/deploy/WebUI.scala @@ -6,7 +6,7 @@ import java.util.Date /** * Utilities used throughout the web UI. */ -private[spark] object WebUI { +private[spark] object DeployWebUI { val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") def formatDate(date: Date): String = DATE_FORMAT.format(date) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 49300ea742..171e74a3a6 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -1,20 +1,19 @@ package spark.deploy.master -import akka.actor.{ActorRef} +import akka.actor.ActorRef import akka.dispatch.Await import akka.pattern.ask -import akka.util.{Duration} +import akka.util.Duration import akka.util.duration._ +import javax.servlet.http.HttpServletRequest +import net.liftweb.json.JsonAST.JValue +import org.eclipse.jetty.server.Handler import scala.xml.Node import spark.{Logging, Utils} -import spark.util.{WebUI => UtilsWebUI} - -import spark.deploy._ -import org.eclipse.jetty.server.Handler +import spark.util.WebUI import spark.util.WebUI._ +import spark.deploy._ import spark.deploy.MasterState -import javax.servlet.http.HttpServletRequest -import net.liftweb.json.JsonAST.JValue /** * Web UI server for the standalone master. @@ -29,7 +28,7 @@ class MasterWebUI(master: ActorRef) extends Logging { def start() { try { - val (server, boundPort) = UtilsWebUI.startJettyServer("0.0.0.0", port, handlers) + val (server, boundPort) = WebUI.startJettyServer("0.0.0.0", port, handlers) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => @@ -98,7 +97,7 @@ class MasterWebUI(master: ActorRef) extends Logging { {executorTable(app.executors.values.toList)} ; - UtilsWebUI.makePage(content, "Application Info: " + app.desc.name) + WebUI.makePage(content, "Application Info: " + app.desc.name) } def executorTable(executors: Seq[ExecutorInfo]): Seq[Node] = { @@ -187,7 +186,7 @@ class MasterWebUI(master: ActorRef) extends Logging { {appTable(state.completedApps.sortBy(_.endTime).reverse)} ; - UtilsWebUI.makePage(content, "Spark Master: " + state.uri) + WebUI.makePage(content, "Spark Master: " + state.uri) } def workerTable(workers: Seq[spark.deploy.master.WorkerInfo]) = { @@ -246,10 +245,10 @@ class MasterWebUI(master: ActorRef) extends Logging { {app.coresGranted} {Utils.memoryMegabytesToString(app.desc.memoryPerSlave)} - {WebUI.formatDate(app.submitDate)} + {DeployWebUI.formatDate(app.submitDate)} {app.desc.user} {app.state.toString} - {WebUI.formatDuration(app.duration)} + {DeployWebUI.formatDuration(app.duration)} } } diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index 3257f0f513..80b285b3f6 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -5,17 +5,16 @@ import akka.dispatch.Await import akka.pattern.ask import akka.util.{Duration, Timeout} import akka.util.duration._ - -import spark.deploy.{JsonProtocol, WorkerState, RequestWorkerState} import java.io.File -import spark.util.{WebUI => UtilsWebUI} -import spark.{Utils, Logging} +import javax.servlet.http.HttpServletRequest +import net.liftweb.json.JsonAST.JValue import org.eclipse.jetty.server.Handler -import spark.util.WebUI._ import scala.io.Source -import javax.servlet.http.HttpServletRequest +import spark.{Utils, Logging} +import spark.deploy.{JsonProtocol, WorkerState, RequestWorkerState} +import spark.util.{WebUI => UtilsWebUI} +import spark.util.WebUI._ import xml.Node -import net.liftweb.json.JsonAST.JValue /** * Web UI server for the standalone worker. diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala index e9c362fce7..ecffd1edcf 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala @@ -2,15 +2,13 @@ package spark.storage import akka.actor.{ActorRef, ActorSystem} import akka.util.Duration -import akka.util.duration._ +import javax.servlet.http.HttpServletRequest +import org.eclipse.jetty.server.Handler import spark.{Logging, SparkContext} import spark.Utils +import spark.util.WebUI._ import spark.util.WebUI -import org.eclipse.jetty.server.Handler -import javax.servlet.http.HttpServletRequest import xml.Node -import spark.util.WebUI._ - /** * Web UI server for the BlockManager inside each SparkContext. diff --git a/core/src/main/scala/spark/util/WebUI.scala b/core/src/main/scala/spark/util/WebUI.scala index b9ddfdb40f..bde1acdbca 100644 --- a/core/src/main/scala/spark/util/WebUI.scala +++ b/core/src/main/scala/spark/util/WebUI.scala @@ -1,16 +1,13 @@ package spark.util -import xml.Node -import org.eclipse.jetty.server.{Server, Request, Handler} +import annotation.tailrec import javax.servlet.http.{HttpServletResponse, HttpServletRequest} +import net.liftweb.json._ +import org.eclipse.jetty.server.{Server, Request, Handler} import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler} -import util.Try -import util.Success -import util.Failure import spark.Logging -import annotation.tailrec -import net.liftweb.json.JsonAST.JValue -import net.liftweb.json._ +import util.{Try, Success, Failure} +import xml.Node object WebUI extends Logging { type Responder[T] = HttpServletRequest => T -- cgit v1.2.3 From 77c53f7868a036dc7712865e3a43d9fdea47cc98 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 22:18:04 -0700 Subject: Refactoring UI packages --- .../deploy/static/bootstrap-responsive.min.css | 9 - .../spark/deploy/static/bootstrap.min.css | 9 - .../resources/spark/deploy/static/sorttable.js | 495 --------------------- .../resources/spark/deploy/static/spark_logo.png | Bin 14233 -> 0 bytes .../webui/static/bootstrap-responsive.min.css | 9 + .../resources/spark/webui/static/bootstrap.min.css | 9 + .../main/resources/spark/webui/static/sorttable.js | 495 +++++++++++++++++++++ .../resources/spark/webui/static/spark_logo.png | Bin 0 -> 14233 bytes core/src/main/scala/spark/SparkContext.scala | 8 +- .../scala/spark/deploy/master/MasterWebUI.scala | 13 +- .../scala/spark/deploy/worker/WorkerWebUI.scala | 42 +- .../main/scala/spark/storage/BlockManagerUI.scala | 202 --------- core/src/main/scala/spark/ui/BlockManagerUI.scala | 181 ++++++++ core/src/main/scala/spark/ui/SparkUI.scala | 41 ++ core/src/main/scala/spark/ui/WebUI.scala | 121 +++++ core/src/main/scala/spark/util/WebUI.scala | 107 ----- 16 files changed, 873 insertions(+), 868 deletions(-) delete mode 100644 core/src/main/resources/spark/deploy/static/bootstrap-responsive.min.css delete mode 100644 core/src/main/resources/spark/deploy/static/bootstrap.min.css delete mode 100644 core/src/main/resources/spark/deploy/static/sorttable.js delete mode 100644 core/src/main/resources/spark/deploy/static/spark_logo.png create mode 100644 core/src/main/resources/spark/webui/static/bootstrap-responsive.min.css create mode 100644 core/src/main/resources/spark/webui/static/bootstrap.min.css create mode 100644 core/src/main/resources/spark/webui/static/sorttable.js create mode 100644 core/src/main/resources/spark/webui/static/spark_logo.png delete mode 100644 core/src/main/scala/spark/storage/BlockManagerUI.scala create mode 100644 core/src/main/scala/spark/ui/BlockManagerUI.scala create mode 100644 core/src/main/scala/spark/ui/SparkUI.scala create mode 100644 core/src/main/scala/spark/ui/WebUI.scala delete mode 100644 core/src/main/scala/spark/util/WebUI.scala diff --git a/core/src/main/resources/spark/deploy/static/bootstrap-responsive.min.css b/core/src/main/resources/spark/deploy/static/bootstrap-responsive.min.css deleted file mode 100644 index 1f55036a07..0000000000 --- a/core/src/main/resources/spark/deploy/static/bootstrap-responsive.min.css +++ /dev/null @@ -1,9 +0,0 @@ -/*! - * Bootstrap Responsive v2.0.4 - * - * Copyright 2012 Twitter, Inc - * Licensed under the Apache License v2.0 - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Designed and built with all the love in the world @twitter by @mdo and @fat. - */.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:28px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box}.hidden{display:none;visibility:hidden}.visible-phone{display:none!important}.visible-tablet{display:none!important}.hidden-desktop{display:none!important}@media(max-width:767px){.visible-phone{display:inherit!important}.hidden-phone{display:none!important}.hidden-desktop{display:inherit!important}.visible-desktop{display:none!important}}@media(min-width:768px) and (max-width:979px){.visible-tablet{display:inherit!important}.hidden-tablet{display:none!important}.hidden-desktop{display:inherit!important}.visible-desktop{display:none!important}}@media(max-width:480px){.nav-collapse{-webkit-transform:translate3d(0,0,0)}.page-header h1 small{display:block;line-height:18px}input[type="checkbox"],input[type="radio"]{border:1px solid #ccc}.form-horizontal .control-group>label{float:none;width:auto;padding-top:0;text-align:left}.form-horizontal .controls{margin-left:0}.form-horizontal .control-list{padding-top:0}.form-horizontal .form-actions{padding-right:10px;padding-left:10px}.modal{position:absolute;top:10px;right:10px;left:10px;width:auto;margin:0}.modal.fade.in{top:auto}.modal-header .close{padding:10px;margin:-10px}.carousel-caption{position:static}}@media(max-width:767px){body{padding-right:20px;padding-left:20px}.navbar-fixed-top,.navbar-fixed-bottom{margin-right:-20px;margin-left:-20px}.container-fluid{padding:0}.dl-horizontal dt{float:none;width:auto;clear:none;text-align:left}.dl-horizontal dd{margin-left:0}.container{width:auto}.row-fluid{width:100%}.row,.thumbnails{margin-left:0}[class*="span"],.row-fluid [class*="span"]{display:block;float:none;width:auto;margin-left:0}.input-large,.input-xlarge,.input-xxlarge,input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input{display:block;width:100%;min-height:28px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box}.input-prepend input,.input-append input,.input-prepend input[class*="span"],.input-append input[class*="span"]{display:inline-block;width:auto}}@media(min-width:768px) and (max-width:979px){.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:724px}.span12{width:724px}.span11{width:662px}.span10{width:600px}.span9{width:538px}.span8{width:476px}.span7{width:414px}.span6{width:352px}.span5{width:290px}.span4{width:228px}.span3{width:166px}.span2{width:104px}.span1{width:42px}.offset12{margin-left:764px}.offset11{margin-left:702px}.offset10{margin-left:640px}.offset9{margin-left:578px}.offset8{margin-left:516px}.offset7{margin-left:454px}.offset6{margin-left:392px}.offset5{margin-left:330px}.offset4{margin-left:268px}.offset3{margin-left:206px}.offset2{margin-left:144px}.offset1{margin-left:82px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:28px;margin-left:2.762430939%;*margin-left:2.709239449638298%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:99.999999993%;*width:99.9468085036383%}.row-fluid .span11{width:91.436464082%;*width:91.38327259263829%}.row-fluid .span10{width:82.87292817100001%;*width:82.8197366816383%}.row-fluid .span9{width:74.30939226%;*width:74.25620077063829%}.row-fluid .span8{width:65.74585634900001%;*width:65.6926648596383%}.row-fluid .span7{width:57.182320438000005%;*width:57.129128948638304%}.row-fluid .span6{width:48.618784527%;*width:48.5655930376383%}.row-fluid .span5{width:40.055248616%;*width:40.0020571266383%}.row-fluid .span4{width:31.491712705%;*width:31.4385212156383%}.row-fluid .span3{width:22.928176794%;*width:22.874985304638297%}.row-fluid .span2{width:14.364640883%;*width:14.311449393638298%}.row-fluid .span1{width:5.801104972%;*width:5.747913482638298%}input,textarea,.uneditable-input{margin-left:0}input.span12,textarea.span12,.uneditable-input.span12{width:714px}input.span11,textarea.span11,.uneditable-input.span11{width:652px}input.span10,textarea.span10,.uneditable-input.span10{width:590px}input.span9,textarea.span9,.uneditable-input.span9{width:528px}input.span8,textarea.span8,.uneditable-input.span8{width:466px}input.span7,textarea.span7,.uneditable-input.span7{width:404px}input.span6,textarea.span6,.uneditable-input.span6{width:342px}input.span5,textarea.span5,.uneditable-input.span5{width:280px}input.span4,textarea.span4,.uneditable-input.span4{width:218px}input.span3,textarea.span3,.uneditable-input.span3{width:156px}input.span2,textarea.span2,.uneditable-input.span2{width:94px}input.span1,textarea.span1,.uneditable-input.span1{width:32px}}@media(min-width:1200px){.row{margin-left:-30px;*zoom:1}.row:before,.row:after{display:table;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:30px}.container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:1170px}.span12{width:1170px}.span11{width:1070px}.span10{width:970px}.span9{width:870px}.span8{width:770px}.span7{width:670px}.span6{width:570px}.span5{width:470px}.span4{width:370px}.span3{width:270px}.span2{width:170px}.span1{width:70px}.offset12{margin-left:1230px}.offset11{margin-left:1130px}.offset10{margin-left:1030px}.offset9{margin-left:930px}.offset8{margin-left:830px}.offset7{margin-left:730px}.offset6{margin-left:630px}.offset5{margin-left:530px}.offset4{margin-left:430px}.offset3{margin-left:330px}.offset2{margin-left:230px}.offset1{margin-left:130px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:28px;margin-left:2.564102564%;*margin-left:2.510911074638298%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.45299145300001%;*width:91.3997999636383%}.row-fluid .span10{width:82.905982906%;*width:82.8527914166383%}.row-fluid .span9{width:74.358974359%;*width:74.30578286963829%}.row-fluid .span8{width:65.81196581200001%;*width:65.7587743226383%}.row-fluid .span7{width:57.264957265%;*width:57.2117657756383%}.row-fluid .span6{width:48.717948718%;*width:48.6647572286383%}.row-fluid .span5{width:40.170940171000005%;*width:40.117748681638304%}.row-fluid .span4{width:31.623931624%;*width:31.5707401346383%}.row-fluid .span3{width:23.076923077%;*width:23.0237315876383%}.row-fluid .span2{width:14.529914530000001%;*width:14.4767230406383%}.row-fluid .span1{width:5.982905983%;*width:5.929714493638298%}input,textarea,.uneditable-input{margin-left:0}input.span12,textarea.span12,.uneditable-input.span12{width:1160px}input.span11,textarea.span11,.uneditable-input.span11{width:1060px}input.span10,textarea.span10,.uneditable-input.span10{width:960px}input.span9,textarea.span9,.uneditable-input.span9{width:860px}input.span8,textarea.span8,.uneditable-input.span8{width:760px}input.span7,textarea.span7,.uneditable-input.span7{width:660px}input.span6,textarea.span6,.uneditable-input.span6{width:560px}input.span5,textarea.span5,.uneditable-input.span5{width:460px}input.span4,textarea.span4,.uneditable-input.span4{width:360px}input.span3,textarea.span3,.uneditable-input.span3{width:260px}input.span2,textarea.span2,.uneditable-input.span2{width:160px}input.span1,textarea.span1,.uneditable-input.span1{width:60px}.thumbnails{margin-left:-30px}.thumbnails>li{margin-left:30px}.row-fluid .thumbnails{margin-left:0}}@media(max-width:979px){body{padding-top:0}.navbar-fixed-top,.navbar-fixed-bottom{position:static}.navbar-fixed-top{margin-bottom:18px}.navbar-fixed-bottom{margin-top:18px}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding:5px}.navbar .container{width:auto;padding:0}.navbar .brand{padding-right:10px;padding-left:10px;margin:0 0 0 -5px}.nav-collapse{clear:both}.nav-collapse .nav{float:none;margin:0 0 9px}.nav-collapse .nav>li{float:none}.nav-collapse .nav>li>a{margin-bottom:2px}.nav-collapse .nav>.divider-vertical{display:none}.nav-collapse .nav .nav-header{color:#999;text-shadow:none}.nav-collapse .nav>li>a,.nav-collapse .dropdown-menu a{padding:6px 15px;font-weight:bold;color:#999;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.nav-collapse .btn{padding:4px 10px 4px;font-weight:normal;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.nav-collapse .dropdown-menu li+li a{margin-bottom:2px}.nav-collapse .nav>li>a:hover,.nav-collapse .dropdown-menu a:hover{background-color:#222}.nav-collapse.in .btn-group{padding:0;margin-top:5px}.nav-collapse .dropdown-menu{position:static;top:auto;left:auto;display:block;float:none;max-width:none;padding:0;margin:0 15px;background-color:transparent;border:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.nav-collapse .dropdown-menu:before,.nav-collapse .dropdown-menu:after{display:none}.nav-collapse .dropdown-menu .divider{display:none}.nav-collapse .navbar-form,.nav-collapse .navbar-search{float:none;padding:9px 15px;margin:9px 0;border-top:1px solid #222;border-bottom:1px solid #222;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.1);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.1);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.1)}.navbar .nav-collapse .nav.pull-right{float:none;margin-left:0}.nav-collapse,.nav-collapse.collapse{height:0;overflow:hidden}.navbar .btn-navbar{display:block}.navbar-static .navbar-inner{padding-right:10px;padding-left:10px}}@media(min-width:980px){.nav-collapse.collapse{height:auto!important;overflow:visible!important}} diff --git a/core/src/main/resources/spark/deploy/static/bootstrap.min.css b/core/src/main/resources/spark/deploy/static/bootstrap.min.css deleted file mode 100644 index b74b4546a6..0000000000 --- a/core/src/main/resources/spark/deploy/static/bootstrap.min.css +++ /dev/null @@ -1,9 +0,0 @@ -/*! - * Bootstrap v2.0.4 - * - * Copyright 2012 Twitter, Inc - * Licensed under the Apache License v2.0 - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Designed and built with all the love in the world @twitter by @mdo and @fat. - */article,aside,details,figcaption,figure,footer,header,hgroup,nav,section{display:block}audio,canvas,video{display:inline-block;*display:inline;*zoom:1}audio:not([controls]){display:none}html{font-size:100%;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%}a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}a:hover,a:active{outline:0}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sup{top:-0.5em}sub{bottom:-0.25em}img{max-width:100%;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic}#map_canvas img{max-width:none}button,input,select,textarea{margin:0;font-size:100%;vertical-align:middle}button,input{*overflow:visible;line-height:normal}button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0}button,input[type="button"],input[type="reset"],input[type="submit"]{cursor:pointer;-webkit-appearance:button}input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;-webkit-appearance:textfield}input[type="search"]::-webkit-search-decoration,input[type="search"]::-webkit-search-cancel-button{-webkit-appearance:none}textarea{overflow:auto;vertical-align:top}.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:28px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;line-height:18px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.span12{width:940px}.span11{width:860px}.span10{width:780px}.span9{width:700px}.span8{width:620px}.span7{width:540px}.span6{width:460px}.span5{width:380px}.span4{width:300px}.span3{width:220px}.span2{width:140px}.span1{width:60px}.offset12{margin-left:980px}.offset11{margin-left:900px}.offset10{margin-left:820px}.offset9{margin-left:740px}.offset8{margin-left:660px}.offset7{margin-left:580px}.offset6{margin-left:500px}.offset5{margin-left:420px}.offset4{margin-left:340px}.offset3{margin-left:260px}.offset2{margin-left:180px}.offset1{margin-left:100px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:28px;margin-left:2.127659574%;*margin-left:2.0744680846382977%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:99.99999998999999%;*width:99.94680850063828%}.row-fluid .span11{width:91.489361693%;*width:91.4361702036383%}.row-fluid .span10{width:82.97872339599999%;*width:82.92553190663828%}.row-fluid .span9{width:74.468085099%;*width:74.4148936096383%}.row-fluid .span8{width:65.95744680199999%;*width:65.90425531263828%}.row-fluid .span7{width:57.446808505%;*width:57.3936170156383%}.row-fluid .span6{width:48.93617020799999%;*width:48.88297871863829%}.row-fluid .span5{width:40.425531911%;*width:40.3723404216383%}.row-fluid .span4{width:31.914893614%;*width:31.8617021246383%}.row-fluid .span3{width:23.404255317%;*width:23.3510638276383%}.row-fluid .span2{width:14.89361702%;*width:14.8404255306383%}.row-fluid .span1{width:6.382978723%;*width:6.329787233638298%}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;content:""}.container-fluid:after{clear:both}p{margin:0 0 9px}p small{font-size:11px;color:#999}.lead{margin-bottom:18px;font-size:20px;font-weight:200;line-height:27px}h1,h2,h3,h4,h5,h6{margin:0;font-family:inherit;font-weight:bold;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;color:#999}h1{font-size:30px;line-height:36px}h1 small{font-size:18px}h2{font-size:24px;line-height:36px}h2 small{font-size:18px}h3{font-size:18px;line-height:27px}h3 small{font-size:14px}h4,h5,h6{line-height:18px}h4{font-size:14px}h4 small{font-size:12px}h5{font-size:12px}h6{font-size:11px;color:#999;text-transform:uppercase}.page-header{padding-bottom:17px;margin:18px 0;border-bottom:1px solid #eee}.page-header h1{line-height:1}ul,ol{padding:0;margin:0 0 9px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}ul{list-style:disc}ol{list-style:decimal}li{line-height:18px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:18px}dt,dd{line-height:18px}dt{font-weight:bold;line-height:17px}dd{margin-left:9px}.dl-horizontal dt{float:left;width:120px;overflow:hidden;clear:left;text-align:right;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:130px}hr{margin:18px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}strong{font-weight:bold}em{font-style:italic}.muted{color:#999}abbr[title]{cursor:help;border-bottom:1px dotted #999}abbr.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:0 0 0 15px;margin:0 0 18px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:22.5px}blockquote small{display:block;line-height:18px;color:#999}blockquote small:before{content:'\2014 \00A0'}blockquote.pull-right{float:right;padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0}blockquote.pull-right p,blockquote.pull-right small{text-align:right}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:18px;font-style:normal;line-height:18px}small{font-size:100%}cite{font-style:normal}code,pre{padding:0 3px 2px;font-family:Menlo,Monaco,Consolas,"Courier New",monospace;font-size:12px;color:#333;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}code{padding:2px 4px;color:#d14;background-color:#f7f7f9;border:1px solid #e1e1e8}pre{display:block;padding:8.5px;margin:0 0 9px;font-size:12.025px;line-height:18px;word-break:break-all;word-wrap:break-word;white-space:pre;white-space:pre-wrap;background-color:#f5f5f5;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.15);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}pre.prettyprint{margin-bottom:18px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 18px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:27px;font-size:19.5px;line-height:36px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:13.5px;color:#999}label,input,button,select,textarea{font-size:13px;font-weight:normal;line-height:18px}input,button,select,textarea{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif}label{display:block;margin-bottom:5px}select,textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{display:inline-block;height:18px;padding:4px;margin-bottom:9px;font-size:13px;line-height:18px;color:#555}input,textarea{width:210px}textarea{height:auto}textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{background-color:#fff;border:1px solid #ccc;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-webkit-transition:border linear .2s,box-shadow linear .2s;-moz-transition:border linear .2s,box-shadow linear .2s;-ms-transition:border linear .2s,box-shadow linear .2s;-o-transition:border linear .2s,box-shadow linear .2s;transition:border linear .2s,box-shadow linear .2s}textarea:focus,input[type="text"]:focus,input[type="password"]:focus,input[type="datetime"]:focus,input[type="datetime-local"]:focus,input[type="date"]:focus,input[type="month"]:focus,input[type="time"]:focus,input[type="week"]:focus,input[type="number"]:focus,input[type="email"]:focus,input[type="url"]:focus,input[type="search"]:focus,input[type="tel"]:focus,input[type="color"]:focus,.uneditable-input:focus{border-color:rgba(82,168,236,0.8);outline:0;outline:thin dotted \9;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6)}input[type="radio"],input[type="checkbox"]{margin:3px 0;*margin-top:0;line-height:normal;cursor:pointer}input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}.uneditable-textarea{width:auto;height:auto}select,input[type="file"]{height:28px;*margin-top:4px;line-height:28px}select{width:220px;border:1px solid #bbb}select[multiple],select[size]{height:auto}select:focus,input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.radio,.checkbox{min-height:18px;padding-left:18px}.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-18px}.controls>.radio:first-child,.controls>.checkbox:first-child{padding-top:5px}.radio.inline,.checkbox.inline{display:inline-block;padding-top:5px;margin-bottom:0;vertical-align:middle}.radio.inline+.radio.inline,.checkbox.inline+.checkbox.inline{margin-left:10px}.input-mini{width:60px}.input-small{width:90px}.input-medium{width:150px}.input-large{width:210px}.input-xlarge{width:270px}.input-xxlarge{width:530px}input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"]{float:none;margin-left:0}.input-append input[class*="span"],.input-append .uneditable-input[class*="span"],.input-prepend input[class*="span"],.input-prepend .uneditable-input[class*="span"],.row-fluid .input-prepend [class*="span"],.row-fluid .input-append [class*="span"]{display:inline-block}input,textarea,.uneditable-input{margin-left:0}input.span12,textarea.span12,.uneditable-input.span12{width:930px}input.span11,textarea.span11,.uneditable-input.span11{width:850px}input.span10,textarea.span10,.uneditable-input.span10{width:770px}input.span9,textarea.span9,.uneditable-input.span9{width:690px}input.span8,textarea.span8,.uneditable-input.span8{width:610px}input.span7,textarea.span7,.uneditable-input.span7{width:530px}input.span6,textarea.span6,.uneditable-input.span6{width:450px}input.span5,textarea.span5,.uneditable-input.span5{width:370px}input.span4,textarea.span4,.uneditable-input.span4{width:290px}input.span3,textarea.span3,.uneditable-input.span3{width:210px}input.span2,textarea.span2,.uneditable-input.span2{width:130px}input.span1,textarea.span1,.uneditable-input.span1{width:50px}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee;border-color:#ddd}input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent}.control-group.warning>label,.control-group.warning .help-block,.control-group.warning .help-inline{color:#c09853}.control-group.warning .checkbox,.control-group.warning .radio,.control-group.warning input,.control-group.warning select,.control-group.warning textarea{color:#c09853;border-color:#c09853}.control-group.warning .checkbox:focus,.control-group.warning .radio:focus,.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:0 0 6px #dbc59e;-moz-box-shadow:0 0 6px #dbc59e;box-shadow:0 0 6px #dbc59e}.control-group.warning .input-prepend .add-on,.control-group.warning .input-append .add-on{color:#c09853;background-color:#fcf8e3;border-color:#c09853}.control-group.error>label,.control-group.error .help-block,.control-group.error .help-inline{color:#b94a48}.control-group.error .checkbox,.control-group.error .radio,.control-group.error input,.control-group.error select,.control-group.error textarea{color:#b94a48;border-color:#b94a48}.control-group.error .checkbox:focus,.control-group.error .radio:focus,.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:0 0 6px #d59392;-moz-box-shadow:0 0 6px #d59392;box-shadow:0 0 6px #d59392}.control-group.error .input-prepend .add-on,.control-group.error .input-append .add-on{color:#b94a48;background-color:#f2dede;border-color:#b94a48}.control-group.success>label,.control-group.success .help-block,.control-group.success .help-inline{color:#468847}.control-group.success .checkbox,.control-group.success .radio,.control-group.success input,.control-group.success select,.control-group.success textarea{color:#468847;border-color:#468847}.control-group.success .checkbox:focus,.control-group.success .radio:focus,.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:0 0 6px #7aba7b;-moz-box-shadow:0 0 6px #7aba7b;box-shadow:0 0 6px #7aba7b}.control-group.success .input-prepend .add-on,.control-group.success .input-append .add-on{color:#468847;background-color:#dff0d8;border-color:#468847}input:focus:required:invalid,textarea:focus:required:invalid,select:focus:required:invalid{color:#b94a48;border-color:#ee5f5b}input:focus:required:invalid:focus,textarea:focus:required:invalid:focus,select:focus:required:invalid:focus{border-color:#e9322d;-webkit-box-shadow:0 0 6px #f8b9b7;-moz-box-shadow:0 0 6px #f8b9b7;box-shadow:0 0 6px #f8b9b7}.form-actions{padding:17px 20px 18px;margin-top:18px;margin-bottom:18px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;content:""}.form-actions:after{clear:both}.uneditable-input{overflow:hidden;white-space:nowrap;cursor:not-allowed;background-color:#fff;border-color:#eee;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);box-shadow:inset 0 1px 2px rgba(0,0,0,0.025)}:-moz-placeholder{color:#999}:-ms-input-placeholder{color:#999}::-webkit-input-placeholder{color:#999}.help-block,.help-inline{color:#555}.help-block{display:block;margin-bottom:9px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-prepend,.input-append{margin-bottom:5px}.input-prepend input,.input-append input,.input-prepend select,.input-append select,.input-prepend .uneditable-input,.input-append .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;vertical-align:middle;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-prepend input:focus,.input-append input:focus,.input-prepend select:focus,.input-append select:focus,.input-prepend .uneditable-input:focus,.input-append .uneditable-input:focus{z-index:2}.input-prepend .uneditable-input,.input-append .uneditable-input{border-left-color:#ccc}.input-prepend .add-on,.input-append .add-on{display:inline-block;width:auto;height:18px;min-width:16px;padding:4px 5px;font-weight:normal;line-height:18px;text-align:center;text-shadow:0 1px 0 #fff;vertical-align:middle;background-color:#eee;border:1px solid #ccc}.input-prepend .add-on,.input-append .add-on,.input-prepend .btn,.input-append .btn{margin-left:-1px;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend .active,.input-append .active{background-color:#a9dba9;border-color:#46a546}.input-prepend .add-on,.input-prepend .btn{margin-right:-1px}.input-prepend .add-on:first-child,.input-prepend .btn:first-child{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append .uneditable-input{border-right-color:#ccc;border-left-color:#eee}.input-append .add-on:last-child,.input-append .btn:last-child{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-prepend.input-append input,.input-prepend.input-append select,.input-prepend.input-append .uneditable-input{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:14px;-moz-border-radius:14px;border-radius:14px}.form-search input,.form-inline input,.form-horizontal input,.form-search textarea,.form-inline textarea,.form-horizontal textarea,.form-search select,.form-inline select,.form-horizontal select,.form-search .help-inline,.form-inline .help-inline,.form-horizontal .help-inline,.form-search .uneditable-input,.form-inline .uneditable-input,.form-horizontal .uneditable-input,.form-search .input-prepend,.form-inline .input-prepend,.form-horizontal .input-prepend,.form-search .input-append,.form-inline .input-append,.form-horizontal .input-append{display:inline-block;*display:inline;margin-bottom:0;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label{display:inline-block}.form-search .input-append,.form-inline .input-append,.form-search .input-prepend,.form-inline .input-prepend{margin-bottom:0}.form-search .radio,.form-search .checkbox,.form-inline .radio,.form-inline .checkbox{padding-left:0;margin-bottom:0;vertical-align:middle}.form-search .radio input[type="radio"],.form-search .checkbox input[type="checkbox"],.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:left;margin-right:3px;margin-left:0}.control-group{margin-bottom:9px}legend+.control-group{margin-top:18px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:18px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;content:""}.form-horizontal .control-group:after{clear:both}.form-horizontal .control-label{float:left;width:140px;padding-top:5px;text-align:right}.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:160px;*margin-left:0}.form-horizontal .controls:first-child{*padding-left:160px}.form-horizontal .help-block{margin-top:9px;margin-bottom:0}.form-horizontal .form-actions{padding-left:160px}table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0}.table{width:100%;margin-bottom:18px}.table th,.table td{padding:8px;line-height:18px;text-align:left;vertical-align:top;border-top:1px solid #ddd}.table th{font-weight:bold}.table thead th{vertical-align:bottom}.table caption+thead tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child th,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child th,.table thead:first-child tr:first-child td{border-top:0}.table tbody+tbody{border-top:2px solid #ddd}.table-condensed th,.table-condensed td{padding:4px 5px}.table-bordered{border:1px solid #ddd;border-collapse:separate;*border-collapse:collapsed;border-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.table-bordered th,.table-bordered td{border-left:1px solid #ddd}.table-bordered caption+thead tr:first-child th,.table-bordered caption+tbody tr:first-child th,.table-bordered caption+tbody tr:first-child td,.table-bordered colgroup+thead tr:first-child th,.table-bordered colgroup+tbody tr:first-child th,.table-bordered colgroup+tbody tr:first-child td,.table-bordered thead:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child td{border-top:0}.table-bordered thead:first-child tr:first-child th:first-child,.table-bordered tbody:first-child tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered thead:first-child tr:first-child th:last-child,.table-bordered tbody:first-child tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-bordered thead:last-child tr:last-child th:first-child,.table-bordered tbody:last-child tr:last-child td:first-child{-webkit-border-radius:0 0 0 4px;-moz-border-radius:0 0 0 4px;border-radius:0 0 0 4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px}.table-bordered thead:last-child tr:last-child th:last-child,.table-bordered tbody:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px}.table-striped tbody tr:nth-child(odd) td,.table-striped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table tbody tr:hover td,.table tbody tr:hover th{background-color:#f5f5f5}table .span1{float:none;width:44px;margin-left:0}table .span2{float:none;width:124px;margin-left:0}table .span3{float:none;width:204px;margin-left:0}table .span4{float:none;width:284px;margin-left:0}table .span5{float:none;width:364px;margin-left:0}table .span6{float:none;width:444px;margin-left:0}table .span7{float:none;width:524px;margin-left:0}table .span8{float:none;width:604px;margin-left:0}table .span9{float:none;width:684px;margin-left:0}table .span10{float:none;width:764px;margin-left:0}table .span11{float:none;width:844px;margin-left:0}table .span12{float:none;width:924px;margin-left:0}table .span13{float:none;width:1004px;margin-left:0}table .span14{float:none;width:1084px;margin-left:0}table .span15{float:none;width:1164px;margin-left:0}table .span16{float:none;width:1244px;margin-left:0}table .span17{float:none;width:1324px;margin-left:0}table .span18{float:none;width:1404px;margin-left:0}table .span19{float:none;width:1484px;margin-left:0}table .span20{float:none;width:1564px;margin-left:0}table .span21{float:none;width:1644px;margin-left:0}table .span22{float:none;width:1724px;margin-left:0}table .span23{float:none;width:1804px;margin-left:0}table .span24{float:none;width:1884px;margin-left:0}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;*margin-right:.3em;line-height:14px;vertical-align:text-top;background-image:url("../img/glyphicons-halflings.png");background-position:14px 14px;background-repeat:no-repeat}[class^="icon-"]:last-child,[class*=" icon-"]:last-child{*margin-left:0}.icon-white{background-image:url("../img/glyphicons-halflings-white.png")}.icon-glass{background-position:0 0}.icon-music{background-position:-24px 0}.icon-search{background-position:-48px 0}.icon-envelope{background-position:-72px 0}.icon-heart{background-position:-96px 0}.icon-star{background-position:-120px 0}.icon-star-empty{background-position:-144px 0}.icon-user{background-position:-168px 0}.icon-film{background-position:-192px 0}.icon-th-large{background-position:-216px 0}.icon-th{background-position:-240px 0}.icon-th-list{background-position:-264px 0}.icon-ok{background-position:-288px 0}.icon-remove{background-position:-312px 0}.icon-zoom-in{background-position:-336px 0}.icon-zoom-out{background-position:-360px 0}.icon-off{background-position:-384px 0}.icon-signal{background-position:-408px 0}.icon-cog{background-position:-432px 0}.icon-trash{background-position:-456px 0}.icon-home{background-position:0 -24px}.icon-file{background-position:-24px -24px}.icon-time{background-position:-48px -24px}.icon-road{background-position:-72px -24px}.icon-download-alt{background-position:-96px -24px}.icon-download{background-position:-120px -24px}.icon-upload{background-position:-144px -24px}.icon-inbox{background-position:-168px -24px}.icon-play-circle{background-position:-192px -24px}.icon-repeat{background-position:-216px -24px}.icon-refresh{background-position:-240px -24px}.icon-list-alt{background-position:-264px -24px}.icon-lock{background-position:-287px -24px}.icon-flag{background-position:-312px -24px}.icon-headphones{background-position:-336px -24px}.icon-volume-off{background-position:-360px -24px}.icon-volume-down{background-position:-384px -24px}.icon-volume-up{background-position:-408px -24px}.icon-qrcode{background-position:-432px -24px}.icon-barcode{background-position:-456px -24px}.icon-tag{background-position:0 -48px}.icon-tags{background-position:-25px -48px}.icon-book{background-position:-48px -48px}.icon-bookmark{background-position:-72px -48px}.icon-print{background-position:-96px -48px}.icon-camera{background-position:-120px -48px}.icon-font{background-position:-144px -48px}.icon-bold{background-position:-167px -48px}.icon-italic{background-position:-192px -48px}.icon-text-height{background-position:-216px -48px}.icon-text-width{background-position:-240px -48px}.icon-align-left{background-position:-264px -48px}.icon-align-center{background-position:-288px -48px}.icon-align-right{background-position:-312px -48px}.icon-align-justify{background-position:-336px -48px}.icon-list{background-position:-360px -48px}.icon-indent-left{background-position:-384px -48px}.icon-indent-right{background-position:-408px -48px}.icon-facetime-video{background-position:-432px -48px}.icon-picture{background-position:-456px -48px}.icon-pencil{background-position:0 -72px}.icon-map-marker{background-position:-24px -72px}.icon-adjust{background-position:-48px -72px}.icon-tint{background-position:-72px -72px}.icon-edit{background-position:-96px -72px}.icon-share{background-position:-120px -72px}.icon-check{background-position:-144px -72px}.icon-move{background-position:-168px -72px}.icon-step-backward{background-position:-192px -72px}.icon-fast-backward{background-position:-216px -72px}.icon-backward{background-position:-240px -72px}.icon-play{background-position:-264px -72px}.icon-pause{background-position:-288px -72px}.icon-stop{background-position:-312px -72px}.icon-forward{background-position:-336px -72px}.icon-fast-forward{background-position:-360px -72px}.icon-step-forward{background-position:-384px -72px}.icon-eject{background-position:-408px -72px}.icon-chevron-left{background-position:-432px -72px}.icon-chevron-right{background-position:-456px -72px}.icon-plus-sign{background-position:0 -96px}.icon-minus-sign{background-position:-24px -96px}.icon-remove-sign{background-position:-48px -96px}.icon-ok-sign{background-position:-72px -96px}.icon-question-sign{background-position:-96px -96px}.icon-info-sign{background-position:-120px -96px}.icon-screenshot{background-position:-144px -96px}.icon-remove-circle{background-position:-168px -96px}.icon-ok-circle{background-position:-192px -96px}.icon-ban-circle{background-position:-216px -96px}.icon-arrow-left{background-position:-240px -96px}.icon-arrow-right{background-position:-264px -96px}.icon-arrow-up{background-position:-289px -96px}.icon-arrow-down{background-position:-312px -96px}.icon-share-alt{background-position:-336px -96px}.icon-resize-full{background-position:-360px -96px}.icon-resize-small{background-position:-384px -96px}.icon-plus{background-position:-408px -96px}.icon-minus{background-position:-433px -96px}.icon-asterisk{background-position:-456px -96px}.icon-exclamation-sign{background-position:0 -120px}.icon-gift{background-position:-24px -120px}.icon-leaf{background-position:-48px -120px}.icon-fire{background-position:-72px -120px}.icon-eye-open{background-position:-96px -120px}.icon-eye-close{background-position:-120px -120px}.icon-warning-sign{background-position:-144px -120px}.icon-plane{background-position:-168px -120px}.icon-calendar{background-position:-192px -120px}.icon-random{background-position:-216px -120px}.icon-comment{background-position:-240px -120px}.icon-magnet{background-position:-264px -120px}.icon-chevron-up{background-position:-288px -120px}.icon-chevron-down{background-position:-313px -119px}.icon-retweet{background-position:-336px -120px}.icon-shopping-cart{background-position:-360px -120px}.icon-folder-close{background-position:-384px -120px}.icon-folder-open{background-position:-408px -120px}.icon-resize-vertical{background-position:-432px -119px}.icon-resize-horizontal{background-position:-456px -118px}.icon-hdd{background-position:0 -144px}.icon-bullhorn{background-position:-24px -144px}.icon-bell{background-position:-48px -144px}.icon-certificate{background-position:-72px -144px}.icon-thumbs-up{background-position:-96px -144px}.icon-thumbs-down{background-position:-120px -144px}.icon-hand-right{background-position:-144px -144px}.icon-hand-left{background-position:-168px -144px}.icon-hand-up{background-position:-192px -144px}.icon-hand-down{background-position:-216px -144px}.icon-circle-arrow-right{background-position:-240px -144px}.icon-circle-arrow-left{background-position:-264px -144px}.icon-circle-arrow-up{background-position:-288px -144px}.icon-circle-arrow-down{background-position:-312px -144px}.icon-globe{background-position:-336px -144px}.icon-wrench{background-position:-360px -144px}.icon-tasks{background-position:-384px -144px}.icon-filter{background-position:-408px -144px}.icon-briefcase{background-position:-432px -144px}.icon-fullscreen{background-position:-456px -144px}.dropup,.dropdown{position:relative}.dropdown-toggle{*margin-bottom:-3px}.dropdown-toggle:active,.open .dropdown-toggle{outline:0}.caret{display:inline-block;width:0;height:0;vertical-align:top;border-top:4px solid #000;border-right:4px solid transparent;border-left:4px solid transparent;content:"";opacity:.3;filter:alpha(opacity=30)}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown:hover .caret,.open .caret{opacity:1;filter:alpha(opacity=100)}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:4px 0;margin:1px 0 0;list-style:none;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);*border-right-width:2px;*border-bottom-width:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{*width:100%;height:1px;margin:8px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:block;padding:3px 15px;clear:both;font-weight:normal;line-height:18px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#08c}.open{*z-index:1000}.open>.dropdown-menu{display:block}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #000;content:"\2191"}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}.typeahead{margin-top:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #eee;border:1px solid rgba(0,0,0,0.05);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);box-shadow:inset 0 1px 1px rgba(0,0,0,0.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,0.15)}.well-large{padding:24px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.well-small{padding:9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.fade{opacity:0;-webkit-transition:opacity .15s linear;-moz-transition:opacity .15s linear;-ms-transition:opacity .15s linear;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;-webkit-transition:height .35s ease;-moz-transition:height .35s ease;-ms-transition:height .35s ease;-o-transition:height .35s ease;transition:height .35s ease}.collapse.in{height:auto}.close{float:right;font-size:20px;font-weight:bold;line-height:18px;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover{color:#000;text-decoration:none;cursor:pointer;opacity:.4;filter:alpha(opacity=40)}button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none}.btn{display:inline-block;*display:inline;padding:4px 10px 4px;margin-bottom:0;*margin-left:.3em;font-size:13px;line-height:18px;*line-height:20px;color:#333;text-align:center;text-shadow:0 1px 1px rgba(255,255,255,0.75);vertical-align:middle;cursor:pointer;background-color:#f5f5f5;*background-color:#e6e6e6;background-image:-ms-linear-gradient(top,#fff,#e6e6e6);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e6e6e6));background-image:-webkit-linear-gradient(top,#fff,#e6e6e6);background-image:-o-linear-gradient(top,#fff,#e6e6e6);background-image:linear-gradient(top,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #ccc;*border:0;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-bottom-color:#b3b3b3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffff',endColorstr='#e6e6e6',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);*zoom:1;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn:hover,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{background-color:#e6e6e6;*background-color:#d9d9d9}.btn:active,.btn.active{background-color:#ccc \9}.btn:first-child{*margin-left:0}.btn:hover{color:#333;text-decoration:none;background-color:#e6e6e6;*background-color:#d9d9d9;background-position:0 -15px;-webkit-transition:background-position .1s linear;-moz-transition:background-position .1s linear;-ms-transition:background-position .1s linear;-o-transition:background-position .1s linear;transition:background-position .1s linear}.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn.active,.btn:active{background-color:#e6e6e6;background-color:#d9d9d9 \9;background-image:none;outline:0;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn.disabled,.btn[disabled]{cursor:default;background-color:#e6e6e6;background-image:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-large{padding:9px 14px;font-size:15px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:1px}.btn-small{padding:5px 9px;font-size:11px;line-height:16px}.btn-small [class^="icon-"]{margin-top:-1px}.btn-mini{padding:2px 6px;font-size:11px;line-height:14px}.btn-primary,.btn-primary:hover,.btn-warning,.btn-warning:hover,.btn-danger,.btn-danger:hover,.btn-success,.btn-success:hover,.btn-info,.btn-info:hover,.btn-inverse,.btn-inverse:hover{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.btn-primary.active,.btn-warning.active,.btn-danger.active,.btn-success.active,.btn-info.active,.btn-inverse.active{color:rgba(255,255,255,0.75)}.btn{border-color:#ccc;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25)}.btn-primary{background-color:#0074cc;*background-color:#05c;background-image:-ms-linear-gradient(top,#08c,#05c);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#05c));background-image:-webkit-linear-gradient(top,#08c,#05c);background-image:-o-linear-gradient(top,#08c,#05c);background-image:-moz-linear-gradient(top,#08c,#05c);background-image:linear-gradient(top,#08c,#05c);background-repeat:repeat-x;border-color:#05c #05c #003580;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#0088cc',endColorstr='#0055cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{background-color:#05c;*background-color:#004ab3}.btn-primary:active,.btn-primary.active{background-color:#004099 \9}.btn-warning{background-color:#faa732;*background-color:#f89406;background-image:-ms-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(top,#fbb450,#f89406);background-repeat:repeat-x;border-color:#f89406 #f89406 #ad6704;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fbb450',endColorstr='#f89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{background-color:#da4f49;*background-color:#bd362f;background-image:-ms-linear-gradient(top,#ee5f5b,#bd362f);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#bd362f));background-image:-webkit-linear-gradient(top,#ee5f5b,#bd362f);background-image:-o-linear-gradient(top,#ee5f5b,#bd362f);background-image:-moz-linear-gradient(top,#ee5f5b,#bd362f);background-image:linear-gradient(top,#ee5f5b,#bd362f);background-repeat:repeat-x;border-color:#bd362f #bd362f #802420;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ee5f5b',endColorstr='#bd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{background-color:#5bb75b;*background-color:#51a351;background-image:-ms-linear-gradient(top,#62c462,#51a351);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#51a351));background-image:-webkit-linear-gradient(top,#62c462,#51a351);background-image:-o-linear-gradient(top,#62c462,#51a351);background-image:-moz-linear-gradient(top,#62c462,#51a351);background-image:linear-gradient(top,#62c462,#51a351);background-repeat:repeat-x;border-color:#51a351 #51a351 #387038;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#62c462',endColorstr='#51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{background-color:#49afcd;*background-color:#2f96b4;background-image:-ms-linear-gradient(top,#5bc0de,#2f96b4);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#2f96b4));background-image:-webkit-linear-gradient(top,#5bc0de,#2f96b4);background-image:-o-linear-gradient(top,#5bc0de,#2f96b4);background-image:-moz-linear-gradient(top,#5bc0de,#2f96b4);background-image:linear-gradient(top,#5bc0de,#2f96b4);background-repeat:repeat-x;border-color:#2f96b4 #2f96b4 #1f6377;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#5bc0de',endColorstr='#2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{background-color:#414141;*background-color:#222;background-image:-ms-linear-gradient(top,#555,#222);background-image:-webkit-gradient(linear,0 0,0 100%,from(#555),to(#222));background-image:-webkit-linear-gradient(top,#555,#222);background-image:-o-linear-gradient(top,#555,#222);background-image:-moz-linear-gradient(top,#555,#222);background-image:linear-gradient(top,#555,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#555555',endColorstr='#222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:2px;*padding-bottom:2px}button.btn::-moz-focus-inner,input[type="submit"].btn::-moz-focus-inner{padding:0;border:0}button.btn.btn-large,input[type="submit"].btn.btn-large{*padding-top:7px;*padding-bottom:7px}button.btn.btn-small,input[type="submit"].btn.btn-small{*padding-top:3px;*padding-bottom:3px}button.btn.btn-mini,input[type="submit"].btn.btn-mini{*padding-top:1px;*padding-bottom:1px}.btn-group{position:relative;*margin-left:.3em;*zoom:1}.btn-group:before,.btn-group:after{display:table;content:""}.btn-group:after{clear:both}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:9px;margin-bottom:9px}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-group>.btn{position:relative;float:left;margin-left:-1px;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn:first-child{margin-left:0;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.btn-group>.btn:last-child,.btn-group>.dropdown-toggle{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.btn-group>.btn.large:first-child{margin-left:0;-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.btn-group>.btn.large:last-child,.btn-group>.large.dropdown-toggle{-webkit-border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.btn-group>.btn:hover,.btn-group>.btn:focus,.btn-group>.btn:active,.btn-group>.btn.active{z-index:2}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.dropdown-toggle{*padding-top:4px;padding-right:8px;*padding-bottom:4px;padding-left:8px;-webkit-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn-group>.btn-mini.dropdown-toggle{padding-right:5px;padding-left:5px}.btn-group>.btn-small.dropdown-toggle{*padding-top:4px;*padding-bottom:4px}.btn-group>.btn-large.dropdown-toggle{padding-right:12px;padding-left:12px}.btn-group.open .dropdown-toggle{background-image:none;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn-group.open .btn.dropdown-toggle{background-color:#e6e6e6}.btn-group.open .btn-primary.dropdown-toggle{background-color:#05c}.btn-group.open .btn-warning.dropdown-toggle{background-color:#f89406}.btn-group.open .btn-danger.dropdown-toggle{background-color:#bd362f}.btn-group.open .btn-success.dropdown-toggle{background-color:#51a351}.btn-group.open .btn-info.dropdown-toggle{background-color:#2f96b4}.btn-group.open .btn-inverse.dropdown-toggle{background-color:#222}.btn .caret{margin-top:7px;margin-left:0}.btn:hover .caret,.open.btn-group .caret{opacity:1;filter:alpha(opacity=100)}.btn-mini .caret{margin-top:5px}.btn-small .caret{margin-top:6px}.btn-large .caret{margin-top:6px;border-top-width:5px;border-right-width:5px;border-left-width:5px}.dropup .btn-large .caret{border-top:0;border-bottom:5px solid #000}.btn-primary .caret,.btn-warning .caret,.btn-danger .caret,.btn-info .caret,.btn-success .caret,.btn-inverse .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:.75;filter:alpha(opacity=75)}.alert{padding:8px 35px 8px 14px;margin-bottom:18px;color:#c09853;text-shadow:0 1px 0 rgba(255,255,255,0.5);background-color:#fcf8e3;border:1px solid #fbeed5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.alert-heading{color:inherit}.alert .close{position:relative;top:-2px;right:-21px;line-height:18px}.alert-success{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.alert-danger,.alert-error{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.alert-info{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.alert-block{padding-top:14px;padding-bottom:14px}.alert-block>p,.alert-block>ul{margin-bottom:0}.alert-block p+p{margin-top:5px}.nav{margin-bottom:18px;margin-left:0;list-style:none}.nav>li>a{display:block}.nav>li>a:hover{text-decoration:none;background-color:#eee}.nav>.pull-right{float:right}.nav .nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:18px;color:#999;text-shadow:0 1px 0 rgba(255,255,255,0.5);text-transform:uppercase}.nav li+.nav-header{margin-top:9px}.nav-list{padding-right:15px;padding-left:15px;margin-bottom:0}.nav-list>li>a,.nav-list .nav-header{margin-right:-15px;margin-left:-15px;text-shadow:0 1px 0 rgba(255,255,255,0.5)}.nav-list>li>a{padding:3px 15px}.nav-list>.active>a,.nav-list>.active>a:hover{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.2);background-color:#08c}.nav-list [class^="icon-"]{margin-right:2px}.nav-list .divider{*width:100%;height:1px;margin:8px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.nav-tabs,.nav-pills{*zoom:1}.nav-tabs:before,.nav-pills:before,.nav-tabs:after,.nav-pills:after{display:table;content:""}.nav-tabs:after,.nav-pills:after{clear:both}.nav-tabs>li,.nav-pills>li{float:left}.nav-tabs>li>a,.nav-pills>li>a{padding-right:12px;padding-left:12px;margin-right:2px;line-height:14px}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{margin-bottom:-1px}.nav-tabs>li>a{padding-top:8px;padding-bottom:8px;line-height:18px;border:1px solid transparent;-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover{border-color:#eee #eee #ddd}.nav-tabs>.active>a,.nav-tabs>.active>a:hover{color:#555;cursor:default;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent}.nav-pills>li>a{padding-top:8px;padding-bottom:8px;margin-top:2px;margin-bottom:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.nav-pills>.active>a,.nav-pills>.active>a:hover{color:#fff;background-color:#08c}.nav-stacked>li{float:none}.nav-stacked>li>a{margin-right:0}.nav-tabs.nav-stacked{border-bottom:0}.nav-tabs.nav-stacked>li>a{border:1px solid #ddd;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.nav-tabs.nav-stacked>li:first-child>a{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.nav-tabs.nav-stacked>li>a:hover{z-index:2;border-color:#ddd}.nav-pills.nav-stacked>li>a{margin-bottom:3px}.nav-pills.nav-stacked>li:last-child>a{margin-bottom:1px}.nav-tabs .dropdown-menu{-webkit-border-radius:0 0 5px 5px;-moz-border-radius:0 0 5px 5px;border-radius:0 0 5px 5px}.nav-pills .dropdown-menu{-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.nav-tabs .dropdown-toggle .caret,.nav-pills .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav-tabs .dropdown-toggle:hover .caret,.nav-pills .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .active .dropdown-toggle .caret,.nav-pills .active .dropdown-toggle .caret{border-top-color:#333;border-bottom-color:#333}.nav>.dropdown.active>a:hover{color:#000;cursor:pointer}.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover{color:#fff;background-color:#999;border-color:#999}.nav li.dropdown.open .caret,.nav li.dropdown.open.active .caret,.nav li.dropdown.open a:hover .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:1;filter:alpha(opacity=100)}.tabs-stacked .open>a:hover{border-color:#999}.tabbable{*zoom:1}.tabbable:before,.tabbable:after{display:table;content:""}.tabbable:after{clear:both}.tab-content{overflow:auto}.tabs-below>.nav-tabs,.tabs-right>.nav-tabs,.tabs-left>.nav-tabs{border-bottom:0}.tab-content>.tab-pane,.pill-content>.pill-pane{display:none}.tab-content>.active,.pill-content>.active{display:block}.tabs-below>.nav-tabs{border-top:1px solid #ddd}.tabs-below>.nav-tabs>li{margin-top:-1px;margin-bottom:0}.tabs-below>.nav-tabs>li>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.tabs-below>.nav-tabs>li>a:hover{border-top-color:#ddd;border-bottom-color:transparent}.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover{border-color:transparent #ddd #ddd #ddd}.tabs-left>.nav-tabs>li,.tabs-right>.nav-tabs>li{float:none}.tabs-left>.nav-tabs>li>a,.tabs-right>.nav-tabs>li>a{min-width:74px;margin-right:0;margin-bottom:3px}.tabs-left>.nav-tabs{float:left;margin-right:19px;border-right:1px solid #ddd}.tabs-left>.nav-tabs>li>a{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.tabs-left>.nav-tabs>li>a:hover{border-color:#eee #ddd #eee #eee}.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover{border-color:#ddd transparent #ddd #ddd;*border-right-color:#fff}.tabs-right>.nav-tabs{float:right;margin-left:19px;border-left:1px solid #ddd}.tabs-right>.nav-tabs>li>a{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.tabs-right>.nav-tabs>li>a:hover{border-color:#eee #eee #eee #ddd}.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover{border-color:#ddd #ddd #ddd transparent;*border-left-color:#fff}.navbar{*position:relative;*z-index:2;margin-bottom:18px;overflow:visible}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#2c2c2c;background-image:-moz-linear-gradient(top,#333,#222);background-image:-ms-linear-gradient(top,#333,#222);background-image:-webkit-gradient(linear,0 0,0 100%,from(#333),to(#222));background-image:-webkit-linear-gradient(top,#333,#222);background-image:-o-linear-gradient(top,#333,#222);background-image:linear-gradient(top,#333,#222);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#333333',endColorstr='#222222',GradientType=0);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.25),inset 0 -1px 0 rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.25),inset 0 -1px 0 rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.25),inset 0 -1px 0 rgba(0,0,0,0.1)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar{color:#999}.navbar .brand:hover{text-decoration:none}.navbar .brand{display:block;float:left;padding:8px 20px 12px;margin-left:-20px;font-size:20px;font-weight:200;line-height:1;color:#999}.navbar .navbar-text{margin-bottom:0;line-height:40px}.navbar .navbar-link{color:#999}.navbar .navbar-link:hover{color:#fff}.navbar .btn,.navbar .btn-group{margin-top:5px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;content:""}.navbar-form:after{clear:both}.navbar-form input,.navbar-form select,.navbar-form .radio,.navbar-form .checkbox{margin-top:5px}.navbar-form input,.navbar-form select{display:inline-block;margin-bottom:0}.navbar-form input[type="image"],.navbar-form input[type="checkbox"],.navbar-form input[type="radio"]{margin-top:3px}.navbar-form .input-append,.navbar-form .input-prepend{margin-top:6px;white-space:nowrap}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0}.navbar-search{position:relative;float:left;margin-top:6px;margin-bottom:0}.navbar-search .search-query{padding:4px 9px;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;color:#fff;background-color:#626262;border:1px solid #151515;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-webkit-transition:none;-moz-transition:none;-ms-transition:none;-o-transition:none;transition:none}.navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-search .search-query:focus,.navbar-search .search-query.focused{padding:5px 10px;color:#333;text-shadow:0 1px 0 #fff;background-color:#fff;border:0;outline:0;-webkit-box-shadow:0 0 3px rgba(0,0,0,0.15);-moz-box-shadow:0 0 3px rgba(0,0,0,0.15);box-shadow:0 0 3px rgba(0,0,0,0.15)}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-bottom:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding-right:0;padding-left:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-bottom{bottom:0}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{display:block;float:left}.navbar .nav>li>a{float:none;padding:9px 10px 11px;line-height:19px;color:#999;text-decoration:none;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar .btn{display:inline-block;padding:4px 10px 4px;margin:5px 5px 6px;line-height:18px}.navbar .btn-group{padding:5px 5px 6px;margin:0}.navbar .nav>li>a:hover{color:#fff;text-decoration:none;background-color:transparent}.navbar .nav .active>a,.navbar .nav .active>a:hover{color:#fff;text-decoration:none;background-color:#222}.navbar .divider-vertical{width:1px;height:40px;margin:0 9px;overflow:hidden;background-color:#222;border-right:1px solid #333}.navbar .nav.pull-right{margin-right:0;margin-left:10px}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;background-color:#2c2c2c;*background-color:#222;background-image:-ms-linear-gradient(top,#333,#222);background-image:-webkit-gradient(linear,0 0,0 100%,from(#333),to(#222));background-image:-webkit-linear-gradient(top,#333,#222);background-image:-o-linear-gradient(top,#333,#222);background-image:linear-gradient(top,#333,#222);background-image:-moz-linear-gradient(top,#333,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#333333',endColorstr='#222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075)}.navbar .btn-navbar:hover,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{background-color:#222;*background-color:#151515}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#080808 \9}.navbar .btn-navbar .icon-bar{display:block;width:18px;height:2px;background-color:#f5f5f5;-webkit-border-radius:1px;-moz-border-radius:1px;border-radius:1px;-webkit-box-shadow:0 1px 0 rgba(0,0,0,0.25);-moz-box-shadow:0 1px 0 rgba(0,0,0,0.25);box-shadow:0 1px 0 rgba(0,0,0,0.25)}.btn-navbar .icon-bar+.icon-bar{margin-top:3px}.navbar .dropdown-menu:before{position:absolute;top:-7px;left:9px;display:inline-block;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-left:7px solid transparent;border-bottom-color:rgba(0,0,0,0.2);content:''}.navbar .dropdown-menu:after{position:absolute;top:-6px;left:10px;display:inline-block;border-right:6px solid transparent;border-bottom:6px solid #fff;border-left:6px solid transparent;content:''}.navbar-fixed-bottom .dropdown-menu:before{top:auto;bottom:-7px;border-top:7px solid #ccc;border-bottom:0;border-top-color:rgba(0,0,0,0.2)}.navbar-fixed-bottom .dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown .dropdown-toggle .caret,.navbar .nav li.dropdown.open .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar .nav li.dropdown.active .caret{opacity:1;filter:alpha(opacity=100)}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{background-color:transparent}.navbar .nav li.dropdown.active>.dropdown-toggle:hover{color:#fff}.navbar .pull-right .dropdown-menu,.navbar .dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right .dropdown-menu:before,.navbar .dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right .dropdown-menu:after,.navbar .dropdown-menu.pull-right:after{right:13px;left:auto}.breadcrumb{padding:7px 14px;margin:0 0 18px;list-style:none;background-color:#fbfbfb;background-image:-moz-linear-gradient(top,#fff,#f5f5f5);background-image:-ms-linear-gradient(top,#fff,#f5f5f5);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#f5f5f5));background-image:-webkit-linear-gradient(top,#fff,#f5f5f5);background-image:-o-linear-gradient(top,#fff,#f5f5f5);background-image:linear-gradient(top,#fff,#f5f5f5);background-repeat:repeat-x;border:1px solid #ddd;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffff',endColorstr='#f5f5f5',GradientType=0);-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#999}.breadcrumb .active a{color:#333}.pagination{height:36px;margin:18px 0}.pagination ul{display:inline-block;*display:inline;margin-bottom:0;margin-left:0;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;*zoom:1;-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:0 1px 2px rgba(0,0,0,0.05);box-shadow:0 1px 2px rgba(0,0,0,0.05)}.pagination li{display:inline}.pagination a{float:left;padding:0 14px;line-height:34px;text-decoration:none;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a{background-color:#f5f5f5}.pagination .active a{color:#999;cursor:default}.pagination .disabled span,.pagination .disabled a,.pagination .disabled a:hover{color:#999;cursor:default;background-color:transparent}.pagination li:first-child a{border-left-width:1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.pagination li:last-child a{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.pagination-centered{text-align:center}.pagination-right{text-align:right}.pager{margin-bottom:18px;margin-left:0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;content:""}.pager:after{clear:both}.pager li{display:inline}.pager a{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.pager a:hover{text-decoration:none;background-color:#f5f5f5}.pager .next a{float:right}.pager .previous a{float:left}.pager .disabled a,.pager .disabled a:hover{color:#999;cursor:default;background-color:#fff}.modal-open .dropdown-menu{z-index:2050}.modal-open .dropdown.open{*z-index:2050}.modal-open .popover{z-index:2060}.modal-open .tooltip{z-index:2070}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000}.modal-backdrop.fade{opacity:0}.modal-backdrop,.modal-backdrop.fade.in{opacity:.8;filter:alpha(opacity=80)}.modal{position:fixed;top:50%;left:50%;z-index:1050;width:560px;margin:-250px 0 0 -280px;overflow:auto;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,0.3);*border:1px solid #999;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3);-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box}.modal.fade{top:-25%;-webkit-transition:opacity .3s linear,top .3s ease-out;-moz-transition:opacity .3s linear,top .3s ease-out;-ms-transition:opacity .3s linear,top .3s ease-out;-o-transition:opacity .3s linear,top .3s ease-out;transition:opacity .3s linear,top .3s ease-out}.modal.fade.in{top:50%}.modal-header{padding:9px 15px;border-bottom:1px solid #eee}.modal-header .close{margin-top:2px}.modal-body{max-height:400px;padding:15px;overflow-y:auto}.modal-form{margin-bottom:0}.modal-footer{padding:14px 15px 15px;margin-bottom:0;text-align:right;background-color:#f5f5f5;border-top:1px solid #ddd;-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;*zoom:1;-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.modal-footer:before,.modal-footer:after{display:table;content:""}.modal-footer:after{clear:both}.modal-footer .btn+.btn{margin-bottom:0;margin-left:5px}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.tooltip{position:absolute;z-index:1020;display:block;padding:5px;font-size:11px;opacity:0;filter:alpha(opacity=0);visibility:visible}.tooltip.in{opacity:.8;filter:alpha(opacity=80)}.tooltip.top{margin-top:-2px}.tooltip.right{margin-left:2px}.tooltip.bottom{margin-top:2px}.tooltip.left{margin-left:-2px}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top:5px solid #000;border-right:5px solid transparent;border-left:5px solid transparent}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-top:5px solid transparent;border-bottom:5px solid transparent;border-left:5px solid #000}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-right:5px solid transparent;border-bottom:5px solid #000;border-left:5px solid transparent}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-top:5px solid transparent;border-right:5px solid #000;border-bottom:5px solid transparent}.tooltip-inner{max-width:200px;padding:3px 8px;color:#fff;text-align:center;text-decoration:none;background-color:#000;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.tooltip-arrow{position:absolute;width:0;height:0}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;padding:5px}.popover.top{margin-top:-5px}.popover.right{margin-left:5px}.popover.bottom{margin-top:5px}.popover.left{margin-left:-5px}.popover.top .arrow{bottom:0;left:50%;margin-left:-5px;border-top:5px solid #000;border-right:5px solid transparent;border-left:5px solid transparent}.popover.right .arrow{top:50%;left:0;margin-top:-5px;border-top:5px solid transparent;border-right:5px solid #000;border-bottom:5px solid transparent}.popover.bottom .arrow{top:0;left:50%;margin-left:-5px;border-right:5px solid transparent;border-bottom:5px solid #000;border-left:5px solid transparent}.popover.left .arrow{top:50%;right:0;margin-top:-5px;border-top:5px solid transparent;border-bottom:5px solid transparent;border-left:5px solid #000}.popover .arrow{position:absolute;width:0;height:0}.popover-inner{width:280px;padding:3px;overflow:hidden;background:#000;background:rgba(0,0,0,0.8);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3)}.popover-title{padding:9px 15px;line-height:1;background-color:#f5f5f5;border-bottom:1px solid #eee;-webkit-border-radius:3px 3px 0 0;-moz-border-radius:3px 3px 0 0;border-radius:3px 3px 0 0}.popover-content{padding:14px;background-color:#fff;-webkit-border-radius:0 0 3px 3px;-moz-border-radius:0 0 3px 3px;border-radius:0 0 3px 3px;-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:18px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:1;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:0 1px 1px rgba(0,0,0,0.075);box-shadow:0 1px 1px rgba(0,0,0,0.075)}a.thumbnail:hover{border-color:#08c;-webkit-box-shadow:0 1px 4px rgba(0,105,214,0.25);-moz-box-shadow:0 1px 4px rgba(0,105,214,0.25);box-shadow:0 1px 4px rgba(0,105,214,0.25)}.thumbnail>img{display:block;max-width:100%;margin-right:auto;margin-left:auto}.thumbnail .caption{padding:9px}.label,.badge{font-size:10.998px;font-weight:bold;line-height:14px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);white-space:nowrap;vertical-align:baseline;background-color:#999}.label{padding:1px 4px 2px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.badge{padding:1px 9px 2px;-webkit-border-radius:9px;-moz-border-radius:9px;border-radius:9px}a.label:hover,a.badge:hover{color:#fff;text-decoration:none;cursor:pointer}.label-important,.badge-important{background-color:#b94a48}.label-important[href],.badge-important[href]{background-color:#953b39}.label-warning,.badge-warning{background-color:#f89406}.label-warning[href],.badge-warning[href]{background-color:#c67605}.label-success,.badge-success{background-color:#468847}.label-success[href],.badge-success[href]{background-color:#356635}.label-info,.badge-info{background-color:#3a87ad}.label-info[href],.badge-info[href]{background-color:#2d6987}.label-inverse,.badge-inverse{background-color:#333}.label-inverse[href],.badge-inverse[href]{background-color:#1a1a1a}@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-moz-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-ms-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-o-keyframes progress-bar-stripes{from{background-position:0 0}to{background-position:40px 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}.progress{height:18px;margin-bottom:18px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-ms-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f5f5f5),to(#f9f9f9));background-image:-webkit-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-o-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:linear-gradient(top,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#f5f5f5',endColorstr='#f9f9f9',GradientType=0);-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1)}.progress .bar{width:0;height:18px;font-size:12px;color:#fff;text-align:center;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e90d2;background-image:-moz-linear-gradient(top,#149bdf,#0480be);background-image:-webkit-gradient(linear,0 0,0 100%,from(#149bdf),to(#0480be));background-image:-webkit-linear-gradient(top,#149bdf,#0480be);background-image:-o-linear-gradient(top,#149bdf,#0480be);background-image:linear-gradient(top,#149bdf,#0480be);background-image:-ms-linear-gradient(top,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#149bdf',endColorstr='#0480be',GradientType=0);-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-ms-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress-striped .bar{background-color:#149bdf;background-image:-o-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-webkit-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-ms-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);-webkit-background-size:40px 40px;-moz-background-size:40px 40px;-o-background-size:40px 40px;background-size:40px 40px}.progress.active .bar{-webkit-animation:progress-bar-stripes 2s linear infinite;-moz-animation:progress-bar-stripes 2s linear infinite;-ms-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite}.progress-danger .bar{background-color:#dd514c;background-image:-moz-linear-gradient(top,#ee5f5b,#c43c35);background-image:-ms-linear-gradient(top,#ee5f5b,#c43c35);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#c43c35));background-image:-webkit-linear-gradient(top,#ee5f5b,#c43c35);background-image:-o-linear-gradient(top,#ee5f5b,#c43c35);background-image:linear-gradient(top,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ee5f5b',endColorstr='#c43c35',GradientType=0)}.progress-danger.progress-striped .bar{background-color:#ee5f5b;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-ms-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-success .bar{background-color:#5eb95e;background-image:-moz-linear-gradient(top,#62c462,#57a957);background-image:-ms-linear-gradient(top,#62c462,#57a957);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#57a957));background-image:-webkit-linear-gradient(top,#62c462,#57a957);background-image:-o-linear-gradient(top,#62c462,#57a957);background-image:linear-gradient(top,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#62c462',endColorstr='#57a957',GradientType=0)}.progress-success.progress-striped .bar{background-color:#62c462;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-ms-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-info .bar{background-color:#4bb1cf;background-image:-moz-linear-gradient(top,#5bc0de,#339bb9);background-image:-ms-linear-gradient(top,#5bc0de,#339bb9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#339bb9));background-image:-webkit-linear-gradient(top,#5bc0de,#339bb9);background-image:-o-linear-gradient(top,#5bc0de,#339bb9);background-image:linear-gradient(top,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#5bc0de',endColorstr='#339bb9',GradientType=0)}.progress-info.progress-striped .bar{background-color:#5bc0de;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-ms-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-warning .bar{background-color:#faa732;background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:-ms-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(top,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fbb450',endColorstr='#f89406',GradientType=0)}.progress-warning.progress-striped .bar{background-color:#fbb450;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-ms-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.accordion{margin-bottom:18px}.accordion-group{margin-bottom:2px;border:1px solid #e5e5e5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.accordion-heading{border-bottom:0}.accordion-heading .accordion-toggle{display:block;padding:8px 15px}.accordion-toggle{cursor:pointer}.accordion-inner{padding:9px 15px;border-top:1px solid #e5e5e5}.carousel{position:relative;margin-bottom:18px;line-height:1}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel .item{position:relative;display:none;-webkit-transition:.6s ease-in-out left;-moz-transition:.6s ease-in-out left;-ms-transition:.6s ease-in-out left;-o-transition:.6s ease-in-out left;transition:.6s ease-in-out left}.carousel .item>img{display:block;line-height:1}.carousel .active,.carousel .next,.carousel .prev{display:block}.carousel .active{left:0}.carousel .next,.carousel .prev{position:absolute;top:0;width:100%}.carousel .next{left:100%}.carousel .prev{left:-100%}.carousel .next.left,.carousel .prev.right{left:0}.carousel .active.left{left:-100%}.carousel .active.right{left:100%}.carousel-control{position:absolute;top:40%;left:15px;width:40px;height:40px;margin-top:-20px;font-size:60px;font-weight:100;line-height:30px;color:#fff;text-align:center;background:#222;border:3px solid #fff;-webkit-border-radius:23px;-moz-border-radius:23px;border-radius:23px;opacity:.5;filter:alpha(opacity=50)}.carousel-control.right{right:15px;left:auto}.carousel-control:hover{color:#fff;text-decoration:none;opacity:.9;filter:alpha(opacity=90)}.carousel-caption{position:absolute;right:0;bottom:0;left:0;padding:10px 15px 5px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{color:#fff}.hero-unit{padding:60px;margin-bottom:30px;background-color:#eee;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.hero-unit h1{margin-bottom:0;font-size:60px;line-height:1;letter-spacing:-1px;color:inherit}.hero-unit p{font-size:18px;font-weight:200;line-height:27px;color:inherit}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden} diff --git a/core/src/main/resources/spark/deploy/static/sorttable.js b/core/src/main/resources/spark/deploy/static/sorttable.js deleted file mode 100644 index 7abb9011cc..0000000000 --- a/core/src/main/resources/spark/deploy/static/sorttable.js +++ /dev/null @@ -1,495 +0,0 @@ -/* - SortTable - version 2 - 7th April 2007 - Stuart Langridge, http://www.kryogenix.org/code/browser/sorttable/ - - Instructions: - Download this file - Add to your HTML - Add class="sortable" to any table you'd like to make sortable - Click on the headers to sort - - Thanks to many, many people for contributions and suggestions. - Licenced as X11: http://www.kryogenix.org/code/browser/licence.html - This basically means: do what you want with it. -*/ - - -var stIsIE = /*@cc_on!@*/false; - -sorttable = { - init: function() { - // quit if this function has already been called - if (arguments.callee.done) return; - // flag this function so we don't do the same thing twice - arguments.callee.done = true; - // kill the timer - if (_timer) clearInterval(_timer); - - if (!document.createElement || !document.getElementsByTagName) return; - - sorttable.DATE_RE = /^(\d\d?)[\/\.-](\d\d?)[\/\.-]((\d\d)?\d\d)$/; - - forEach(document.getElementsByTagName('table'), function(table) { - if (table.className.search(/\bsortable\b/) != -1) { - sorttable.makeSortable(table); - } - }); - - }, - - makeSortable: function(table) { - if (table.getElementsByTagName('thead').length == 0) { - // table doesn't have a tHead. Since it should have, create one and - // put the first table row in it. - the = document.createElement('thead'); - the.appendChild(table.rows[0]); - table.insertBefore(the,table.firstChild); - } - // Safari doesn't support table.tHead, sigh - if (table.tHead == null) table.tHead = table.getElementsByTagName('thead')[0]; - - if (table.tHead.rows.length != 1) return; // can't cope with two header rows - - // Sorttable v1 put rows with a class of "sortbottom" at the bottom (as - // "total" rows, for example). This is B&R, since what you're supposed - // to do is put them in a tfoot. So, if there are sortbottom rows, - // for backwards compatibility, move them to tfoot (creating it if needed). - sortbottomrows = []; - for (var i=0; i5' : ' ▴'; - this.appendChild(sortrevind); - return; - } - if (this.className.search(/\bsorttable_sorted_reverse\b/) != -1) { - // if we're already sorted by this column in reverse, just - // re-reverse the table, which is quicker - sorttable.reverse(this.sorttable_tbody); - this.className = this.className.replace('sorttable_sorted_reverse', - 'sorttable_sorted'); - this.removeChild(document.getElementById('sorttable_sortrevind')); - sortfwdind = document.createElement('span'); - sortfwdind.id = "sorttable_sortfwdind"; - sortfwdind.innerHTML = stIsIE ? ' 6' : ' ▾'; - this.appendChild(sortfwdind); - return; - } - - // remove sorttable_sorted classes - theadrow = this.parentNode; - forEach(theadrow.childNodes, function(cell) { - if (cell.nodeType == 1) { // an element - cell.className = cell.className.replace('sorttable_sorted_reverse',''); - cell.className = cell.className.replace('sorttable_sorted',''); - } - }); - sortfwdind = document.getElementById('sorttable_sortfwdind'); - if (sortfwdind) { sortfwdind.parentNode.removeChild(sortfwdind); } - sortrevind = document.getElementById('sorttable_sortrevind'); - if (sortrevind) { sortrevind.parentNode.removeChild(sortrevind); } - - this.className += ' sorttable_sorted'; - sortfwdind = document.createElement('span'); - sortfwdind.id = "sorttable_sortfwdind"; - sortfwdind.innerHTML = stIsIE ? ' 6' : ' ▾'; - this.appendChild(sortfwdind); - - // build an array to sort. This is a Schwartzian transform thing, - // i.e., we "decorate" each row with the actual sort key, - // sort based on the sort keys, and then put the rows back in order - // which is a lot faster because you only do getInnerText once per row - row_array = []; - col = this.sorttable_columnindex; - rows = this.sorttable_tbody.rows; - for (var j=0; j 12) { - // definitely dd/mm - return sorttable.sort_ddmm; - } else if (second > 12) { - return sorttable.sort_mmdd; - } else { - // looks like a date, but we can't tell which, so assume - // that it's dd/mm (English imperialism!) and keep looking - sortfn = sorttable.sort_ddmm; - } - } - } - } - return sortfn; - }, - - getInnerText: function(node) { - // gets the text we want to use for sorting for a cell. - // strips leading and trailing whitespace. - // this is *not* a generic getInnerText function; it's special to sorttable. - // for example, you can override the cell text with a customkey attribute. - // it also gets .value for fields. - - if (!node) return ""; - - hasInputs = (typeof node.getElementsByTagName == 'function') && - node.getElementsByTagName('input').length; - - if (node.getAttribute("sorttable_customkey") != null) { - return node.getAttribute("sorttable_customkey"); - } - else if (typeof node.textContent != 'undefined' && !hasInputs) { - return node.textContent.replace(/^\s+|\s+$/g, ''); - } - else if (typeof node.innerText != 'undefined' && !hasInputs) { - return node.innerText.replace(/^\s+|\s+$/g, ''); - } - else if (typeof node.text != 'undefined' && !hasInputs) { - return node.text.replace(/^\s+|\s+$/g, ''); - } - else { - switch (node.nodeType) { - case 3: - if (node.nodeName.toLowerCase() == 'input') { - return node.value.replace(/^\s+|\s+$/g, ''); - } - case 4: - return node.nodeValue.replace(/^\s+|\s+$/g, ''); - break; - case 1: - case 11: - var innerText = ''; - for (var i = 0; i < node.childNodes.length; i++) { - innerText += sorttable.getInnerText(node.childNodes[i]); - } - return innerText.replace(/^\s+|\s+$/g, ''); - break; - default: - return ''; - } - } - }, - - reverse: function(tbody) { - // reverse the rows in a tbody - newrows = []; - for (var i=0; i=0; i--) { - tbody.appendChild(newrows[i]); - } - delete newrows; - }, - - /* sort functions - each sort function takes two parameters, a and b - you are comparing a[0] and b[0] */ - sort_numeric: function(a,b) { - aa = parseFloat(a[0].replace(/[^0-9.-]/g,'')); - if (isNaN(aa)) aa = 0; - bb = parseFloat(b[0].replace(/[^0-9.-]/g,'')); - if (isNaN(bb)) bb = 0; - return aa-bb; - }, - sort_alpha: function(a,b) { - if (a[0]==b[0]) return 0; - if (a[0] 0 ) { - var q = list[i]; list[i] = list[i+1]; list[i+1] = q; - swap = true; - } - } // for - t--; - - if (!swap) break; - - for(var i = t; i > b; --i) { - if ( comp_func(list[i], list[i-1]) < 0 ) { - var q = list[i]; list[i] = list[i-1]; list[i-1] = q; - swap = true; - } - } // for - b++; - - } // while(swap) - } -} - -/* ****************************************************************** - Supporting functions: bundled here to avoid depending on a library - ****************************************************************** */ - -// Dean Edwards/Matthias Miller/John Resig - -/* for Mozilla/Opera9 */ -if (document.addEventListener) { - document.addEventListener("DOMContentLoaded", sorttable.init, false); -} - -/* for Internet Explorer */ -/*@cc_on @*/ -/*@if (@_win32) - document.write(" to your HTML + Add class="sortable" to any table you'd like to make sortable + Click on the headers to sort + + Thanks to many, many people for contributions and suggestions. + Licenced as X11: http://www.kryogenix.org/code/browser/licence.html + This basically means: do what you want with it. +*/ + + +var stIsIE = /*@cc_on!@*/false; + +sorttable = { + init: function() { + // quit if this function has already been called + if (arguments.callee.done) return; + // flag this function so we don't do the same thing twice + arguments.callee.done = true; + // kill the timer + if (_timer) clearInterval(_timer); + + if (!document.createElement || !document.getElementsByTagName) return; + + sorttable.DATE_RE = /^(\d\d?)[\/\.-](\d\d?)[\/\.-]((\d\d)?\d\d)$/; + + forEach(document.getElementsByTagName('table'), function(table) { + if (table.className.search(/\bsortable\b/) != -1) { + sorttable.makeSortable(table); + } + }); + + }, + + makeSortable: function(table) { + if (table.getElementsByTagName('thead').length == 0) { + // table doesn't have a tHead. Since it should have, create one and + // put the first table row in it. + the = document.createElement('thead'); + the.appendChild(table.rows[0]); + table.insertBefore(the,table.firstChild); + } + // Safari doesn't support table.tHead, sigh + if (table.tHead == null) table.tHead = table.getElementsByTagName('thead')[0]; + + if (table.tHead.rows.length != 1) return; // can't cope with two header rows + + // Sorttable v1 put rows with a class of "sortbottom" at the bottom (as + // "total" rows, for example). This is B&R, since what you're supposed + // to do is put them in a tfoot. So, if there are sortbottom rows, + // for backwards compatibility, move them to tfoot (creating it if needed). + sortbottomrows = []; + for (var i=0; i5' : ' ▴'; + this.appendChild(sortrevind); + return; + } + if (this.className.search(/\bsorttable_sorted_reverse\b/) != -1) { + // if we're already sorted by this column in reverse, just + // re-reverse the table, which is quicker + sorttable.reverse(this.sorttable_tbody); + this.className = this.className.replace('sorttable_sorted_reverse', + 'sorttable_sorted'); + this.removeChild(document.getElementById('sorttable_sortrevind')); + sortfwdind = document.createElement('span'); + sortfwdind.id = "sorttable_sortfwdind"; + sortfwdind.innerHTML = stIsIE ? ' 6' : ' ▾'; + this.appendChild(sortfwdind); + return; + } + + // remove sorttable_sorted classes + theadrow = this.parentNode; + forEach(theadrow.childNodes, function(cell) { + if (cell.nodeType == 1) { // an element + cell.className = cell.className.replace('sorttable_sorted_reverse',''); + cell.className = cell.className.replace('sorttable_sorted',''); + } + }); + sortfwdind = document.getElementById('sorttable_sortfwdind'); + if (sortfwdind) { sortfwdind.parentNode.removeChild(sortfwdind); } + sortrevind = document.getElementById('sorttable_sortrevind'); + if (sortrevind) { sortrevind.parentNode.removeChild(sortrevind); } + + this.className += ' sorttable_sorted'; + sortfwdind = document.createElement('span'); + sortfwdind.id = "sorttable_sortfwdind"; + sortfwdind.innerHTML = stIsIE ? ' 6' : ' ▾'; + this.appendChild(sortfwdind); + + // build an array to sort. This is a Schwartzian transform thing, + // i.e., we "decorate" each row with the actual sort key, + // sort based on the sort keys, and then put the rows back in order + // which is a lot faster because you only do getInnerText once per row + row_array = []; + col = this.sorttable_columnindex; + rows = this.sorttable_tbody.rows; + for (var j=0; j 12) { + // definitely dd/mm + return sorttable.sort_ddmm; + } else if (second > 12) { + return sorttable.sort_mmdd; + } else { + // looks like a date, but we can't tell which, so assume + // that it's dd/mm (English imperialism!) and keep looking + sortfn = sorttable.sort_ddmm; + } + } + } + } + return sortfn; + }, + + getInnerText: function(node) { + // gets the text we want to use for sorting for a cell. + // strips leading and trailing whitespace. + // this is *not* a generic getInnerText function; it's special to sorttable. + // for example, you can override the cell text with a customkey attribute. + // it also gets .value for fields. + + if (!node) return ""; + + hasInputs = (typeof node.getElementsByTagName == 'function') && + node.getElementsByTagName('input').length; + + if (node.getAttribute("sorttable_customkey") != null) { + return node.getAttribute("sorttable_customkey"); + } + else if (typeof node.textContent != 'undefined' && !hasInputs) { + return node.textContent.replace(/^\s+|\s+$/g, ''); + } + else if (typeof node.innerText != 'undefined' && !hasInputs) { + return node.innerText.replace(/^\s+|\s+$/g, ''); + } + else if (typeof node.text != 'undefined' && !hasInputs) { + return node.text.replace(/^\s+|\s+$/g, ''); + } + else { + switch (node.nodeType) { + case 3: + if (node.nodeName.toLowerCase() == 'input') { + return node.value.replace(/^\s+|\s+$/g, ''); + } + case 4: + return node.nodeValue.replace(/^\s+|\s+$/g, ''); + break; + case 1: + case 11: + var innerText = ''; + for (var i = 0; i < node.childNodes.length; i++) { + innerText += sorttable.getInnerText(node.childNodes[i]); + } + return innerText.replace(/^\s+|\s+$/g, ''); + break; + default: + return ''; + } + } + }, + + reverse: function(tbody) { + // reverse the rows in a tbody + newrows = []; + for (var i=0; i=0; i--) { + tbody.appendChild(newrows[i]); + } + delete newrows; + }, + + /* sort functions + each sort function takes two parameters, a and b + you are comparing a[0] and b[0] */ + sort_numeric: function(a,b) { + aa = parseFloat(a[0].replace(/[^0-9.-]/g,'')); + if (isNaN(aa)) aa = 0; + bb = parseFloat(b[0].replace(/[^0-9.-]/g,'')); + if (isNaN(bb)) bb = 0; + return aa-bb; + }, + sort_alpha: function(a,b) { + if (a[0]==b[0]) return 0; + if (a[0] 0 ) { + var q = list[i]; list[i] = list[i+1]; list[i+1] = q; + swap = true; + } + } // for + t--; + + if (!swap) break; + + for(var i = t; i > b; --i) { + if ( comp_func(list[i], list[i-1]) < 0 ) { + var q = list[i]; list[i] = list[i-1]; list[i-1] = q; + swap = true; + } + } // for + b++; + + } // while(swap) + } +} + +/* ****************************************************************** + Supporting functions: bundled here to avoid depending on a library + ****************************************************************** */ + +// Dean Edwards/Matthias Miller/John Resig + +/* for Mozilla/Opera9 */ +if (document.addEventListener) { + document.addEventListener("DOMContentLoaded", sorttable.init, false); +} + +/* for Internet Explorer */ +/*@cc_on @*/ +/*@if (@_win32) + document.write(" + {title} + + + +
    +
    +
    + +

    + {title} +

    +
    +
    + {content} +
    + + + } +} diff --git a/core/src/main/scala/spark/util/WebUI.scala b/core/src/main/scala/spark/util/WebUI.scala deleted file mode 100644 index bde1acdbca..0000000000 --- a/core/src/main/scala/spark/util/WebUI.scala +++ /dev/null @@ -1,107 +0,0 @@ -package spark.util - -import annotation.tailrec -import javax.servlet.http.{HttpServletResponse, HttpServletRequest} -import net.liftweb.json._ -import org.eclipse.jetty.server.{Server, Request, Handler} -import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler} -import spark.Logging -import util.{Try, Success, Failure} -import xml.Node - -object WebUI extends Logging { - type Responder[T] = HttpServletRequest => T - - implicit def jsonResponderToHandler(responder: Responder[JValue]): Handler = - createHandler(responder, "text/json", (in: JValue) => pretty(render(in))) - - implicit def htmlResponderToHandler(responder: Responder[Seq[Node]]): Handler = - createHandler(responder, "text/html") - - implicit def textResponderToHandler(responder: Responder[String]): Handler = - createHandler(responder, "text/plain") - - def createHandler[T <% AnyRef](responder: Responder[T], contentType: String, - extractFn: T => String = (in: Any) => in.toString): Handler = { - new AbstractHandler { - def handle(target: String, - baseRequest: Request, - request: HttpServletRequest, - response: HttpServletResponse) { - response.setContentType("%s;charset=utf-8".format(contentType)) - response.setStatus(HttpServletResponse.SC_OK) - baseRequest.setHandled(true) - val result = responder(request) - response.getWriter().println(extractFn(result)) - } - } - } - - def createStaticHandler(resourceBase: String): ResourceHandler = { - val staticHandler = new ResourceHandler - Option(getClass.getClassLoader.getResource(resourceBase)) match { - case Some(res) => - staticHandler.setResourceBase (res.toString) - staticHandler - } - } - - def startJettyServer(ip: String, port: Int, handlers: Array[(String, Handler)]): (Server, Int) = { - val handlersToRegister = handlers.map { case(path, handler) => - if (path == "*") { - handler - } else { - val contextHandler = new ContextHandler(path) - contextHandler.setHandler(handler) - contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] - } - } - - val handlerList = new HandlerList - handlerList.setHandlers(handlersToRegister) - - @tailrec - def connect(currentPort: Int): (Server, Int) = { - val server = new Server(port) - server.setHandler(handlerList) - Try { server.start() } match { - case s: Success[_] => (server, currentPort) - case f: Failure[_] => - logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) - connect((currentPort + 1) % 65536) - } - } - - connect(port) - } - - def makePage(content: => Seq[Node], title: String): Seq[Node] = { - - - - - - - {title} - - - -
    -
    -
    - -

    - {title} -

    -
    -
    -
    - {content} -
    - - - } -} -- cgit v1.2.3 From bc4a811c57ee126559721a3a4284510baef184d1 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 19 Jun 2013 12:56:01 -0700 Subject: Stash --- core/src/main/scala/spark/ui/JobProgressUI.scala | 14 ++++++++++++++ 1 file changed, 14 insertions(+) create mode 100644 core/src/main/scala/spark/ui/JobProgressUI.scala diff --git a/core/src/main/scala/spark/ui/JobProgressUI.scala b/core/src/main/scala/spark/ui/JobProgressUI.scala new file mode 100644 index 0000000000..eb782d85c8 --- /dev/null +++ b/core/src/main/scala/spark/ui/JobProgressUI.scala @@ -0,0 +1,14 @@ +package spark.ui + +import spark.SparkContext +import spark.scheduler.SparkListener + +private[spark] +class JobProgressUI(sc: SparkContext) { + sc.addSparkListener() +} + +class JobProgressListener extends SparkListener { + +} + -- cgit v1.2.3 From 9fd5dc3ea9016b59667fd903992843605087e43b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 20 Jun 2013 09:02:56 -0700 Subject: Initial steps towards job progress UI --- core/src/main/scala/spark/SparkContext.scala | 8 +- .../main/scala/spark/scheduler/DAGScheduler.scala | 1 + core/src/main/scala/spark/scheduler/Stage.scala | 1 + core/src/main/scala/spark/ui/JobProgressUI.scala | 178 ++++++++++++++++++++- core/src/main/scala/spark/ui/SparkUI.scala | 2 +- core/src/main/scala/spark/ui/WebUI.scala | 3 +- 6 files changed, 181 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index cc634b3ec4..155a5ee721 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -94,10 +94,6 @@ class SparkContext( isLocal) SparkEnv.set(env) - // Start the Spark UI - private[spark] val ui = new SparkUI(this) - ui.start() - // Used to store a URL for each static file/jar together with the file's local timestamp private[spark] val addedFiles = HashMap[String, Long]() private[spark] val addedJars = HashMap[String, Long]() @@ -215,6 +211,10 @@ class SparkContext( @volatile private var dagScheduler = new DAGScheduler(taskScheduler) dagScheduler.start() + // Start the Spark UI + private[spark] val ui = new SparkUI(this) + ui.start() + /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = { val conf = SparkHadoopUtil.newConfiguration() diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index f7d60be5db..bdd8792ce9 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -503,6 +503,7 @@ class DAGScheduler( case _ => "Unkown" } logInfo("%s (%s) finished in %s s".format(stage, stage.origin, serviceTime)) + stage.completionTime = Some(System.currentTimeMillis) val stageComp = StageCompleted(stageToInfos(stage)) sparkListeners.foreach{_.onStageCompleted(stageComp)} running -= stage diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala index 7fc9e13fd9..539cf8233b 100644 --- a/core/src/main/scala/spark/scheduler/Stage.scala +++ b/core/src/main/scala/spark/scheduler/Stage.scala @@ -34,6 +34,7 @@ private[spark] class Stage( /** When first task was submitted to scheduler. */ var submissionTime: Option[Long] = None + var completionTime: Option[Long] = None private var nextAttemptId = 0 diff --git a/core/src/main/scala/spark/ui/JobProgressUI.scala b/core/src/main/scala/spark/ui/JobProgressUI.scala index eb782d85c8..1f12df10b8 100644 --- a/core/src/main/scala/spark/ui/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/JobProgressUI.scala @@ -1,14 +1,182 @@ package spark.ui -import spark.SparkContext -import spark.scheduler.SparkListener +import spark.{Utils, SparkContext} +import spark.scheduler._ +import spark.scheduler.SparkListenerTaskEnd +import spark.scheduler.StageCompleted +import spark.scheduler.SparkListenerStageSubmitted +import org.eclipse.jetty.server.Handler +import javax.servlet.http.HttpServletRequest +import xml.Node +import WebUI._ +import collection.mutable._ +import spark.Success +import akka.util.Duration +import java.text.SimpleDateFormat +import java.util.Date +import spark.scheduler.cluster.TaskInfo +import collection.mutable +import org.hsqldb.lib.HashMappedList +import spark.executor.TaskMetrics +import spark.scheduler.SparkListenerTaskEnd +import scala.Some +import spark.scheduler.SparkListenerStageSubmitted +import scala.Seq +import spark.scheduler.StageCompleted +import spark.scheduler.SparkListenerJobStart private[spark] -class JobProgressUI(sc: SparkContext) { - sc.addSparkListener() +class JobProgressUI(sc: SparkContext) extends UIComponent { + val listener = new JobProgressListener + val fmt = new SimpleDateFormat("EEE, MMM d yyyy HH:mm:ss") + + sc.addSparkListener(listener) + + def getHandlers = Seq[(String, Handler)]( + ("/stages/stage", (request: HttpServletRequest) => stagePage(request)), + ("/stages", (request: HttpServletRequest) => indexPage) + ) + + def stagePage(request: HttpServletRequest): Seq[Node] = { + val stageId = request.getParameter("id").toInt + val content = +

    Percentile Metrics

    + + + + + + + + + + {listener.stageToTaskInfos(stageId).map{ case(i, m) => taskRow(i, m) }} + +
    Service TimeRemote Bytes ReadShuffle Bytes Written
    +

    Tasks

    + + + + + + + + + + + + {listener.stageToTaskInfos(stageId).map{ case(i, m) => taskRow(i, m) }} + +
    Task IDService Time (ms)Locality levelWorkerLaunch Time
    ; + WebUI.headerSparkPage(content, "Stage Details: %s".format(stageId)) + } + + + + def taskRow(info: TaskInfo, metrics: TaskMetrics): Seq[Node] = { + + {info.taskId} + {metrics.executorRunTime} + {info.taskLocality} + {info.hostPort} + {fmt.format(new Date(info.launchTime))} + + } + + def indexPage: Seq[Node] = { + val content = +

    Active Stages

    + + + + + + + + + + + + {listener.activeStages.map(stageRow)} + +
    Stage IDOriginSubmittedDurationTasks: Complete/Total
    +

    Completed Stages

    + + + + + + + + + + + + {listener.completedStages.map(stageRow)} + +
    Stage IDOriginSubmittedDurationTasks: Complete/Total
    ; + + WebUI.headerSparkPage(content, "Spark Stages") + } + + def getElapsedTime(submitted: Option[Long], completed: Long): String = { + submitted match { + case Some(t) => Duration(completed - t, "milliseconds").printHMS + case _ => "Unknown" + } + } + + def stageRow(s: Stage): Seq[Node] = { + val submissionTime = s.submissionTime match { + case Some(t) => fmt.format(new Date(t)) + case None => "Unknown" + } + + {s.id} + {s.origin} + {submissionTime} + {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))} + {listener.stageToTasksComplete.getOrElse(s.id, 0)} / {s.numPartitions} + {listener.stageToTasksFailed.getOrElse(s.id, 0) match { + case f if f > 0 => "(%s failed)".format(f) + case _ => + }} + + + } } -class JobProgressListener extends SparkListener { +private[spark] class JobProgressListener extends SparkListener { + val activeStages = HashSet[Stage]() + val stageToTasksComplete = HashMap[Int, Int]() + val stageToTasksFailed = HashMap[Int, Int]() + val stageToTaskInfos = HashMap[Int, ArrayBuffer[(TaskInfo, TaskMetrics)]]() + val completedStages = ListBuffer[Stage]() // Todo (pwendell): Evict these over time + + override def onJobStart(jobStart: SparkListenerJobStart) { } + + override def onStageCompleted(stageCompleted: StageCompleted) = { + val stage = stageCompleted.stageInfo.stage + activeStages -= stage + stage +=: completedStages + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = + activeStages += stageSubmitted.stage + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val sid = taskEnd.event.task.stageId + taskEnd.event.reason match { + case Success => + stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 + case _ => + stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 + } + val taskList = stageToTaskInfos.getOrElse(sid, ArrayBuffer[(TaskInfo, TaskMetrics)]()) + taskList += ((taskEnd.event.taskInfo, taskEnd.event.taskMetrics)) + stageToTaskInfos(sid) = taskList + } + override def onJobEnd(jobEnd: SparkListenerEvents) { } } diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 6c9affc1f7..63b75df36f 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -14,7 +14,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), ("*", (request: HttpServletRequest) => WebUI.headerSparkPage(

    Test

    , "Test page")) ) - val components = Seq(new BlockManagerUI(sc)) + val components = Seq(new BlockManagerUI(sc), new JobProgressUI(sc)) def start() { /** Start an HTTP server to run the Web interface */ diff --git a/core/src/main/scala/spark/ui/WebUI.scala b/core/src/main/scala/spark/ui/WebUI.scala index 16251e727a..e80d48dd4a 100644 --- a/core/src/main/scala/spark/ui/WebUI.scala +++ b/core/src/main/scala/spark/ui/WebUI.scala @@ -58,7 +58,6 @@ object WebUI extends Logging { handler } else { val contextHandler = new ContextHandler(path) - println("Adding handler for path: " + path) contextHandler.setHandler(handler) contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] } @@ -84,7 +83,7 @@ object WebUI extends Logging { /** Page with Spark logo, title, and Spark UI headers */ def headerSparkPage(content: => Seq[Node], title: String): Seq[Node] = { val newContent = -

    Storage | Jobs

    ; +

    Storage | Jobs

    ; sparkPage(newContent ++ content, title) } -- cgit v1.2.3 From ce81c320acfd265b8de7c30a46c6a3adc4f39a74 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 20 Jun 2013 10:45:52 -0700 Subject: Adding helper function to make listing tables --- core/src/main/scala/spark/ui/BlockManagerUI.scala | 155 +++++++++------------- core/src/main/scala/spark/ui/JobProgressUI.scala | 71 +++------- core/src/main/scala/spark/ui/WebUI.scala | 12 ++ 3 files changed, 97 insertions(+), 141 deletions(-) diff --git a/core/src/main/scala/spark/ui/BlockManagerUI.scala b/core/src/main/scala/spark/ui/BlockManagerUI.scala index f319751590..3be5064837 100644 --- a/core/src/main/scala/spark/ui/BlockManagerUI.scala +++ b/core/src/main/scala/spark/ui/BlockManagerUI.scala @@ -1,14 +1,15 @@ package spark.ui -import akka.actor.{ActorRef, ActorSystem} import akka.util.Duration import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler -import spark.{Logging, SparkContext} -import spark.Utils -import WebUI._ +import spark.{RDD, Logging, SparkContext, Utils} +import spark.ui.WebUI._ import xml.Node import spark.storage.StorageUtils +import spark.storage.StorageStatus +import spark.storage.RDDInfo +import spark.storage.BlockManagerMasterActor.BlockStatus /** * Web UI server for the BlockManager inside each SparkContext. @@ -33,6 +34,14 @@ class BlockManagerUI(sc: SparkContext) filterStorageStatusByPrefix(storageStatusList, prefix) val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head + val workerHeaders = Seq("Host", "Memory Usage", "Disk Usage") + val workers = filteredStorageStatusList.map((prefix, _)) + val workerTable = listingTable(workerHeaders, workerRow, workers) + + val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk") + val blocks = filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1) + val blockTable = listingTable(blockHeaders, blockRow, blocks) + val content =
    @@ -64,67 +73,38 @@ class BlockManagerUI(sc: SparkContext)

    RDD Summary

    -
    - - - - - - - - - - - {filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1).map { - case (k,v) => - - - - - - - } - } - -
    Block NameStorage LevelSize in MemorySize on Disk
    {k} - {v.storageLevel.description} - {Utils.memoryBytesToString(v.memSize)}{Utils.memoryBytesToString(v.diskSize)}
    +
    {blockTable}
    -
    -
    -
    -

    Worker Summary

    -
    - - - - - - - - - - {filteredStorageStatusList.map { - status => - - - - - - } - } - -
    HostMemory UsageDisk Usage
    {status.blockManagerId.host + ":" + status.blockManagerId.port} - {Utils.memoryBytesToString(status.memUsed(prefix))} - ({Utils.memoryBytesToString(status.memRemaining)} Total Available) - {Utils.memoryBytesToString(status.diskUsed(prefix))}
    -
    -
    ; +
    ++ {workerTable}; WebUI.headerSparkPage(content, "RDD Info: " + id) } + def blockRow(blk: (String, BlockStatus)): Seq[Node] = { + val (id, block) = blk + + {id} + + {block.storageLevel.description} + + {Utils.memoryBytesToString(block.memSize)} + {Utils.memoryBytesToString(block.diskSize)} + + } + + def workerRow(worker: (String, StorageStatus)): Seq[Node] = { + val (prefix, status) = worker + + {status.blockManagerId.host + ":" + status.blockManagerId.port} + + {Utils.memoryBytesToString(status.memUsed(prefix))} + ({Utils.memoryBytesToString(status.memRemaining)} Total Available) + + {Utils.memoryBytesToString(status.diskUsed(prefix))} + + } + def indexPage: Seq[Node] = { val storageStatusList = sc.getExecutorStorageStatus // Calculate macro-level statistics @@ -132,7 +112,16 @@ class BlockManagerUI(sc: SparkContext) val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) .reduceOption(_+_).getOrElse(0L) + + val rddHeaders = Seq( + "RDD Name", + "Storage Level", + "Cached Partitions", + "Fraction Partitions Cached", + "Size in Memory", + "Size on Disk") val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) + val rddTable = listingTable(rddHeaders, rddRow, rdds) val content =
    @@ -144,38 +133,24 @@ class BlockManagerUI(sc: SparkContext)
  • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
  • -
    -
    - - - - - - - - - - - - - {for (rdd <- rdds) yield - - - - - - - - - } - -
    RDD NameStorage LevelCached PartitionsFraction Partitions CachedSize in MemorySize on Disk
    - - {rdd.name} - - {rdd.storageLevel.description} - {rdd.numCachedPartitions}{rdd.numCachedPartitions / rdd.numPartitions.toDouble}{Utils.memoryBytesToString(rdd.memSize)}{Utils.memoryBytesToString(rdd.diskSize)}
    ; +
    ++ {rddTable}; WebUI.headerSparkPage(content, "Spark Storage ") } + + def rddRow(rdd: RDDInfo): Seq[Node] = { + + + + {rdd.name} + + + {rdd.storageLevel.description} + + {rdd.numCachedPartitions} + {rdd.numCachedPartitions / rdd.numPartitions.toDouble} + {Utils.memoryBytesToString(rdd.memSize)} + {Utils.memoryBytesToString(rdd.diskSize)} + + } } diff --git a/core/src/main/scala/spark/ui/JobProgressUI.scala b/core/src/main/scala/spark/ui/JobProgressUI.scala index 1f12df10b8..b51e62bb1a 100644 --- a/core/src/main/scala/spark/ui/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/JobProgressUI.scala @@ -39,6 +39,11 @@ class JobProgressUI(sc: SparkContext) extends UIComponent { def stagePage(request: HttpServletRequest): Seq[Node] = { val stageId = request.getParameter("id").toInt + + val taskHeaders = Seq("Task ID", "Service Time (ms)", "Locality Level", "Worker", "Launch Time") + val tasks = listener.stageToTaskInfos(stageId) + val taskTable = listingTable(taskHeaders, taskRow, tasks) + val content =

    Percentile Metrics

    @@ -53,27 +58,13 @@ class JobProgressUI(sc: SparkContext) extends UIComponent { {listener.stageToTaskInfos(stageId).map{ case(i, m) => taskRow(i, m) }}
    -

    Tasks

    - - - - - - - - - - - - {listener.stageToTaskInfos(stageId).map{ case(i, m) => taskRow(i, m) }} - -
    Task IDService Time (ms)Locality levelWorkerLaunch Time
    ; +

    Tasks

    ++ {taskTable}; + WebUI.headerSparkPage(content, "Stage Details: %s".format(stageId)) } - - - def taskRow(info: TaskInfo, metrics: TaskMetrics): Seq[Node] = { + def taskRow(taskData: (TaskInfo, TaskMetrics)): Seq[Node] = { + val (info, metrics) = taskData {info.taskId} {metrics.executorRunTime} @@ -84,37 +75,16 @@ class JobProgressUI(sc: SparkContext) extends UIComponent { } def indexPage: Seq[Node] = { + val stageHeaders = Seq("Stage ID", "Origin", "Submitted", "Duration", "Tasks: Complete/Total") + val activeStages = listener.activeStages.toSeq + val completedStages = listener.completedStages.toSeq + + val activeStageTable = listingTable(stageHeaders, stageRow, activeStages) + val completedStageTable = listingTable(stageHeaders, stageRow, completedStages) + val content = -

    Active Stages

    - - - - - - - - - - - - {listener.activeStages.map(stageRow)} - -
    Stage IDOriginSubmittedDurationTasks: Complete/Total
    -

    Completed Stages

    - - - - - - - - - - - - {listener.completedStages.map(stageRow)} - -
    Stage IDOriginSubmittedDurationTasks: Complete/Total
    ; +

    Active Stages

    ++ {activeStageTable} +

    Completed Stages

    ++ {completedStageTable} WebUI.headerSparkPage(content, "Spark Stages") } @@ -139,7 +109,7 @@ class JobProgressUI(sc: SparkContext) extends UIComponent { {listener.stageToTasksComplete.getOrElse(s.id, 0)} / {s.numPartitions} {listener.stageToTasksFailed.getOrElse(s.id, 0) match { case f if f > 0 => "(%s failed)".format(f) - case _ => + case _ => }} @@ -178,5 +148,4 @@ private[spark] class JobProgressListener extends SparkListener { } override def onJobEnd(jobEnd: SparkListenerEvents) { } -} - +} \ No newline at end of file diff --git a/core/src/main/scala/spark/ui/WebUI.scala b/core/src/main/scala/spark/ui/WebUI.scala index e80d48dd4a..265624a726 100644 --- a/core/src/main/scala/spark/ui/WebUI.scala +++ b/core/src/main/scala/spark/ui/WebUI.scala @@ -14,6 +14,7 @@ abstract class UIComponent { } object WebUI extends Logging { + // CORE WEB UI COMPONENTS type Responder[T] = HttpServletRequest => T implicit def jsonResponderToHandler(responder: Responder[JValue]): Handler = @@ -80,6 +81,7 @@ object WebUI extends Logging { connect(port) } + // HELPER FUNCTIONS AND SHORTCUTS /** Page with Spark logo, title, and Spark UI headers */ def headerSparkPage(content: => Seq[Node], title: String): Seq[Node] = { val newContent = @@ -117,4 +119,14 @@ object WebUI extends Logging { } + + /** Shortcut for making a table derived from a sequence of objects. */ + def listingTable[T](headers: Seq[String], makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { + + {headers.map(h => "".format(h))} + + {rows.map(r => makeRow(r))} + +
    %s
    + } } -- cgit v1.2.3 From dcf6a68177912ffa69ee75345c8172fd2db9e0ea Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 20 Jun 2013 12:05:49 -0700 Subject: Refactoring into different modules --- core/src/main/scala/spark/SparkContext.scala | 2 +- core/src/main/scala/spark/ui/BlockManagerUI.scala | 156 --------------------- core/src/main/scala/spark/ui/JobProgressUI.scala | 151 -------------------- core/src/main/scala/spark/ui/SparkUI.scala | 2 + core/src/main/scala/spark/ui/WebUI.scala | 7 +- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 55 ++++++++ .../main/scala/spark/ui/jobs/JobProgressUI.scala | 82 +++++++++++ core/src/main/scala/spark/ui/jobs/StagePage.scala | 63 +++++++++ .../scala/spark/ui/storage/BlockManagerUI.scala | 26 ++++ .../main/scala/spark/ui/storage/IndexPage.scala | 61 ++++++++ core/src/main/scala/spark/ui/storage/RDDPage.scala | 94 +++++++++++++ 11 files changed, 390 insertions(+), 309 deletions(-) delete mode 100644 core/src/main/scala/spark/ui/BlockManagerUI.scala delete mode 100644 core/src/main/scala/spark/ui/JobProgressUI.scala create mode 100644 core/src/main/scala/spark/ui/jobs/IndexPage.scala create mode 100644 core/src/main/scala/spark/ui/jobs/JobProgressUI.scala create mode 100644 core/src/main/scala/spark/ui/jobs/StagePage.scala create mode 100644 core/src/main/scala/spark/ui/storage/BlockManagerUI.scala create mode 100644 core/src/main/scala/spark/ui/storage/IndexPage.scala create mode 100644 core/src/main/scala/spark/ui/storage/RDDPage.scala diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 155a5ee721..901cda4174 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -48,7 +48,7 @@ import spark.scheduler.local.LocalScheduler import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import spark.storage.{StorageStatus, StorageUtils, RDDInfo} import spark.util.{MetadataCleaner, TimeStampedHashMap} -import ui.{SparkUI, BlockManagerUI} +import ui.{SparkUI} /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark diff --git a/core/src/main/scala/spark/ui/BlockManagerUI.scala b/core/src/main/scala/spark/ui/BlockManagerUI.scala deleted file mode 100644 index 3be5064837..0000000000 --- a/core/src/main/scala/spark/ui/BlockManagerUI.scala +++ /dev/null @@ -1,156 +0,0 @@ -package spark.ui - -import akka.util.Duration -import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.Handler -import spark.{RDD, Logging, SparkContext, Utils} -import spark.ui.WebUI._ -import xml.Node -import spark.storage.StorageUtils -import spark.storage.StorageStatus -import spark.storage.RDDInfo -import spark.storage.BlockManagerMasterActor.BlockStatus - -/** - * Web UI server for the BlockManager inside each SparkContext. - */ -private[spark] -class BlockManagerUI(sc: SparkContext) - extends UIComponent with Logging { - implicit val timeout = Duration.create( - System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") - - - def getHandlers = Seq[(String, Handler)]( - ("/storage/rdd", (request: HttpServletRequest) => rddPage(request)), - ("/storage", (request: HttpServletRequest) => indexPage) - ) - - def rddPage(request: HttpServletRequest): Seq[Node] = { - val id = request.getParameter("id") - val prefix = "rdd_" + id.toString - val storageStatusList = sc.getExecutorStorageStatus - val filteredStorageStatusList = StorageUtils. - filterStorageStatusByPrefix(storageStatusList, prefix) - val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head - - val workerHeaders = Seq("Host", "Memory Usage", "Disk Usage") - val workers = filteredStorageStatusList.map((prefix, _)) - val workerTable = listingTable(workerHeaders, workerRow, workers) - - val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk") - val blocks = filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1) - val blockTable = listingTable(blockHeaders, blockRow, blocks) - - val content = -
    -
    -
      -
    • - Storage Level: - {rddInfo.storageLevel.description} -
    • -
    • - Cached Partitions: - {rddInfo.numCachedPartitions} -
    • -
    • - Total Partitions: - {rddInfo.numPartitions} -
    • -
    • - Memory Size: - {Utils.memoryBytesToString(rddInfo.memSize)} -
    • -
    • - Disk Size: - {Utils.memoryBytesToString(rddInfo.diskSize)} -
    • -
    -
    -
    -
    -
    -
    -

    RDD Summary

    -
    {blockTable} -
    -
    -
    ++ {workerTable}; - - WebUI.headerSparkPage(content, "RDD Info: " + id) - } - - def blockRow(blk: (String, BlockStatus)): Seq[Node] = { - val (id, block) = blk - - {id} - - {block.storageLevel.description} - - {Utils.memoryBytesToString(block.memSize)} - {Utils.memoryBytesToString(block.diskSize)} - - } - - def workerRow(worker: (String, StorageStatus)): Seq[Node] = { - val (prefix, status) = worker - - {status.blockManagerId.host + ":" + status.blockManagerId.port} - - {Utils.memoryBytesToString(status.memUsed(prefix))} - ({Utils.memoryBytesToString(status.memRemaining)} Total Available) - - {Utils.memoryBytesToString(status.diskUsed(prefix))} - - } - - def indexPage: Seq[Node] = { - val storageStatusList = sc.getExecutorStorageStatus - // Calculate macro-level statistics - val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) - .reduceOption(_+_).getOrElse(0L) - - val rddHeaders = Seq( - "RDD Name", - "Storage Level", - "Cached Partitions", - "Fraction Partitions Cached", - "Size in Memory", - "Size on Disk") - val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) - val rddTable = listingTable(rddHeaders, rddRow, rdds) - - val content = -
    -
    -
      -
    • Memory: - {Utils.memoryBytesToString(maxMem - remainingMem)} Used - ({Utils.memoryBytesToString(remainingMem)} Available)
    • -
    • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
    • -
    -
    -
    ++ {rddTable}; - - WebUI.headerSparkPage(content, "Spark Storage ") - } - - def rddRow(rdd: RDDInfo): Seq[Node] = { - - - - {rdd.name} - - - {rdd.storageLevel.description} - - {rdd.numCachedPartitions} - {rdd.numCachedPartitions / rdd.numPartitions.toDouble} - {Utils.memoryBytesToString(rdd.memSize)} - {Utils.memoryBytesToString(rdd.diskSize)} - - } -} diff --git a/core/src/main/scala/spark/ui/JobProgressUI.scala b/core/src/main/scala/spark/ui/JobProgressUI.scala deleted file mode 100644 index b51e62bb1a..0000000000 --- a/core/src/main/scala/spark/ui/JobProgressUI.scala +++ /dev/null @@ -1,151 +0,0 @@ -package spark.ui - -import spark.{Utils, SparkContext} -import spark.scheduler._ -import spark.scheduler.SparkListenerTaskEnd -import spark.scheduler.StageCompleted -import spark.scheduler.SparkListenerStageSubmitted -import org.eclipse.jetty.server.Handler -import javax.servlet.http.HttpServletRequest -import xml.Node -import WebUI._ -import collection.mutable._ -import spark.Success -import akka.util.Duration -import java.text.SimpleDateFormat -import java.util.Date -import spark.scheduler.cluster.TaskInfo -import collection.mutable -import org.hsqldb.lib.HashMappedList -import spark.executor.TaskMetrics -import spark.scheduler.SparkListenerTaskEnd -import scala.Some -import spark.scheduler.SparkListenerStageSubmitted -import scala.Seq -import spark.scheduler.StageCompleted -import spark.scheduler.SparkListenerJobStart - -private[spark] -class JobProgressUI(sc: SparkContext) extends UIComponent { - val listener = new JobProgressListener - val fmt = new SimpleDateFormat("EEE, MMM d yyyy HH:mm:ss") - - sc.addSparkListener(listener) - - def getHandlers = Seq[(String, Handler)]( - ("/stages/stage", (request: HttpServletRequest) => stagePage(request)), - ("/stages", (request: HttpServletRequest) => indexPage) - ) - - def stagePage(request: HttpServletRequest): Seq[Node] = { - val stageId = request.getParameter("id").toInt - - val taskHeaders = Seq("Task ID", "Service Time (ms)", "Locality Level", "Worker", "Launch Time") - val tasks = listener.stageToTaskInfos(stageId) - val taskTable = listingTable(taskHeaders, taskRow, tasks) - - val content = -

    Percentile Metrics

    - - - - - - - - - - {listener.stageToTaskInfos(stageId).map{ case(i, m) => taskRow(i, m) }} - -
    Service TimeRemote Bytes ReadShuffle Bytes Written
    -

    Tasks

    ++ {taskTable}; - - WebUI.headerSparkPage(content, "Stage Details: %s".format(stageId)) - } - - def taskRow(taskData: (TaskInfo, TaskMetrics)): Seq[Node] = { - val (info, metrics) = taskData - - {info.taskId} - {metrics.executorRunTime} - {info.taskLocality} - {info.hostPort} - {fmt.format(new Date(info.launchTime))} - - } - - def indexPage: Seq[Node] = { - val stageHeaders = Seq("Stage ID", "Origin", "Submitted", "Duration", "Tasks: Complete/Total") - val activeStages = listener.activeStages.toSeq - val completedStages = listener.completedStages.toSeq - - val activeStageTable = listingTable(stageHeaders, stageRow, activeStages) - val completedStageTable = listingTable(stageHeaders, stageRow, completedStages) - - val content = -

    Active Stages

    ++ {activeStageTable} -

    Completed Stages

    ++ {completedStageTable} - - WebUI.headerSparkPage(content, "Spark Stages") - } - - def getElapsedTime(submitted: Option[Long], completed: Long): String = { - submitted match { - case Some(t) => Duration(completed - t, "milliseconds").printHMS - case _ => "Unknown" - } - } - - def stageRow(s: Stage): Seq[Node] = { - val submissionTime = s.submissionTime match { - case Some(t) => fmt.format(new Date(t)) - case None => "Unknown" - } - - {s.id} - {s.origin} - {submissionTime} - {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))} - {listener.stageToTasksComplete.getOrElse(s.id, 0)} / {s.numPartitions} - {listener.stageToTasksFailed.getOrElse(s.id, 0) match { - case f if f > 0 => "(%s failed)".format(f) - case _ => - }} - - - } -} - -private[spark] class JobProgressListener extends SparkListener { - val activeStages = HashSet[Stage]() - val stageToTasksComplete = HashMap[Int, Int]() - val stageToTasksFailed = HashMap[Int, Int]() - val stageToTaskInfos = HashMap[Int, ArrayBuffer[(TaskInfo, TaskMetrics)]]() - val completedStages = ListBuffer[Stage]() // Todo (pwendell): Evict these over time - - override def onJobStart(jobStart: SparkListenerJobStart) { } - - override def onStageCompleted(stageCompleted: StageCompleted) = { - val stage = stageCompleted.stageInfo.stage - activeStages -= stage - stage +=: completedStages - } - - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = - activeStages += stageSubmitted.stage - - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val sid = taskEnd.event.task.stageId - taskEnd.event.reason match { - case Success => - stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 - case _ => - stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 - } - val taskList = stageToTaskInfos.getOrElse(sid, ArrayBuffer[(TaskInfo, TaskMetrics)]()) - taskList += ((taskEnd.event.taskInfo, taskEnd.event.taskMetrics)) - stageToTaskInfos(sid) = taskList - } - - override def onJobEnd(jobEnd: SparkListenerEvents) { } -} \ No newline at end of file diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 63b75df36f..a1f6cc60ec 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -1,8 +1,10 @@ package spark.ui +import jobs.JobProgressUI import spark.{Logging, SparkContext, Utils} import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler +import storage.BlockManagerUI import WebUI._ private[spark] class SparkUI(sc: SparkContext) extends Logging { diff --git a/core/src/main/scala/spark/ui/WebUI.scala b/core/src/main/scala/spark/ui/WebUI.scala index 265624a726..6dedd28fc1 100644 --- a/core/src/main/scala/spark/ui/WebUI.scala +++ b/core/src/main/scala/spark/ui/WebUI.scala @@ -13,6 +13,10 @@ abstract class UIComponent { def getHandlers(): Seq[(String, Handler)] } +abstract class View[T] { + def render(request: HttpServletRequest): T +} + object WebUI extends Logging { // CORE WEB UI COMPONENTS type Responder[T] = HttpServletRequest => T @@ -82,6 +86,7 @@ object WebUI extends Logging { } // HELPER FUNCTIONS AND SHORTCUTS + /** Page with Spark logo, title, and Spark UI headers */ def headerSparkPage(content: => Seq[Node], title: String): Seq[Node] = { val newContent = @@ -123,7 +128,7 @@ object WebUI extends Logging { /** Shortcut for making a table derived from a sequence of objects. */ def listingTable[T](headers: Seq[String], makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { - {headers.map(h => "".format(h))} + {headers.map(h => )} {rows.map(r => makeRow(r))} diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala new file mode 100644 index 0000000000..1740524f49 --- /dev/null +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -0,0 +1,55 @@ +package spark.ui.jobs + +import spark.ui.{WebUI, View} +import xml.{NodeSeq, Node} +import spark.ui.WebUI._ +import scala.Some +import akka.util.Duration +import spark.scheduler.Stage +import java.util.Date +import javax.servlet.http.HttpServletRequest + +class IndexPage(parent: JobProgressUI) extends View[Seq[Node]] { + val listener = parent.listener + val dateFmt = parent.dateFmt + + def render(request: HttpServletRequest): Seq[Node] = { + val stageHeaders = Seq("Stage ID", "Origin", "Submitted", "Duration", "Tasks: Complete/Total") + val activeStages = listener.activeStages.toSeq + val completedStages = listener.completedStages.toSeq + + val activeStageTable: NodeSeq = listingTable(stageHeaders, stageRow, activeStages) + val completedStageTable = listingTable(stageHeaders, stageRow, completedStages) + + val content =

    Active Stages

    ++ activeStageTable ++ +

    Completed Stages

    ++ completedStageTable + + WebUI.headerSparkPage(content, "Spark Stages") + } + + def getElapsedTime(submitted: Option[Long], completed: Long): String = { + submitted match { + case Some(t) => Duration(completed - t, "milliseconds").printHMS + case _ => "Unknown" + } + } + + def stageRow(s: Stage): Seq[Node] = { + val submissionTime = s.submissionTime match { + case Some(t) => dateFmt.format(new Date(t)) + case None => "Unknown" + } + + + + + + + + } +} diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala new file mode 100644 index 0000000000..9ee962bf2e --- /dev/null +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -0,0 +1,82 @@ +package spark.ui.jobs + +import spark.{Utils, SparkContext} +import spark.scheduler._ +import spark.scheduler.SparkListenerTaskEnd +import spark.scheduler.StageCompleted +import spark.scheduler.SparkListenerStageSubmitted +import org.eclipse.jetty.server.Handler +import javax.servlet.http.HttpServletRequest +import xml.Node +import collection.mutable._ +import spark.Success +import akka.util.Duration +import java.text.SimpleDateFormat +import java.util.Date +import spark.scheduler.cluster.TaskInfo +import collection.mutable +import org.hsqldb.lib.HashMappedList +import spark.executor.TaskMetrics +import spark.scheduler.SparkListenerTaskEnd +import scala.Some +import spark.scheduler.SparkListenerStageSubmitted +import scala.Seq +import spark.scheduler.StageCompleted +import spark.scheduler.SparkListenerJobStart +import spark.ui.{WebUI, UIComponent} +import spark.ui.WebUI._ +import spark.scheduler.SparkListenerTaskEnd +import scala.Some +import spark.scheduler.SparkListenerStageSubmitted +import spark.scheduler.StageCompleted +import spark.scheduler.SparkListenerJobStart + +private[spark] +class JobProgressUI(sc: SparkContext) extends UIComponent { + val listener = new JobProgressListener + val dateFmt = new SimpleDateFormat("EEE, MMM d yyyy HH:mm:ss") + + sc.addSparkListener(listener) + + val indexPage = new IndexPage(this) + val stagePage = new StagePage(this) + + def getHandlers = Seq[(String, Handler)]( + ("/stages/stage", (request: HttpServletRequest) => stagePage.render(request)), + ("/stages", (request: HttpServletRequest) => indexPage.render(request)) + ) +} + +private[spark] class JobProgressListener extends SparkListener { + val activeStages = HashSet[Stage]() + val stageToTasksComplete = HashMap[Int, Int]() + val stageToTasksFailed = HashMap[Int, Int]() + val stageToTaskInfos = HashMap[Int, ArrayBuffer[(TaskInfo, TaskMetrics)]]() + val completedStages = ListBuffer[Stage]() // Todo (pwendell): Evict these over time + + override def onJobStart(jobStart: SparkListenerJobStart) { } + + override def onStageCompleted(stageCompleted: StageCompleted) = { + val stage = stageCompleted.stageInfo.stage + activeStages -= stage + stage +=: completedStages + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = + activeStages += stageSubmitted.stage + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val sid = taskEnd.event.task.stageId + taskEnd.event.reason match { + case Success => + stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 + case _ => + stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 + } + val taskList = stageToTaskInfos.getOrElse(sid, ArrayBuffer[(TaskInfo, TaskMetrics)]()) + taskList += ((taskEnd.event.taskInfo, taskEnd.event.taskMetrics)) + stageToTaskInfos(sid) = taskList + } + + override def onJobEnd(jobEnd: SparkListenerEvents) { } +} \ No newline at end of file diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala new file mode 100644 index 0000000000..49f5b1c73c --- /dev/null +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -0,0 +1,63 @@ +package spark.ui.jobs + +import javax.servlet.http.HttpServletRequest +import xml.Node +import spark.ui.WebUI._ +import spark.ui.WebUI +import spark.ui.View +import spark.util.Distribution +import spark.scheduler.cluster.TaskInfo +import spark.executor.TaskMetrics +import java.util.Date + +class StagePage(parent: JobProgressUI) extends View[Seq[Node]] { + val listener = parent.listener + val dateFmt = parent.dateFmt + + def render(request: HttpServletRequest): Seq[Node] = { + val stageId = request.getParameter("id").toInt + + val taskHeaders = Seq("Task ID", "Service Time (ms)", "Locality Level", "Worker", "Launch Time") + val tasks = listener.stageToTaskInfos(stageId) + val taskTable = listingTable(taskHeaders, taskRow, tasks) + + val serviceTimes = tasks.map{case (info, metrics) => metrics.executorRunTime.toDouble} + val serviceQuantiles = "Service Time " ++ Distribution(serviceTimes).get.getQuantiles().map(_.toString) + val quantileHeaders = Seq("Metric", "Min", "25%", "50%", "75%", "Max") + + val quantileTable = listingTable(quantileHeaders, Seq(serviceQuantiles), quantileRow) + + val content = +

    Percentile Metrics

    +
    %s
    {h}
    {s.id}{s.origin}{submissionTime}{getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))}{listener.stageToTasksComplete.getOrElse(s.id, 0)} / {s.numPartitions} + {listener.stageToTasksFailed.getOrElse(s.id, 0) match { + case f if f > 0 => "(%s failed)".format(f) + case _ => + }} +
    + + + + + + + + + {listener.stageToTaskInfos(stageId).map{ case(i, m) => taskRow(i, m) }} + +
    Service TimeRemote Bytes ReadShuffle Bytes Written
    +

    Tasks

    ++ {taskTable}; + + WebUI.headerSparkPage(content, "Stage Details: %s".format(stageId)) + } + + def quantileRow(data: Seq[String]) = {data.map(d => d)} + + def taskRow(taskData: (TaskInfo, TaskMetrics)): Seq[Node] = { + val (info, metrics) = taskData + + {info.taskId} + {metrics.executorRunTime} + {info.taskLocality} + {info.hostPort} + {dateFmt.format(new Date(info.launchTime))} + + } + + +} diff --git a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala new file mode 100644 index 0000000000..21555384dd --- /dev/null +++ b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala @@ -0,0 +1,26 @@ +package spark.ui.storage + +import akka.util.Duration +import javax.servlet.http.HttpServletRequest +import org.eclipse.jetty.server.Handler +import spark.{Logging, SparkContext} +import spark.ui.WebUI._ +import spark.ui.{UIComponent} + +/** + * Web UI server for the BlockManager inside each SparkContext. + */ +private[spark] +class BlockManagerUI(val sc: SparkContext) + extends UIComponent with Logging { + implicit val timeout = Duration.create( + System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + + val indexPage = new IndexPage(this) + val rddPage = new RDDPage(this) + + def getHandlers = Seq[(String, Handler)]( + ("/storage/rdd", (request: HttpServletRequest) => rddPage.render(request)), + ("/storage", (request: HttpServletRequest) => indexPage.render(request)) + ) +} diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala new file mode 100644 index 0000000000..2f4857d48a --- /dev/null +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -0,0 +1,61 @@ +package spark.ui.storage + +import xml.Node +import spark.storage.{RDDInfo, StorageUtils} +import spark.ui.WebUI._ +import spark.Utils +import spark.ui.{View, WebUI} +import javax.servlet.http.HttpServletRequest + +class IndexPage(parent: BlockManagerUI) extends View[Seq[Node]] { + val sc = parent.sc + + def render(request: HttpServletRequest): Seq[Node] = { + val storageStatusList = sc.getExecutorStorageStatus + // Calculate macro-level statistics + val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) + .reduceOption(_+_).getOrElse(0L) + + val rddHeaders = Seq( + "RDD Name", + "Storage Level", + "Cached Partitions", + "Fraction Partitions Cached", + "Size in Memory", + "Size on Disk") + val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) + val rddTable = listingTable(rddHeaders, rddRow, rdds) + + val content = +
    +
    +
      +
    • Memory: + {Utils.memoryBytesToString(maxMem - remainingMem)} Used + ({Utils.memoryBytesToString(remainingMem)} Available)
    • +
    • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
    • +
    +
    +
    ++ {rddTable}; + + WebUI.headerSparkPage(content, "Spark Storage ") + } + + def rddRow(rdd: RDDInfo): Seq[Node] = { + + + + {rdd.name} + + + {rdd.storageLevel.description} + + {rdd.numCachedPartitions} + {rdd.numCachedPartitions / rdd.numPartitions.toDouble} + {Utils.memoryBytesToString(rdd.memSize)} + {Utils.memoryBytesToString(rdd.diskSize)} + + } +} diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala new file mode 100644 index 0000000000..957c400080 --- /dev/null +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -0,0 +1,94 @@ +package spark.ui.storage + +import javax.servlet.http.HttpServletRequest +import xml.Node +import spark.storage.{StorageStatus, StorageUtils} +import spark.ui.WebUI._ +import spark.Utils +import spark.ui.WebUI +import spark.ui.View +import spark.storage.BlockManagerMasterActor.BlockStatus + + +class RDDPage(parent: BlockManagerUI) extends View[Seq[Node]] { + val sc = parent.sc + + def render(request: HttpServletRequest): Seq[Node] = { + val id = request.getParameter("id") + val prefix = "rdd_" + id.toString + val storageStatusList = sc.getExecutorStorageStatus + val filteredStorageStatusList = StorageUtils. + filterStorageStatusByPrefix(storageStatusList, prefix) + val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head + + val workerHeaders = Seq("Host", "Memory Usage", "Disk Usage") + val workers = filteredStorageStatusList.map((prefix, _)) + val workerTable = listingTable(workerHeaders, workerRow, workers) + + val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk") + val blocks = filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1) + val blockTable = listingTable(blockHeaders, blockRow, blocks) + + val content = +
    +
    +
      +
    • + Storage Level: + {rddInfo.storageLevel.description} +
    • +
    • + Cached Partitions: + {rddInfo.numCachedPartitions} +
    • +
    • + Total Partitions: + {rddInfo.numPartitions} +
    • +
    • + Memory Size: + {Utils.memoryBytesToString(rddInfo.memSize)} +
    • +
    • + Disk Size: + {Utils.memoryBytesToString(rddInfo.diskSize)} +
    • +
    +
    +
    +
    +
    +
    +

    RDD Summary

    +
    {blockTable} +
    +
    +
    ++ {workerTable}; + + WebUI.headerSparkPage(content, "RDD Info: " + id) + } + + def blockRow(blk: (String, BlockStatus)): Seq[Node] = { + val (id, block) = blk + + {id} + + {block.storageLevel.description} + + {Utils.memoryBytesToString(block.memSize)} + {Utils.memoryBytesToString(block.diskSize)} + + } + + def workerRow(worker: (String, StorageStatus)): Seq[Node] = { + val (prefix, status) = worker + + {status.blockManagerId.host + ":" + status.blockManagerId.port} + + {Utils.memoryBytesToString(status.memUsed(prefix))} + ({Utils.memoryBytesToString(status.memRemaining)} Total Available) + + {Utils.memoryBytesToString(status.diskUsed(prefix))} + + } +} -- cgit v1.2.3 From 17776323a6ea21170a10169af66e86c4a6cc65fb Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 20 Jun 2013 17:46:33 -0700 Subject: More work on percentile data: --- core/src/main/scala/spark/ui/jobs/StagePage.scala | 51 ++++++++++++++--------- 1 file changed, 31 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 49f5b1c73c..64b8c8418c 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -9,6 +9,7 @@ import spark.util.Distribution import spark.scheduler.cluster.TaskInfo import spark.executor.TaskMetrics import java.util.Date +import scala.collection.mutable.ListBuffer class StagePage(parent: JobProgressUI) extends View[Seq[Node]] { val listener = parent.listener @@ -16,37 +17,45 @@ class StagePage(parent: JobProgressUI) extends View[Seq[Node]] { def render(request: HttpServletRequest): Seq[Node] = { val stageId = request.getParameter("id").toInt - - val taskHeaders = Seq("Task ID", "Service Time (ms)", "Locality Level", "Worker", "Launch Time") val tasks = listener.stageToTaskInfos(stageId) + + val hasShuffleRead = tasks.head._2.shuffleReadMetrics.isDefined + val hasShuffleWrite = tasks.head._2.shuffleWriteMetrics.isDefined + + val taskHeaders = + ListBuffer("Task ID", "Service Time (ms)", "Locality Level", "Worker", "Launch Time") + if (hasShuffleRead) { taskHeaders += "Shuffle Read (bytes)" } + if (hasShuffleWrite) { taskHeaders += "Shuffle Write (bytes)" } + val taskTable = listingTable(taskHeaders, taskRow, tasks) + // TODO(pwendell): Consider factoring this more nicely with the functions in SparkListener val serviceTimes = tasks.map{case (info, metrics) => metrics.executorRunTime.toDouble} - val serviceQuantiles = "Service Time " ++ Distribution(serviceTimes).get.getQuantiles().map(_.toString) - val quantileHeaders = Seq("Metric", "Min", "25%", "50%", "75%", "Max") + val serviceQuantiles = "Service Time" +: Distribution(serviceTimes).get.getQuantiles().map(_.toString) - val quantileTable = listingTable(quantileHeaders, Seq(serviceQuantiles), quantileRow) + val shuffleReadSizes = tasks.map{ + case(info, metrics) => metrics.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble} + val shuffleReadQuantiles = "Shuffle Read" +: Distribution(shuffleReadSizes).get.getQuantiles().map(_.toString) + + val shuffleWriteSizes = tasks.map{ + case(info, metrics) => metrics.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble} + val shuffleWriteQuantiles = "Shuffle Write" +: Distribution(shuffleWriteSizes).get.getQuantiles().map(_.toString) + + + val listings: Seq[Seq[String]] = Seq(serviceQuantiles, + if (hasShuffleRead) shuffleReadQuantiles else Nil, + if (hasShuffleWrite) shuffleWriteQuantiles else Nil) + + val quantileHeaders = Seq("Metric", "Min", "25%", "50%", "75%", "Max") + val quantileTable = listingTable(quantileHeaders, quantileRow, listings) val content = -

    Percentile Metrics

    - - - - - - - - - - {listener.stageToTaskInfos(stageId).map{ case(i, m) => taskRow(i, m) }} - -
    Service TimeRemote Bytes ReadShuffle Bytes Written
    -

    Tasks

    ++ {taskTable}; +

    Summary Metrics

    ++ quantileTable ++

    Tasks

    ++ taskTable; WebUI.headerSparkPage(content, "Stage Details: %s".format(stageId)) } - def quantileRow(data: Seq[String]) = {data.map(d => d)} + def quantileRow(data: Seq[String]): Seq[Node] = {data.map(d => {d})} def taskRow(taskData: (TaskInfo, TaskMetrics)): Seq[Node] = { val (info, metrics) = taskData @@ -56,6 +65,8 @@ class StagePage(parent: JobProgressUI) extends View[Seq[Node]] { {info.taskLocality} {info.hostPort} {dateFmt.format(new Date(info.launchTime))} + {metrics.shuffleReadMetrics.map{m => {m.remoteBytesRead}}.getOrElse("") } + {metrics.shuffleWriteMetrics.map{m => {m.shuffleBytesWritten}}.getOrElse("") } } -- cgit v1.2.3 From 5c872e9ef55a467a8a5e71f0bd12537eac9d447f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 20 Jun 2013 21:06:18 -0700 Subject: Documentation and some refactoring --- .../scala/spark/deploy/master/MasterWebUI.scala | 14 +- .../scala/spark/deploy/worker/WorkerWebUI.scala | 6 +- core/src/main/scala/spark/ui/JettyUI.scala | 145 +++++++++++++++++++++ core/src/main/scala/spark/ui/SparkUI.scala | 33 ++--- core/src/main/scala/spark/ui/WebUI.scala | 137 ------------------- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 22 ++-- .../main/scala/spark/ui/jobs/JobProgressUI.scala | 47 +++---- core/src/main/scala/spark/ui/jobs/StagePage.scala | 19 +-- .../scala/spark/ui/storage/BlockManagerUI.scala | 9 +- .../main/scala/spark/ui/storage/IndexPage.scala | 8 +- core/src/main/scala/spark/ui/storage/RDDPage.scala | 12 +- 11 files changed, 227 insertions(+), 225 deletions(-) create mode 100644 core/src/main/scala/spark/ui/JettyUI.scala delete mode 100644 core/src/main/scala/spark/ui/WebUI.scala diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 328a7cb297..1ec5f77bd7 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -10,11 +10,11 @@ import net.liftweb.json.JsonAST.JValue import org.eclipse.jetty.server.Handler import scala.xml.Node import spark.{Logging, Utils} -import spark.ui.WebUI -import WebUI._ +import spark.ui.JettyUI +import JettyUI._ import spark.deploy._ import spark.deploy.MasterState -import spark.ui.WebUI +import spark.ui.JettyUI /** * Web UI server for the standalone master. @@ -29,11 +29,11 @@ class MasterWebUI(master: ActorRef) extends Logging { def start() { try { - val (server, boundPort) = WebUI.startJettyServer("0.0.0.0", port, handlers) + val (server, boundPort) = JettyUI.startJettyServer("0.0.0.0", port, handlers) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => - logError("Failed to create Master WebUI", e) + logError("Failed to create Master JettyUI", e) System.exit(1) } } @@ -99,7 +99,7 @@ class MasterWebUI(master: ActorRef) extends Logging { {executorTable(app.executors.values.toList)} ; - WebUI.sparkPage(content, "Application Info: " + app.desc.name) + JettyUI.sparkPage(content, "Application Info: " + app.desc.name) } def executorTable(executors: Seq[ExecutorInfo]): Seq[Node] = { @@ -189,7 +189,7 @@ class MasterWebUI(master: ActorRef) extends Logging { {appTable(state.completedApps.sortBy(_.endTime).reverse)} ; - WebUI.sparkPage(content, "Spark Master: " + state.uri) + JettyUI.sparkPage(content, "Spark Master: " + state.uri) } def workerTable(workers: Seq[spark.deploy.master.WorkerInfo]) = { diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index f661d99815..0febb9364c 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -12,8 +12,8 @@ import org.eclipse.jetty.server.Handler import scala.io.Source import spark.{Utils, Logging} import spark.deploy.{JsonProtocol, WorkerState, RequestWorkerState} -import spark.ui.{WebUI => UtilsWebUI} -import spark.ui.WebUI._ +import spark.ui.{JettyUI => UtilsWebUI} +import spark.ui.JettyUI._ import xml.Node /** @@ -40,7 +40,7 @@ class WorkerWebUI(worker: ActorRef, workDir: File) extends Logging { logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => - logError("Failed to create Worker WebUI", e) + logError("Failed to create Worker JettyUI", e) System.exit(1) } } diff --git a/core/src/main/scala/spark/ui/JettyUI.scala b/core/src/main/scala/spark/ui/JettyUI.scala new file mode 100644 index 0000000000..c3f01073d5 --- /dev/null +++ b/core/src/main/scala/spark/ui/JettyUI.scala @@ -0,0 +1,145 @@ +package spark.ui + +import annotation.tailrec + +import javax.servlet.http.{HttpServletResponse, HttpServletRequest} + +import net.liftweb.json.{JValue, pretty, render} + +import org.eclipse.jetty.server.{Server, Request, Handler} +import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler} + +import scala.util.{Try, Success, Failure} + +import spark.Logging + +import xml.Node + +private[spark] object JettyUI extends Logging { + // Base type for a function that returns something based on an HTTP request. Allows for + // implicit conversion from many types of functions to jetty Handlers. + type Responder[T] = HttpServletRequest => T + + // Conversions from various types of Responder's to jetty Handlers + implicit def jsonResponderToHandler(responder: Responder[JValue]): Handler = + createHandler(responder, "text/json", (in: JValue) => pretty(render(in))) + + implicit def htmlResponderToHandler(responder: Responder[Seq[Node]]): Handler = + createHandler(responder, "text/html") + + implicit def textResponderToHandler(responder: Responder[String]): Handler = + createHandler(responder, "text/plain") + + private def createHandler[T <% AnyRef](responder: Responder[T], contentType: String, + extractFn: T => String = (in: Any) => in.toString): Handler = { + new AbstractHandler { + def handle(target: String, + baseRequest: Request, + request: HttpServletRequest, + response: HttpServletResponse) { + response.setContentType("%s;charset=utf-8".format(contentType)) + response.setStatus(HttpServletResponse.SC_OK) + baseRequest.setHandled(true) + val result = responder(request) + response.getWriter().println(extractFn(result)) + } + } + } + + /** Creates a handler for serving files from a static directory. */ + def createStaticHandler(resourceBase: String): ResourceHandler = { + val staticHandler = new ResourceHandler + Option(getClass.getClassLoader.getResource(resourceBase)) match { + case Some(res) => + staticHandler.setResourceBase(res.toString) + case None => + logError("Could not find resource path for Web UI: " + resourceBase) + } + staticHandler + } + + /** + * Attempts to start a Jetty server at the supplied ip:port which uses the supplied handlers. + * + * If the desired port number is contented, continues incrementing ports until a free port is + * found. Returns the chosen port and the jetty Server object. + */ + def startJettyServer(ip: String, port: Int, handlers: Seq[(String, Handler)]): (Server, Int) = { + val handlersToRegister = handlers.map { case(path, handler) => + if (path == "*") { + handler + } else { + val contextHandler = new ContextHandler(path) + contextHandler.setHandler(handler) + contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] + } + } + + val handlerList = new HandlerList + handlerList.setHandlers(handlersToRegister.toArray) + + @tailrec + def connect(currentPort: Int): (Server, Int) = { + val server = new Server(port) + server.setHandler(handlerList) + Try { server.start() } match { + case s: Success[_] => (server, currentPort) + case f: Failure[_] => + logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) + connect((currentPort + 1) % 65536) + } + } + connect(port) + } +} + +/** Utility functions for generating XML pages with spark content. */ +object UIUtils { + + /** Returns a page containing the supplied content and the spark web ui headers */ + def headerSparkPage(content: => Seq[Node], title: String): Seq[Node] = { + val newContent = +

    Storage | Jobs

    ; + sparkPage(newContent ++ content, title) + } + + /** Returns a page containing the supplied content and the spark css, js, and logo. */ + def sparkPage(content: => Seq[Node], title: String): Seq[Node] = { + + + + + + + {title} + + + +
    +
    +
    + +

    + {title} +

    +
    +
    + {content} +
    + + + } + + /** Returns an HTML table constructed by generating a row for each object in a sequence. */ + def listingTable[T](headers: Seq[String], makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { + + {headers.map(h => )} + + {rows.map(r => makeRow(r))} + +
    {h}
    + } +} diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index a1f6cc60ec..dd7d33e0fa 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -1,43 +1,46 @@ package spark.ui -import jobs.JobProgressUI -import spark.{Logging, SparkContext, Utils} import javax.servlet.http.HttpServletRequest + import org.eclipse.jetty.server.Handler -import storage.BlockManagerUI -import WebUI._ +import spark.{Logging, SparkContext, Utils} +import spark.ui.storage.BlockManagerUI +import spark.ui.jobs.JobProgressUI +import spark.ui.UIUtils._ +import spark.ui.JettyUI._ + +/** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { val host = Utils.localHostName() val port = Option(System.getProperty("spark.ui.port")).getOrElse(SparkUI.DEFAULT_PORT).toInt - + var boundPort: Option[Int] = None val handlers = Seq[(String, Handler)]( ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), - ("*", (request: HttpServletRequest) => WebUI.headerSparkPage(

    Test

    , "Test page")) + ("*", (request: HttpServletRequest) => headerSparkPage(

    Test

    , "Test page")) ) - val components = Seq(new BlockManagerUI(sc), new JobProgressUI(sc)) + val storage = new BlockManagerUI(sc) + val jobs = new JobProgressUI(sc) + val allHandlers = handlers ++ storage.getHandlers ++ jobs.getHandlers def start() { /** Start an HTTP server to run the Web interface */ try { - val allHandlers = components.flatMap(_.getHandlers) ++ handlers - val (server, boundPort) = WebUI.startJettyServer("0.0.0.0", port, allHandlers) - logInfo("Started Spark Web UI at http://%s:%d".format(host, boundPort)) + val (server, usedPort) = JettyUI.startJettyServer("0.0.0.0", port, allHandlers) + logInfo("Started Spark Web UI at http://%s:%d".format(host, usedPort)) + boundPort = Some(usedPort) } catch { case e: Exception => - logError("Failed to create Spark WebUI", e) + logError("Failed to create Spark JettyUI", e) System.exit(1) } } - private[spark] def appUIAddress = "http://" + host + ":" + port + private[spark] def appUIAddress = "http://" + host + ":" + boundPort.getOrElse("-1") } object SparkUI { val DEFAULT_PORT = "33000" val STATIC_RESOURCE_DIR = "spark/webui/static" } - - - diff --git a/core/src/main/scala/spark/ui/WebUI.scala b/core/src/main/scala/spark/ui/WebUI.scala deleted file mode 100644 index 6dedd28fc1..0000000000 --- a/core/src/main/scala/spark/ui/WebUI.scala +++ /dev/null @@ -1,137 +0,0 @@ -package spark.ui - -import annotation.tailrec -import javax.servlet.http.{HttpServletResponse, HttpServletRequest} -import net.liftweb.json._ -import org.eclipse.jetty.server.{Server, Request, Handler} -import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler} -import spark.Logging -import util.{Try, Success, Failure} -import xml.Node - -abstract class UIComponent { - def getHandlers(): Seq[(String, Handler)] -} - -abstract class View[T] { - def render(request: HttpServletRequest): T -} - -object WebUI extends Logging { - // CORE WEB UI COMPONENTS - type Responder[T] = HttpServletRequest => T - - implicit def jsonResponderToHandler(responder: Responder[JValue]): Handler = - createHandler(responder, "text/json", (in: JValue) => pretty(render(in))) - - implicit def htmlResponderToHandler(responder: Responder[Seq[Node]]): Handler = - createHandler(responder, "text/html") - - implicit def textResponderToHandler(responder: Responder[String]): Handler = - createHandler(responder, "text/plain") - - def createHandler[T <% AnyRef](responder: Responder[T], contentType: String, - extractFn: T => String = (in: Any) => in.toString): Handler = { - new AbstractHandler { - def handle(target: String, - baseRequest: Request, - request: HttpServletRequest, - response: HttpServletResponse) { - response.setContentType("%s;charset=utf-8".format(contentType)) - response.setStatus(HttpServletResponse.SC_OK) - baseRequest.setHandled(true) - val result = responder(request) - response.getWriter().println(extractFn(result)) - } - } - } - - def createStaticHandler(resourceBase: String): ResourceHandler = { - val staticHandler = new ResourceHandler - Option(getClass.getClassLoader.getResource(resourceBase)) match { - case Some(res) => - staticHandler.setResourceBase(res.toString) - case None => - logError("Could not find resource path for Web UI: " + resourceBase) - } - staticHandler - } - - def startJettyServer(ip: String, port: Int, handlers: Seq[(String, Handler)]): (Server, Int) = { - val handlersToRegister = handlers.map { case(path, handler) => - if (path == "*") { - handler - } else { - val contextHandler = new ContextHandler(path) - contextHandler.setHandler(handler) - contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] - } - } - - val handlerList = new HandlerList - handlerList.setHandlers(handlersToRegister.toArray) - - @tailrec - def connect(currentPort: Int): (Server, Int) = { - val server = new Server(port) - server.setHandler(handlerList) - Try { server.start() } match { - case s: Success[_] => (server, currentPort) - case f: Failure[_] => - logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) - connect((currentPort + 1) % 65536) - } - } - connect(port) - } - - // HELPER FUNCTIONS AND SHORTCUTS - - /** Page with Spark logo, title, and Spark UI headers */ - def headerSparkPage(content: => Seq[Node], title: String): Seq[Node] = { - val newContent = -

    Storage | Jobs

    ; - - sparkPage(newContent ++ content, title) - } - - /** Page with Spark logo and title */ - def sparkPage(content: => Seq[Node], title: String): Seq[Node] = { - - - - - - - {title} - - - -
    -
    -
    - -

    - {title} -

    -
    -
    - {content} -
    - - - } - - /** Shortcut for making a table derived from a sequence of objects. */ - def listingTable[T](headers: Seq[String], makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { - - {headers.map(h => )} - - {rows.map(r => makeRow(r))} - -
    {h}
    - } -} diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 1740524f49..811baae811 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -1,15 +1,20 @@ package spark.ui.jobs -import spark.ui.{WebUI, View} -import xml.{NodeSeq, Node} -import spark.ui.WebUI._ -import scala.Some import akka.util.Duration -import spark.scheduler.Stage + import java.util.Date + import javax.servlet.http.HttpServletRequest -class IndexPage(parent: JobProgressUI) extends View[Seq[Node]] { +import scala.Some + +import spark.scheduler.Stage +import spark.ui.UIUtils._ + +import xml.{NodeSeq, Node} + +/** Page showing list of all ongoing and recently finished stages */ +class IndexPage(parent: JobProgressUI) { val listener = parent.listener val dateFmt = parent.dateFmt @@ -24,7 +29,7 @@ class IndexPage(parent: JobProgressUI) extends View[Seq[Node]] { val content =

    Active Stages

    ++ activeStageTable ++

    Completed Stages

    ++ completedStageTable - WebUI.headerSparkPage(content, "Spark Stages") + headerSparkPage(content, "Spark Stages") } def getElapsedTime(submitted: Option[Long], completed: Long): String = { @@ -43,7 +48,8 @@ class IndexPage(parent: JobProgressUI) extends View[Seq[Node]] { {s.id} {s.origin} {submissionTime} - {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))} + {getElapsedTime(s.submissionTime, + s.completionTime.getOrElse(System.currentTimeMillis()))} {listener.stageToTasksComplete.getOrElse(s.id, 0)} / {s.numPartitions} {listener.stageToTasksFailed.getOrElse(s.id, 0) match { case f if f > 0 => "(%s failed)".format(f) diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 9ee962bf2e..31aa0f9f0d 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -1,45 +1,30 @@ package spark.ui.jobs -import spark.{Utils, SparkContext} -import spark.scheduler._ -import spark.scheduler.SparkListenerTaskEnd -import spark.scheduler.StageCompleted -import spark.scheduler.SparkListenerStageSubmitted -import org.eclipse.jetty.server.Handler -import javax.servlet.http.HttpServletRequest -import xml.Node -import collection.mutable._ -import spark.Success -import akka.util.Duration import java.text.SimpleDateFormat -import java.util.Date + +import javax.servlet.http.HttpServletRequest + +import org.eclipse.jetty.server.Handler + +import scala.Seq +import scala.collection.mutable.{HashSet, ListBuffer, HashMap, ArrayBuffer} + +import spark.ui.JettyUI._ +import spark.SparkContext +import spark.scheduler._ import spark.scheduler.cluster.TaskInfo -import collection.mutable -import org.hsqldb.lib.HashMappedList import spark.executor.TaskMetrics -import spark.scheduler.SparkListenerTaskEnd -import scala.Some -import spark.scheduler.SparkListenerStageSubmitted -import scala.Seq -import spark.scheduler.StageCompleted -import spark.scheduler.SparkListenerJobStart -import spark.ui.{WebUI, UIComponent} -import spark.ui.WebUI._ -import spark.scheduler.SparkListenerTaskEnd -import scala.Some -import spark.scheduler.SparkListenerStageSubmitted -import spark.scheduler.StageCompleted -import spark.scheduler.SparkListenerJobStart +import spark.Success -private[spark] -class JobProgressUI(sc: SparkContext) extends UIComponent { +/** Web UI showing progress status of all jobs in the given SparkContext. */ +private[spark] class JobProgressUI(sc: SparkContext) { val listener = new JobProgressListener val dateFmt = new SimpleDateFormat("EEE, MMM d yyyy HH:mm:ss") sc.addSparkListener(listener) - val indexPage = new IndexPage(this) - val stagePage = new StagePage(this) + private val indexPage = new IndexPage(this) + private val stagePage = new StagePage(this) def getHandlers = Seq[(String, Handler)]( ("/stages/stage", (request: HttpServletRequest) => stagePage.render(request)), diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 64b8c8418c..74ac811cef 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -1,17 +1,20 @@ package spark.ui.jobs +import java.util.Date + import javax.servlet.http.HttpServletRequest -import xml.Node -import spark.ui.WebUI._ -import spark.ui.WebUI -import spark.ui.View + +import scala.collection.mutable.ListBuffer + +import spark.ui.UIUtils._ import spark.util.Distribution import spark.scheduler.cluster.TaskInfo import spark.executor.TaskMetrics -import java.util.Date -import scala.collection.mutable.ListBuffer -class StagePage(parent: JobProgressUI) extends View[Seq[Node]] { +import xml.Node + +/** Page showing statistics and task list for a given stage */ +class StagePage(parent: JobProgressUI) { val listener = parent.listener val dateFmt = parent.dateFmt @@ -52,7 +55,7 @@ class StagePage(parent: JobProgressUI) extends View[Seq[Node]] { val content =

    Summary Metrics

    ++ quantileTable ++

    Tasks

    ++ taskTable; - WebUI.headerSparkPage(content, "Stage Details: %s".format(stageId)) + headerSparkPage(content, "Stage Details: %s".format(stageId)) } def quantileRow(data: Seq[String]): Seq[Node] = {data.map(d => {d})} diff --git a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala index 21555384dd..d83c826033 100644 --- a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala @@ -4,15 +4,12 @@ import akka.util.Duration import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler import spark.{Logging, SparkContext} -import spark.ui.WebUI._ +import spark.ui.JettyUI._ import spark.ui.{UIComponent} -/** - * Web UI server for the BlockManager inside each SparkContext. - */ +/** Web UI showing storage status of all RDD's in the given SparkContext. */ private[spark] -class BlockManagerUI(val sc: SparkContext) - extends UIComponent with Logging { +class BlockManagerUI(val sc: SparkContext) extends Logging { implicit val timeout = Duration.create( System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index 2f4857d48a..5ead772bc0 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -2,12 +2,12 @@ package spark.ui.storage import xml.Node import spark.storage.{RDDInfo, StorageUtils} -import spark.ui.WebUI._ import spark.Utils -import spark.ui.{View, WebUI} +import spark.ui.UIUtils._ import javax.servlet.http.HttpServletRequest -class IndexPage(parent: BlockManagerUI) extends View[Seq[Node]] { +/** Page showing list of RDD's currently stored in the cluster */ +class IndexPage(parent: BlockManagerUI) { val sc = parent.sc def render(request: HttpServletRequest): Seq[Node] = { @@ -40,7 +40,7 @@ class IndexPage(parent: BlockManagerUI) extends View[Seq[Node]] { ++ {rddTable}; - WebUI.headerSparkPage(content, "Spark Storage ") + headerSparkPage(content, "Spark Storage ") } def rddRow(rdd: RDDInfo): Seq[Node] = { diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 957c400080..7628fde4aa 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -1,16 +1,16 @@ package spark.ui.storage import javax.servlet.http.HttpServletRequest -import xml.Node + import spark.storage.{StorageStatus, StorageUtils} -import spark.ui.WebUI._ +import spark.ui.UIUtils._ import spark.Utils -import spark.ui.WebUI -import spark.ui.View import spark.storage.BlockManagerMasterActor.BlockStatus +import xml.Node -class RDDPage(parent: BlockManagerUI) extends View[Seq[Node]] { +/** Page showing storage details for a given RDD */ +class RDDPage(parent: BlockManagerUI) { val sc = parent.sc def render(request: HttpServletRequest): Seq[Node] = { @@ -65,7 +65,7 @@ class RDDPage(parent: BlockManagerUI) extends View[Seq[Node]] {
    ++ {workerTable}; - WebUI.headerSparkPage(content, "RDD Info: " + id) + headerSparkPage(content, "RDD Info: " + id) } def blockRow(blk: (String, BlockStatus)): Seq[Node] = { -- cgit v1.2.3 From dd696f3a3d04f5e3b453173c0d4e86690f6bcb1b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 20 Jun 2013 23:20:56 -0700 Subject: Some renaming and comments --- .../scala/spark/deploy/master/MasterWebUI.scala | 14 +- .../scala/spark/deploy/worker/WorkerWebUI.scala | 6 +- core/src/main/scala/spark/ui/JettyUI.scala | 145 -------------------- core/src/main/scala/spark/ui/JettyUtils.scala | 147 +++++++++++++++++++++ core/src/main/scala/spark/ui/SparkUI.scala | 6 +- .../main/scala/spark/ui/jobs/JobProgressUI.scala | 2 +- .../scala/spark/ui/storage/BlockManagerUI.scala | 6 +- .../main/scala/spark/ui/storage/IndexPage.scala | 6 +- 8 files changed, 169 insertions(+), 163 deletions(-) delete mode 100644 core/src/main/scala/spark/ui/JettyUI.scala create mode 100644 core/src/main/scala/spark/ui/JettyUtils.scala diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 1ec5f77bd7..3238661127 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -10,11 +10,11 @@ import net.liftweb.json.JsonAST.JValue import org.eclipse.jetty.server.Handler import scala.xml.Node import spark.{Logging, Utils} -import spark.ui.JettyUI -import JettyUI._ +import spark.ui.JettyUtils +import JettyUtils._ import spark.deploy._ import spark.deploy.MasterState -import spark.ui.JettyUI +import spark.ui.JettyUtils /** * Web UI server for the standalone master. @@ -29,11 +29,11 @@ class MasterWebUI(master: ActorRef) extends Logging { def start() { try { - val (server, boundPort) = JettyUI.startJettyServer("0.0.0.0", port, handlers) + val (server, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => - logError("Failed to create Master JettyUI", e) + logError("Failed to create Master JettyUtils", e) System.exit(1) } } @@ -99,7 +99,7 @@ class MasterWebUI(master: ActorRef) extends Logging { {executorTable(app.executors.values.toList)} ; - JettyUI.sparkPage(content, "Application Info: " + app.desc.name) + JettyUtils.sparkPage(content, "Application Info: " + app.desc.name) } def executorTable(executors: Seq[ExecutorInfo]): Seq[Node] = { @@ -189,7 +189,7 @@ class MasterWebUI(master: ActorRef) extends Logging { {appTable(state.completedApps.sortBy(_.endTime).reverse)} ; - JettyUI.sparkPage(content, "Spark Master: " + state.uri) + JettyUtils.sparkPage(content, "Spark Master: " + state.uri) } def workerTable(workers: Seq[spark.deploy.master.WorkerInfo]) = { diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index 0febb9364c..805e7b52db 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -12,8 +12,8 @@ import org.eclipse.jetty.server.Handler import scala.io.Source import spark.{Utils, Logging} import spark.deploy.{JsonProtocol, WorkerState, RequestWorkerState} -import spark.ui.{JettyUI => UtilsWebUI} -import spark.ui.JettyUI._ +import spark.ui.{JettyUtils => UtilsWebUI} +import spark.ui.JettyUtils._ import xml.Node /** @@ -40,7 +40,7 @@ class WorkerWebUI(worker: ActorRef, workDir: File) extends Logging { logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => - logError("Failed to create Worker JettyUI", e) + logError("Failed to create Worker JettyUtils", e) System.exit(1) } } diff --git a/core/src/main/scala/spark/ui/JettyUI.scala b/core/src/main/scala/spark/ui/JettyUI.scala deleted file mode 100644 index c3f01073d5..0000000000 --- a/core/src/main/scala/spark/ui/JettyUI.scala +++ /dev/null @@ -1,145 +0,0 @@ -package spark.ui - -import annotation.tailrec - -import javax.servlet.http.{HttpServletResponse, HttpServletRequest} - -import net.liftweb.json.{JValue, pretty, render} - -import org.eclipse.jetty.server.{Server, Request, Handler} -import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler} - -import scala.util.{Try, Success, Failure} - -import spark.Logging - -import xml.Node - -private[spark] object JettyUI extends Logging { - // Base type for a function that returns something based on an HTTP request. Allows for - // implicit conversion from many types of functions to jetty Handlers. - type Responder[T] = HttpServletRequest => T - - // Conversions from various types of Responder's to jetty Handlers - implicit def jsonResponderToHandler(responder: Responder[JValue]): Handler = - createHandler(responder, "text/json", (in: JValue) => pretty(render(in))) - - implicit def htmlResponderToHandler(responder: Responder[Seq[Node]]): Handler = - createHandler(responder, "text/html") - - implicit def textResponderToHandler(responder: Responder[String]): Handler = - createHandler(responder, "text/plain") - - private def createHandler[T <% AnyRef](responder: Responder[T], contentType: String, - extractFn: T => String = (in: Any) => in.toString): Handler = { - new AbstractHandler { - def handle(target: String, - baseRequest: Request, - request: HttpServletRequest, - response: HttpServletResponse) { - response.setContentType("%s;charset=utf-8".format(contentType)) - response.setStatus(HttpServletResponse.SC_OK) - baseRequest.setHandled(true) - val result = responder(request) - response.getWriter().println(extractFn(result)) - } - } - } - - /** Creates a handler for serving files from a static directory. */ - def createStaticHandler(resourceBase: String): ResourceHandler = { - val staticHandler = new ResourceHandler - Option(getClass.getClassLoader.getResource(resourceBase)) match { - case Some(res) => - staticHandler.setResourceBase(res.toString) - case None => - logError("Could not find resource path for Web UI: " + resourceBase) - } - staticHandler - } - - /** - * Attempts to start a Jetty server at the supplied ip:port which uses the supplied handlers. - * - * If the desired port number is contented, continues incrementing ports until a free port is - * found. Returns the chosen port and the jetty Server object. - */ - def startJettyServer(ip: String, port: Int, handlers: Seq[(String, Handler)]): (Server, Int) = { - val handlersToRegister = handlers.map { case(path, handler) => - if (path == "*") { - handler - } else { - val contextHandler = new ContextHandler(path) - contextHandler.setHandler(handler) - contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] - } - } - - val handlerList = new HandlerList - handlerList.setHandlers(handlersToRegister.toArray) - - @tailrec - def connect(currentPort: Int): (Server, Int) = { - val server = new Server(port) - server.setHandler(handlerList) - Try { server.start() } match { - case s: Success[_] => (server, currentPort) - case f: Failure[_] => - logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) - connect((currentPort + 1) % 65536) - } - } - connect(port) - } -} - -/** Utility functions for generating XML pages with spark content. */ -object UIUtils { - - /** Returns a page containing the supplied content and the spark web ui headers */ - def headerSparkPage(content: => Seq[Node], title: String): Seq[Node] = { - val newContent = -

    Storage | Jobs

    ; - sparkPage(newContent ++ content, title) - } - - /** Returns a page containing the supplied content and the spark css, js, and logo. */ - def sparkPage(content: => Seq[Node], title: String): Seq[Node] = { - - - - - - - {title} - - - -
    -
    -
    - -

    - {title} -

    -
    -
    - {content} -
    - - - } - - /** Returns an HTML table constructed by generating a row for each object in a sequence. */ - def listingTable[T](headers: Seq[String], makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { - - {headers.map(h => )} - - {rows.map(r => makeRow(r))} - -
    {h}
    - } -} diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala new file mode 100644 index 0000000000..8bb343163a --- /dev/null +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -0,0 +1,147 @@ +package spark.ui + +import annotation.tailrec + +import javax.servlet.http.{HttpServletResponse, HttpServletRequest} + +import net.liftweb.json.{JValue, pretty, render} + +import org.eclipse.jetty.server.{Server, Request, Handler} +import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler} + +import scala.util.{Try, Success, Failure} + +import spark.Logging + +import xml.Node + + +/** Utilities for launching a web server using Jetty's HTTP Server class */ +private[spark] object JettyUtils extends Logging { + // Base type for a function that returns something based on an HTTP request. Allows for + // implicit conversion from many types of functions to jetty Handlers. + type Responder[T] = HttpServletRequest => T + + // Conversions from various types of Responder's to jetty Handlers + implicit def jsonResponderToHandler(responder: Responder[JValue]): Handler = + createHandler(responder, "text/json", (in: JValue) => pretty(render(in))) + + implicit def htmlResponderToHandler(responder: Responder[Seq[Node]]): Handler = + createHandler(responder, "text/html") + + implicit def textResponderToHandler(responder: Responder[String]): Handler = + createHandler(responder, "text/plain") + + private def createHandler[T <% AnyRef](responder: Responder[T], contentType: String, + extractFn: T => String = (in: Any) => in.toString): Handler = { + new AbstractHandler { + def handle(target: String, + baseRequest: Request, + request: HttpServletRequest, + response: HttpServletResponse) { + response.setContentType("%s;charset=utf-8".format(contentType)) + response.setStatus(HttpServletResponse.SC_OK) + baseRequest.setHandled(true) + val result = responder(request) + response.getWriter().println(extractFn(result)) + } + } + } + + /** Creates a handler for serving files from a static directory. */ + def createStaticHandler(resourceBase: String): ResourceHandler = { + val staticHandler = new ResourceHandler + Option(getClass.getClassLoader.getResource(resourceBase)) match { + case Some(res) => + staticHandler.setResourceBase(res.toString) + case None => + logError("Could not find resource path for Web UI: " + resourceBase) + } + staticHandler + } + + /** + * Attempts to start a Jetty server at the supplied ip:port which uses the supplied handlers. + * + * If the desired port number is contented, continues incrementing ports until a free port is + * found. Returns the chosen port and the jetty Server object. + */ + def startJettyServer(ip: String, port: Int, handlers: Seq[(String, Handler)]): (Server, Int) = { + val handlersToRegister = handlers.map { case(path, handler) => + if (path == "*") { + handler + } else { + val contextHandler = new ContextHandler(path) + contextHandler.setHandler(handler) + contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] + } + } + + val handlerList = new HandlerList + handlerList.setHandlers(handlersToRegister.toArray) + + @tailrec + def connect(currentPort: Int): (Server, Int) = { + val server = new Server(port) + server.setHandler(handlerList) + Try { server.start() } match { + case s: Success[_] => (server, currentPort) + case f: Failure[_] => + logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) + connect((currentPort + 1) % 65536) + } + } + connect(port) + } +} + +/** Utility functions for generating XML pages with spark content. */ +object UIUtils { + + /** Returns a page containing the supplied content and the spark web ui headers */ + def headerSparkPage(content: => Seq[Node], title: String): Seq[Node] = { + val newContent = +

    Storage | Jobs

    ; + sparkPage(newContent ++ content, title) + } + + /** Returns a page containing the supplied content and the spark css, js, and logo. */ + def sparkPage(content: => Seq[Node], title: String): Seq[Node] = { + + + + + + + {title} + + + +
    +
    +
    + +

    + {title} +

    +
    +
    + {content} +
    + + + } + + /** Returns an HTML table constructed by generating a row for each object in a sequence. */ + def listingTable[T](headers: Seq[String], makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { + + {headers.map(h => )} + + {rows.map(r => makeRow(r))} + +
    {h}
    + } +} diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index dd7d33e0fa..7428e7a343 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -8,7 +8,7 @@ import spark.{Logging, SparkContext, Utils} import spark.ui.storage.BlockManagerUI import spark.ui.jobs.JobProgressUI import spark.ui.UIUtils._ -import spark.ui.JettyUI._ +import spark.ui.JettyUtils._ /** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { @@ -27,12 +27,12 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { def start() { /** Start an HTTP server to run the Web interface */ try { - val (server, usedPort) = JettyUI.startJettyServer("0.0.0.0", port, allHandlers) + val (server, usedPort) = JettyUtils.startJettyServer("0.0.0.0", port, allHandlers) logInfo("Started Spark Web UI at http://%s:%d".format(host, usedPort)) boundPort = Some(usedPort) } catch { case e: Exception => - logError("Failed to create Spark JettyUI", e) + logError("Failed to create Spark JettyUtils", e) System.exit(1) } } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 31aa0f9f0d..97fda84e06 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -9,7 +9,7 @@ import org.eclipse.jetty.server.Handler import scala.Seq import scala.collection.mutable.{HashSet, ListBuffer, HashMap, ArrayBuffer} -import spark.ui.JettyUI._ +import spark.ui.JettyUtils._ import spark.SparkContext import spark.scheduler._ import spark.scheduler.cluster.TaskInfo diff --git a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala index d83c826033..4f1928fac9 100644 --- a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala @@ -1,11 +1,13 @@ package spark.ui.storage import akka.util.Duration + import javax.servlet.http.HttpServletRequest + import org.eclipse.jetty.server.Handler + import spark.{Logging, SparkContext} -import spark.ui.JettyUI._ -import spark.ui.{UIComponent} +import spark.ui.JettyUtils._ /** Web UI showing storage status of all RDD's in the given SparkContext. */ private[spark] diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index 5ead772bc0..1f9953fb6b 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -1,10 +1,12 @@ package spark.ui.storage -import xml.Node +import javax.servlet.http.HttpServletRequest + import spark.storage.{RDDInfo, StorageUtils} import spark.Utils import spark.ui.UIUtils._ -import javax.servlet.http.HttpServletRequest + +import xml.Node /** Page showing list of RDD's currently stored in the cluster */ class IndexPage(parent: BlockManagerUI) { -- cgit v1.2.3 From 3485e73376fcef524e62547ce3d69bb28a4381ad Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 21 Jun 2013 13:47:27 -0700 Subject: Style cleanup --- .../main/scala/spark/deploy/master/Master.scala | 1 + .../scala/spark/deploy/master/MasterWebUI.scala | 266 --------------------- .../spark/deploy/master/ui/ApplicationPage.scala | 100 ++++++++ .../scala/spark/deploy/master/ui/IndexPage.scala | 115 +++++++++ .../scala/spark/deploy/master/ui/MasterWebUI.scala | 50 ++++ .../main/scala/spark/deploy/worker/Worker.scala | 1 + .../scala/spark/deploy/worker/WorkerWebUI.scala | 148 ------------ .../scala/spark/deploy/worker/ui/IndexPage.scala | 97 ++++++++ .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 64 +++++ core/src/main/scala/spark/ui/JettyUtils.scala | 1 - 10 files changed, 428 insertions(+), 415 deletions(-) delete mode 100644 core/src/main/scala/spark/deploy/master/MasterWebUI.scala create mode 100644 core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala create mode 100644 core/src/main/scala/spark/deploy/master/ui/IndexPage.scala create mode 100644 core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala delete mode 100644 core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala create mode 100644 core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala create mode 100644 core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 6a7bbdfcbf..3eca522c7e 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -13,6 +13,7 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import spark.deploy._ import spark.{Logging, SparkException, Utils} import spark.util.AkkaUtils +import ui.MasterWebUI private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala deleted file mode 100644 index 3238661127..0000000000 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ /dev/null @@ -1,266 +0,0 @@ -package spark.deploy.master - -import akka.actor.ActorRef -import akka.dispatch.Await -import akka.pattern.ask -import akka.util.Duration -import akka.util.duration._ -import javax.servlet.http.HttpServletRequest -import net.liftweb.json.JsonAST.JValue -import org.eclipse.jetty.server.Handler -import scala.xml.Node -import spark.{Logging, Utils} -import spark.ui.JettyUtils -import JettyUtils._ -import spark.deploy._ -import spark.deploy.MasterState -import spark.ui.JettyUtils - -/** - * Web UI server for the standalone master. - */ -private[spark] -class MasterWebUI(master: ActorRef) extends Logging { - implicit val timeout = Duration.create( - System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") - val host = Utils.localHostName() - val port = Option(System.getProperty("master.ui.port")) - .getOrElse(MasterWebUI.DEFAULT_PORT).toInt - - def start() { - try { - val (server, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) - logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Master JettyUtils", e) - System.exit(1) - } - } - - val handlers = Array[(String, Handler)]( - ("/static", createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR)), - ("/app/json", (request: HttpServletRequest) => appDetailJson(request)), - ("/app", (request: HttpServletRequest) => appDetail(request)), - ("*", (request: HttpServletRequest) => index) - ) - - /** Executor details for a particular application */ - def appDetailJson(request: HttpServletRequest): JValue = { - val appId = request.getParameter("appId") - val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] - val state = Await.result(stateFuture, 3 seconds) - val app = state.activeApps.find(_.id == appId).getOrElse({ - state.completedApps.find(_.id == appId).getOrElse(null) - }) - JsonProtocol.writeApplicationInfo(app) - } - - /** Executor details for a particular application */ - def appDetail(request: HttpServletRequest): Seq[Node] = { - val appId = request.getParameter("appId") - val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] - val state = Await.result(stateFuture, 3 seconds) - val app = state.activeApps.find(_.id == appId).getOrElse({ - state.completedApps.find(_.id == appId).getOrElse(null) - }) - val content = -
    -
    -
    -
      -
    • ID: {app.id}
    • -
    • Description: {app.desc.name}
    • -
    • User: {app.desc.user}
    • -
    • Cores: - { - if (app.desc.maxCores == Integer.MAX_VALUE) { - "Unlimited %s granted".format(app.coresGranted) - } else { - "%s (%s granted, %s left)".format( - app.desc.maxCores, app.coresGranted, app.coresLeft) - } - } -
    • -
    • Memory per Slave: {app.desc.memoryPerSlave}
    • -
    • Submit Date: {app.submitDate}
    • -
    • State: {app.state}
    • -
    • Application Detail UI
    • -
    -
    -
    - -
    - -
    -
    -

    Executor Summary

    -
    - {executorTable(app.executors.values.toList)} -
    -
    ; - JettyUtils.sparkPage(content, "Application Info: " + app.desc.name) - } - - def executorTable(executors: Seq[ExecutorInfo]): Seq[Node] = { - - - - - - - - - - - - - {executors.map(executorRow)} - -
    ExecutorIDWorkerCoresMemoryStateLogs
    - } - - def executorRow(executor: ExecutorInfo): Seq[Node] = { - - {executor.id} - - {executor.worker.id} - - {executor.cores} - {executor.memory} - {executor.state} - - stdout - stderr - - - } - - /** Index view listing applications and executors */ - def index: Seq[Node] = { - val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] - val state = Await.result(stateFuture, 3 seconds) - - val content = -
    -
    -
    -
      -
    • URL:{state.uri}
    • -
    • Workers:{state.workers.size}
    • -
    • Cores: {state.workers.map(_.cores).sum} Total, - {state.workers.map(_.coresUsed).sum} Used
    • -
    • Memory: - {Utils.memoryMegabytesToString(state.workers.map(_.memory).sum)} Total, - {Utils.memoryMegabytesToString(state.workers.map(_.memoryUsed).sum)} Used
    • -
    • Applications: - {state.activeApps.size} Running, - {state.completedApps.size} Completed
    • -
    -
    -
    - -
    -
    -

    Workers

    -
    - {workerTable(state.workers.sortBy(_.id))} -
    -
    - -
    - -
    -
    -

    Running Applications

    -
    - {appTable(state.activeApps.sortBy(_.startTime).reverse)} -
    -
    - -
    - -
    -
    -

    Completed Applications

    -
    - {appTable(state.completedApps.sortBy(_.endTime).reverse)} -
    -
    ; - JettyUtils.sparkPage(content, "Spark Master: " + state.uri) - } - - def workerTable(workers: Seq[spark.deploy.master.WorkerInfo]) = { - - - - - - - - - - - - { - workers.map{ worker => - - - - - - - - } - } - -
    IDAddressStateCoresMemory
    - {worker.id} - {worker.host}:{worker.port}{worker.state}{worker.cores} ({worker.coresUsed} Used){Utils.memoryMegabytesToString(worker.memory)} - ({Utils.memoryMegabytesToString(worker.memoryUsed)} Used)
    - } - - def appTable(apps: Seq[spark.deploy.master.ApplicationInfo]) = { - - - - - - - - - - - - - - - { - apps.map{ app => - - - - - - - - - - - } - } - -
    IDDescriptionCoresMemory per NodeSubmit TimeUserStateDuration
    - {app.id} - {app.desc.name} - {app.coresGranted} - {Utils.memoryMegabytesToString(app.desc.memoryPerSlave)}{DeployWebUI.formatDate(app.submitDate)}{app.desc.user}{app.state.toString}{DeployWebUI.formatDuration(app.duration)}
    - } -} - -object MasterWebUI { - val STATIC_RESOURCE_DIR = "spark/webui/static" - val DEFAULT_PORT = "8080" -} \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala new file mode 100644 index 0000000000..2a810b71d2 --- /dev/null +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -0,0 +1,100 @@ +package spark.deploy.master.ui + +import akka.dispatch.Await +import akka.pattern.ask +import akka.util.duration._ + +import javax.servlet.http.HttpServletRequest + +import net.liftweb.json.JsonAST.JValue + +import scala.xml.Node + +import spark.deploy.{RequestMasterState, JsonProtocol, MasterState} +import spark.deploy.master.ExecutorInfo +import spark.ui.UIUtils + +class ApplicationPage(parent: MasterWebUI) { + val master = parent.master + implicit val timeout = parent.timeout + + /** Executor details for a particular application */ + def renderJson(request: HttpServletRequest): JValue = { + val appId = request.getParameter("appId") + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val state = Await.result(stateFuture, 3 seconds) + val app = state.activeApps.find(_.id == appId).getOrElse({ + state.completedApps.find(_.id == appId).getOrElse(null) + }) + JsonProtocol.writeApplicationInfo(app) + } + + /** Executor details for a particular application */ + def render(request: HttpServletRequest): Seq[Node] = { + val appId = request.getParameter("appId") + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val state = Await.result(stateFuture, 3 seconds) + val app = state.activeApps.find(_.id == appId).getOrElse({ + state.completedApps.find(_.id == appId).getOrElse(null) + }) + + val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Logs") + val executors = app.executors.values.toSeq + val executorTable = UIUtils.listingTable(executorHeaders, executorRow, executors) + + val content = +
    +
    +
    +
      +
    • ID: {app.id}
    • +
    • Description: {app.desc.name}
    • +
    • User: {app.desc.user}
    • +
    • Cores: + { + if (app.desc.maxCores == Integer.MAX_VALUE) { + "Unlimited %s granted".format(app.coresGranted) + } else { + "%s (%s granted, %s left)".format( + app.desc.maxCores, app.coresGranted, app.coresLeft) + } + } +
    • +
    • Memory per Slave: {app.desc.memoryPerSlave}
    • +
    • Submit Date: {app.submitDate}
    • +
    • State: {app.state}
    • +
    • Application Detail UI
    • +
    +
    +
    + +
    + +
    +
    +

    Executor Summary

    +
    + {executorTable} +
    +
    ; + UIUtils.sparkPage(content, "Application Info: " + app.desc.name) + } + + def executorRow(executor: ExecutorInfo): Seq[Node] = { + + {executor.id} + + {executor.worker.id} + + {executor.cores} + {executor.memory} + {executor.state} + + stdout + stderr + + + } +} diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala new file mode 100644 index 0000000000..f833c59de8 --- /dev/null +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -0,0 +1,115 @@ +package spark.deploy.master.ui + +import akka.dispatch.Await +import akka.pattern.ask +import akka.util.duration._ + +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import spark.deploy.{RequestMasterState, DeployWebUI, MasterState} +import spark.Utils +import spark.ui.UIUtils +import spark.deploy.master.{ApplicationInfo, WorkerInfo} + +class IndexPage(parent: MasterWebUI) { + val master = parent.master + implicit val timeout = parent.timeout + + /** Index view listing applications and executors */ + def render(request: HttpServletRequest): Seq[Node] = { + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val state = Await.result(stateFuture, 3 seconds) + + val workerHeaders = Seq("Id", "Address", "State", "Cores", "Memory") + val workers = state.workers.sortBy(_.id) + val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers) + + val appHeaders = Seq("ID", "Description", "Cores", "Memory per Node", "Submit Time", "User", + "State", "Duration") + val activeApps = state.activeApps.sortBy(_.startTime).reverse + val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps) + val completedApps = state.completedApps.sortBy(_.endTime).reverse + val completedAppsTable = UIUtils.listingTable(appHeaders, appRow, completedApps) + + val content = +
    +
    +
    +
      +
    • URL:{state.uri}
    • +
    • Workers:{state.workers.size}
    • +
    • Cores: {state.workers.map(_.cores).sum} Total, + {state.workers.map(_.coresUsed).sum} Used
    • +
    • Memory: + {Utils.memoryMegabytesToString(state.workers.map(_.memory).sum)} Total, + {Utils.memoryMegabytesToString(state.workers.map(_.memoryUsed).sum)} Used
    • +
    • Applications: + {state.activeApps.size} Running, + {state.completedApps.size} Completed
    • +
    +
    +
    + +
    +
    +

    Workers

    +
    + {workerTable} +
    +
    + +
    + +
    +
    +

    Running Applications

    +
    + {activeAppsTable} +
    +
    + +
    + +
    +
    +

    Completed Applications

    +
    + {completedAppsTable} +
    +
    ; + UIUtils.sparkPage(content, "Spark Master: " + state.uri) + } + + def workerRow(worker: WorkerInfo): Seq[Node] = { + + + {worker.id} + + {worker.host}:{worker.port} + {worker.state} + {worker.cores} ({worker.coresUsed} Used) + {Utils.memoryMegabytesToString(worker.memory)} + ({Utils.memoryMegabytesToString(worker.memoryUsed)} Used) + + } + + + def appRow(app: ApplicationInfo): Seq[Node] = { + + + {app.id} + + {app.desc.name} + + {app.coresGranted} + + {Utils.memoryMegabytesToString(app.desc.memoryPerSlave)} + {DeployWebUI.formatDate(app.submitDate)} + {app.desc.user} + {app.state.toString} + {DeployWebUI.formatDuration(app.duration)} + + } +} diff --git a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala new file mode 100644 index 0000000000..5fd17f10c6 --- /dev/null +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -0,0 +1,50 @@ +package spark.deploy.master.ui + +import akka.actor.ActorRef +import akka.util.Duration + +import javax.servlet.http.HttpServletRequest + +import org.eclipse.jetty.server.Handler + +import spark.{Logging, Utils} +import spark.ui.JettyUtils +import spark.ui.JettyUtils._ + +/** + * Web UI server for the standalone master. + */ +private[spark] +class MasterWebUI(val master: ActorRef) extends Logging { + implicit val timeout = Duration.create( + System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + val host = Utils.localHostName() + val port = Option(System.getProperty("master.ui.port")) + .getOrElse(MasterWebUI.DEFAULT_PORT).toInt + + val applicationPage = new ApplicationPage(this) + val indexPage = new IndexPage(this) + + def start() { + try { + val (server, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) + } catch { + case e: Exception => + logError("Failed to create Master JettyUtils", e) + System.exit(1) + } + } + + val handlers = Array[(String, Handler)]( + ("/static", createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR)), + ("/app/json", (request: HttpServletRequest) => applicationPage.renderJson(request)), + ("/app", (request: HttpServletRequest) => applicationPage.render(request)), + ("*", (request: HttpServletRequest) => indexPage.render(request)) + ) +} + +object MasterWebUI { + val STATIC_RESOURCE_DIR = "spark/webui/static" + val DEFAULT_PORT = "8080" +} \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 3878fe3f7b..690bdfe128 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -14,6 +14,7 @@ import spark.deploy.LaunchExecutor import spark.deploy.RegisterWorkerFailed import spark.deploy.master.Master import java.io.File +import ui.WorkerWebUI private[spark] class Worker( host: String, diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala deleted file mode 100644 index 805e7b52db..0000000000 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ /dev/null @@ -1,148 +0,0 @@ -package spark.deploy.worker - -import akka.actor.ActorRef -import akka.dispatch.Await -import akka.pattern.ask -import akka.util.{Duration, Timeout} -import akka.util.duration._ -import java.io.File -import javax.servlet.http.HttpServletRequest -import net.liftweb.json.JsonAST.JValue -import org.eclipse.jetty.server.Handler -import scala.io.Source -import spark.{Utils, Logging} -import spark.deploy.{JsonProtocol, WorkerState, RequestWorkerState} -import spark.ui.{JettyUtils => UtilsWebUI} -import spark.ui.JettyUtils._ -import xml.Node - -/** - * Web UI server for the standalone worker. - */ -private[spark] -class WorkerWebUI(worker: ActorRef, workDir: File) extends Logging { - implicit val timeout = Timeout( - Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")) - val host = Utils.localHostName() - val port = Option(System.getProperty("wroker.ui.port")) - .getOrElse(WorkerWebUI.DEFAULT_PORT).toInt - - val handlers = Array[(String, Handler)]( - ("/static", createStaticHandler(WorkerWebUI.STATIC_RESOURCE_DIR)), - ("/log", (request: HttpServletRequest) => log(request)), - ("/json", (request: HttpServletRequest) => indexJson), - ("*", (request: HttpServletRequest) => index) - ) - - def start() { - try { - val (server, boundPort) = UtilsWebUI.startJettyServer("0.0.0.0", port, handlers) - logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Worker JettyUtils", e) - System.exit(1) - } - } - - def indexJson(): JValue = { - val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] - val workerState = Await.result(stateFuture, 3 seconds) - JsonProtocol.writeWorkerState(workerState) - } - - def index(): Seq[Node] = { - val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] - val workerState = Await.result(stateFuture, 3 seconds) - val content = -
    -
    -
    -
      -
    • ID: {workerState.workerId}
    • -
    • - Master URL: {workerState.masterUrl} -
    • -
    • Cores: {workerState.cores} ({workerState.coresUsed} Used)
    • -
    • Memory: {Utils.memoryMegabytesToString(workerState.memory)} - ({Utils.memoryMegabytesToString(workerState.memoryUsed)} Used)
    • -
    -

    Back to Master

    -
    -
    -
    - -
    -
    -

    Running Executors {workerState.executors.size}

    -
    - {executorTable(workerState.executors)} -
    -
    -
    - -
    -
    -

    Finished Executors

    -
    - {executorTable(workerState.finishedExecutors)} -
    -
    ; - - UtilsWebUI.sparkPage(content, "Spark Worker on %s:%s".format(workerState.host, workerState.port)) - } - - def executorTable(executors: Seq[ExecutorRunner]): Seq[Node] = { - - - - - - - - - - - - {executors.map(executorRow)} - -
    ExecutorIDCoresMemoryJob DetailsLogs
    - } - - def executorRow(executor: ExecutorRunner): Seq[Node] = { - - {executor.execId} - {executor.cores} - {Utils.memoryMegabytesToString(executor.memory)} - -
      -
    • ID: {executor.appId}
    • -
    • Name: {executor.appDesc.name}
    • -
    • User: {executor.appDesc.user}
    • -
    - - - stdout - stderr - - - } - - def log(request: HttpServletRequest): String = { - val appId = request.getParameter("appId") - val executorId = request.getParameter("executorId") - val logType = request.getParameter("logType") - val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val source = Source.fromFile(path) - val lines = source.mkString - source.close() - lines - } -} - -object WorkerWebUI { - val STATIC_RESOURCE_DIR = "spark/webui/static" - val DEFAULT_PORT="8081" -} diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala new file mode 100644 index 0000000000..a6aba59e9f --- /dev/null +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -0,0 +1,97 @@ +package spark.deploy.worker.ui + +import akka.dispatch.Await +import akka.pattern.ask +import akka.util.duration._ + +import javax.servlet.http.HttpServletRequest + +import net.liftweb.json.JsonAST.JValue + +import spark.deploy.{RequestWorkerState, JsonProtocol, WorkerState} +import spark.deploy.worker.ExecutorRunner +import spark.Utils +import spark.ui.UIUtils + +import xml.Node + +class IndexPage(parent: WorkerWebUI) { + val worker = parent.worker + val timeout = parent.timeout + + def renderJson(request: HttpServletRequest): JValue = { + val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] + val workerState = Await.result(stateFuture, 3 seconds) + JsonProtocol.writeWorkerState(workerState) + } + + def render(request: HttpServletRequest): Seq[Node] = { + val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] + val workerState = Await.result(stateFuture, 3 seconds) + + val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") + val runningExecutorTable = + UIUtils.listingTable(executorHeaders, executorRow, workerState.executors) + val finishedExecutorTable = + UIUtils.listingTable(executorHeaders, executorRow, workerState.finishedExecutors) + + val content = +
    +
    +
    +
      +
    • ID: {workerState.workerId}
    • +
    • + Master URL: {workerState.masterUrl} +
    • +
    • Cores: {workerState.cores} ({workerState.coresUsed} Used)
    • +
    • Memory: {Utils.memoryMegabytesToString(workerState.memory)} + ({Utils.memoryMegabytesToString(workerState.memoryUsed)} Used)
    • +
    +

    Back to Master

    +
    +
    +
    + +
    +
    +

    Running Executors {workerState.executors.size}

    +
    + {runningExecutorTable} +
    +
    +
    + +
    +
    +

    Finished Executors

    +
    + {finishedExecutorTable} +
    +
    ; + + UIUtils.sparkPage(content, "Spark Worker on %s:%s".format(workerState.host, workerState.port)) + } + + def executorRow(executor: ExecutorRunner): Seq[Node] = { + + {executor.execId} + {executor.cores} + {Utils.memoryMegabytesToString(executor.memory)} + +
      +
    • ID: {executor.appId}
    • +
    • Name: {executor.appDesc.name}
    • +
    • User: {executor.appDesc.user}
    • +
    + + + stdout + stderr + + + } + +} diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala new file mode 100644 index 0000000000..abfc847527 --- /dev/null +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -0,0 +1,64 @@ +package spark.deploy.worker.ui + +import akka.actor.ActorRef +import akka.util.{Duration, Timeout} + +import java.io.File + +import javax.servlet.http.HttpServletRequest + +import org.eclipse.jetty.server.Handler + +import scala.io.Source + +import spark.{Utils, Logging} +import spark.ui.JettyUtils +import spark.ui.JettyUtils._ + +/** + * Web UI server for the standalone worker. + */ +private[spark] +class WorkerWebUI(val worker: ActorRef, val workDir: File) extends Logging { + implicit val timeout = Timeout( + Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")) + val host = Utils.localHostName() + val port = Option(System.getProperty("wroker.ui.port")) + .getOrElse(WorkerWebUI.DEFAULT_PORT).toInt + + val indexPage = new IndexPage(this) + + val handlers = Array[(String, Handler)]( + ("/static", createStaticHandler(WorkerWebUI.STATIC_RESOURCE_DIR)), + ("/log", (request: HttpServletRequest) => log(request)), + ("/json", (request: HttpServletRequest) => indexPage.renderJson(request)), + ("*", (request: HttpServletRequest) => indexPage.render(request)) + ) + + def start() { + try { + val (server, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) + } catch { + case e: Exception => + logError("Failed to create Worker JettyUtils", e) + System.exit(1) + } + } + + def log(request: HttpServletRequest): String = { + val appId = request.getParameter("appId") + val executorId = request.getParameter("executorId") + val logType = request.getParameter("logType") + val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) + val source = Source.fromFile(path) + val lines = source.mkString + source.close() + lines + } +} + +object WorkerWebUI { + val STATIC_RESOURCE_DIR = "spark/webui/static" + val DEFAULT_PORT="8081" +} diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 8bb343163a..93e7129fc0 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -15,7 +15,6 @@ import spark.Logging import xml.Node - /** Utilities for launching a web server using Jetty's HTTP Server class */ private[spark] object JettyUtils extends Logging { // Base type for a function that returns something based on an HTTP request. Allows for -- cgit v1.2.3 From a86bb459e220cfb1375a6f31a94680ab24004b69 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 21 Jun 2013 14:56:57 -0700 Subject: Showing shuffle status and purging old stages --- core/src/main/scala/spark/ui/SparkUI.scala | 2 +- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 12 ++++++- .../main/scala/spark/ui/jobs/JobProgressUI.scala | 37 +++++++++++++++++++++- core/src/main/scala/spark/ui/jobs/StagePage.scala | 20 +++++------- 4 files changed, 56 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 7428e7a343..deb6284a66 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -22,7 +22,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { ) val storage = new BlockManagerUI(sc) val jobs = new JobProgressUI(sc) - val allHandlers = handlers ++ storage.getHandlers ++ jobs.getHandlers + val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ handlers def start() { /** Start an HTTP server to run the Web interface */ diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 811baae811..0174e1ddd1 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -19,7 +19,8 @@ class IndexPage(parent: JobProgressUI) { val dateFmt = parent.dateFmt def render(request: HttpServletRequest): Seq[Node] = { - val stageHeaders = Seq("Stage ID", "Origin", "Submitted", "Duration", "Tasks: Complete/Total") + val stageHeaders = Seq("Stage ID", "Origin", "Submitted", "Duration", "Tasks: Complete/Total", + "Shuffle Activity") val activeStages = listener.activeStages.toSeq val completedStages = listener.completedStages.toSeq @@ -44,6 +45,14 @@ class IndexPage(parent: JobProgressUI) { case Some(t) => dateFmt.format(new Date(t)) case None => "Unknown" } + val (read, write) = (listener.hasShuffleRead(s.id), listener.hasShuffleWrite(s.id)) + val shuffleString = (read, write) match { + case (true, true) => "Read/Write" + case (true, false) => "Read" + case (false, true) => "Write" + case _ => "None" + } + {s.id} {s.origin} @@ -56,6 +65,7 @@ class IndexPage(parent: JobProgressUI) { case _ => }} + {shuffleString} } } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 97fda84e06..d260523590 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -33,11 +33,16 @@ private[spark] class JobProgressUI(sc: SparkContext) { } private[spark] class JobProgressListener extends SparkListener { + // TODO(pwendell) Currently does not handle entirely failed stages + + // How many stages to remember + val RETAINED_STAGES = 1000 + val activeStages = HashSet[Stage]() val stageToTasksComplete = HashMap[Int, Int]() val stageToTasksFailed = HashMap[Int, Int]() val stageToTaskInfos = HashMap[Int, ArrayBuffer[(TaskInfo, TaskMetrics)]]() - val completedStages = ListBuffer[Stage]() // Todo (pwendell): Evict these over time + val completedStages = ListBuffer[Stage]() override def onJobStart(jobStart: SparkListenerJobStart) { } @@ -45,6 +50,18 @@ private[spark] class JobProgressListener extends SparkListener { val stage = stageCompleted.stageInfo.stage activeStages -= stage stage +=: completedStages + if (completedStages.size > RETAINED_STAGES) purgeStages() + } + + /** Remove and garbage collect old stages */ + def purgeStages() { + val toRemove = RETAINED_STAGES / 10 + completedStages.takeRight(toRemove).foreach( s => { + stageToTasksComplete.remove(s.id) + stageToTasksFailed.remove(s.id) + stageToTaskInfos.remove(s.id) + }) + completedStages.trimEnd(toRemove) } override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = @@ -64,4 +81,22 @@ private[spark] class JobProgressListener extends SparkListener { } override def onJobEnd(jobEnd: SparkListenerEvents) { } + + /** Is this stage's input from a shuffle read. */ + def hasShuffleRead(stageID: Int): Boolean = { + // This is written in a slightly complicated way to avoid having to scan all tasks + for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { + if (s._2 != null) return s._2.shuffleReadMetrics.isDefined + } + return false // No tasks have finished for this stage + } + + /** Is this stage's output to a shuffle write. */ + def hasShuffleWrite(stageID: Int): Boolean = { + // This is written in a slightly complicated way to avoid having to scan all tasks + for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { + if (s._2 != null) return s._2.shuffleWriteMetrics.isDefined + } + return false // No tasks have finished for this stage + } } \ No newline at end of file diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 74ac811cef..3c23a86d12 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -4,8 +4,6 @@ import java.util.Date import javax.servlet.http.HttpServletRequest -import scala.collection.mutable.ListBuffer - import spark.ui.UIUtils._ import spark.util.Distribution import spark.scheduler.cluster.TaskInfo @@ -22,13 +20,13 @@ class StagePage(parent: JobProgressUI) { val stageId = request.getParameter("id").toInt val tasks = listener.stageToTaskInfos(stageId) - val hasShuffleRead = tasks.head._2.shuffleReadMetrics.isDefined - val hasShuffleWrite = tasks.head._2.shuffleWriteMetrics.isDefined + val shuffleRead = listener.hasShuffleRead(stageId) + val shuffleWrite = listener.hasShuffleWrite(stageId) - val taskHeaders = - ListBuffer("Task ID", "Service Time (ms)", "Locality Level", "Worker", "Launch Time") - if (hasShuffleRead) { taskHeaders += "Shuffle Read (bytes)" } - if (hasShuffleWrite) { taskHeaders += "Shuffle Write (bytes)" } + val taskHeaders: Seq[String] = + Seq("Task ID", "Service Time (ms)", "Locality Level", "Worker", "Launch Time") ++ + {if (shuffleRead) Seq("Shuffle Read (bytes)") else Nil} ++ + {if (shuffleWrite) Seq("Shuffle Write (bytes)") else Nil} val taskTable = listingTable(taskHeaders, taskRow, tasks) @@ -46,8 +44,8 @@ class StagePage(parent: JobProgressUI) { val listings: Seq[Seq[String]] = Seq(serviceQuantiles, - if (hasShuffleRead) shuffleReadQuantiles else Nil, - if (hasShuffleWrite) shuffleWriteQuantiles else Nil) + if (shuffleRead) shuffleReadQuantiles else Nil, + if (shuffleWrite) shuffleWriteQuantiles else Nil) val quantileHeaders = Seq("Metric", "Min", "25%", "50%", "75%", "Max") val quantileTable = listingTable(quantileHeaders, quantileRow, listings) @@ -72,6 +70,4 @@ class StagePage(parent: JobProgressUI) { {metrics.shuffleWriteMetrics.map{m => {m.shuffleBytesWritten}}.getOrElse("") } } - - } -- cgit v1.2.3 From f91e1c48224366c9cf7a32e99d67e966e540b86b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 21 Jun 2013 15:11:42 -0700 Subject: Linking RDD information when available in stages --- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 13 ++++++++++--- core/src/main/scala/spark/ui/storage/RDDPage.scala | 2 +- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 0174e1ddd1..8ebde7a379 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -10,6 +10,7 @@ import scala.Some import spark.scheduler.Stage import spark.ui.UIUtils._ +import spark.storage.StorageLevel import xml.{NodeSeq, Node} @@ -20,7 +21,7 @@ class IndexPage(parent: JobProgressUI) { def render(request: HttpServletRequest): Seq[Node] = { val stageHeaders = Seq("Stage ID", "Origin", "Submitted", "Duration", "Tasks: Complete/Total", - "Shuffle Activity") + "Shuffle Activity", "RDDs") val activeStages = listener.activeStages.toSeq val completedStages = listener.completedStages.toSeq @@ -46,7 +47,7 @@ class IndexPage(parent: JobProgressUI) { case None => "Unknown" } val (read, write) = (listener.hasShuffleRead(s.id), listener.hasShuffleWrite(s.id)) - val shuffleString = (read, write) match { + val shuffleInfo = (read, write) match { case (true, true) => "Read/Write" case (true, false) => "Read" case (false, true) => "Write" @@ -65,7 +66,13 @@ class IndexPage(parent: JobProgressUI) { case _ => }} - {shuffleString} + {shuffleInfo} + {if (s.rdd.getStorageLevel != StorageLevel.NONE) { + + {Option(s.rdd.name).getOrElse(s.rdd.id)} + + }} + } } diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 7628fde4aa..466f4643bc 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -65,7 +65,7 @@ class RDDPage(parent: BlockManagerUI) {
    ++ {workerTable}; - headerSparkPage(content, "RDD Info: " + id) + headerSparkPage(content, "RDD Info: " + rddInfo.name) } def blockRow(blk: (String, BlockStatus)): Seq[Node] = { -- cgit v1.2.3 From 9a24d1a2d0deb5cbd96b806a1ef3afb9353872dd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 21 Jun 2013 15:17:50 -0700 Subject: Using scala in XML imports --- core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala | 4 ++-- core/src/main/scala/spark/ui/JettyUtils.scala | 3 +-- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 3 +-- core/src/main/scala/spark/ui/jobs/StagePage.scala | 4 ++-- core/src/main/scala/spark/ui/storage/IndexPage.scala | 4 ++-- core/src/main/scala/spark/ui/storage/RDDPage.scala | 4 ++-- 6 files changed, 10 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index a6aba59e9f..9e7b2bfd92 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -8,13 +8,13 @@ import javax.servlet.http.HttpServletRequest import net.liftweb.json.JsonAST.JValue +import scala.xml.Node + import spark.deploy.{RequestWorkerState, JsonProtocol, WorkerState} import spark.deploy.worker.ExecutorRunner import spark.Utils import spark.ui.UIUtils -import xml.Node - class IndexPage(parent: WorkerWebUI) { val worker = parent.worker val timeout = parent.timeout diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 93e7129fc0..726a95594b 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -10,11 +10,10 @@ import org.eclipse.jetty.server.{Server, Request, Handler} import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler} import scala.util.{Try, Success, Failure} +import scala.xml.Node import spark.Logging -import xml.Node - /** Utilities for launching a web server using Jetty's HTTP Server class */ private[spark] object JettyUtils extends Logging { // Base type for a function that returns something based on an HTTP request. Allows for diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 8ebde7a379..87f3abc6b0 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -7,13 +7,12 @@ import java.util.Date import javax.servlet.http.HttpServletRequest import scala.Some +import scala.xml.{NodeSeq, Node} import spark.scheduler.Stage import spark.ui.UIUtils._ import spark.storage.StorageLevel -import xml.{NodeSeq, Node} - /** Page showing list of all ongoing and recently finished stages */ class IndexPage(parent: JobProgressUI) { val listener = parent.listener diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 3c23a86d12..d76c07e458 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -4,13 +4,13 @@ import java.util.Date import javax.servlet.http.HttpServletRequest +import scala.xml.Node + import spark.ui.UIUtils._ import spark.util.Distribution import spark.scheduler.cluster.TaskInfo import spark.executor.TaskMetrics -import xml.Node - /** Page showing statistics and task list for a given stage */ class StagePage(parent: JobProgressUI) { val listener = parent.listener diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index 1f9953fb6b..52b9087099 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -2,12 +2,12 @@ package spark.ui.storage import javax.servlet.http.HttpServletRequest +import scala.xml.Node + import spark.storage.{RDDInfo, StorageUtils} import spark.Utils import spark.ui.UIUtils._ -import xml.Node - /** Page showing list of RDD's currently stored in the cluster */ class IndexPage(parent: BlockManagerUI) { val sc = parent.sc diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 466f4643bc..1031b2e17f 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -2,13 +2,13 @@ package spark.ui.storage import javax.servlet.http.HttpServletRequest +import scala.xml.Node + import spark.storage.{StorageStatus, StorageUtils} import spark.ui.UIUtils._ import spark.Utils import spark.storage.BlockManagerMasterActor.BlockStatus -import xml.Node - /** Page showing storage details for a given RDD */ class RDDPage(parent: BlockManagerUI) { val sc = parent.sc -- cgit v1.2.3 From be6107ce446a25f6fccaf34085beb23e81828495 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 21 Jun 2013 21:30:56 -0700 Subject: Some tweaking with shared page header --- core/src/main/scala/spark/ui/JettyUtils.scala | 15 ++++++++++++--- core/src/main/scala/spark/ui/SparkUI.scala | 2 +- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 2 +- core/src/main/scala/spark/ui/jobs/JobProgressUI.scala | 2 +- core/src/main/scala/spark/ui/jobs/StagePage.scala | 2 +- core/src/main/scala/spark/ui/storage/IndexPage.scala | 2 +- core/src/main/scala/spark/ui/storage/RDDPage.scala | 2 +- 7 files changed, 18 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 726a95594b..225daf5778 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -12,7 +12,7 @@ import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHa import scala.util.{Try, Success, Failure} import scala.xml.Node -import spark.Logging +import spark.{Utils, SparkContext, Logging} /** Utilities for launching a web server using Jetty's HTTP Server class */ private[spark] object JettyUtils extends Logging { @@ -97,9 +97,18 @@ private[spark] object JettyUtils extends Logging { object UIUtils { /** Returns a page containing the supplied content and the spark web ui headers */ - def headerSparkPage(content: => Seq[Node], title: String): Seq[Node] = { + def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String): Seq[Node] = { val newContent = -

    Storage | Jobs

    ; +
    +
    +
      +
    • Master: {sc.master}
    • +
    • Application: {sc.appName}
    • +
    +

    Storage | Jobs

    +
    +
    +
    ; sparkPage(newContent ++ content, title) } diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index deb6284a66..cfa805fcf8 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -18,7 +18,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { val handlers = Seq[(String, Handler)]( ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), - ("*", (request: HttpServletRequest) => headerSparkPage(

    Test

    , "Test page")) + ("*", (request: HttpServletRequest) => headerSparkPage(

    Test

    , sc, "Test page")) ) val storage = new BlockManagerUI(sc) val jobs = new JobProgressUI(sc) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 87f3abc6b0..5a4658499a 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -30,7 +30,7 @@ class IndexPage(parent: JobProgressUI) { val content =

    Active Stages

    ++ activeStageTable ++

    Completed Stages

    ++ completedStageTable - headerSparkPage(content, "Spark Stages") + headerSparkPage(content, parent.sc, "Spark Stages") } def getElapsedTime(submitted: Option[Long], completed: Long): String = { diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index d260523590..47a6bcc2c3 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -17,7 +17,7 @@ import spark.executor.TaskMetrics import spark.Success /** Web UI showing progress status of all jobs in the given SparkContext. */ -private[spark] class JobProgressUI(sc: SparkContext) { +private[spark] class JobProgressUI(val sc: SparkContext) { val listener = new JobProgressListener val dateFmt = new SimpleDateFormat("EEE, MMM d yyyy HH:mm:ss") diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index d76c07e458..42eb1f9eef 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -53,7 +53,7 @@ class StagePage(parent: JobProgressUI) { val content =

    Summary Metrics

    ++ quantileTable ++

    Tasks

    ++ taskTable; - headerSparkPage(content, "Stage Details: %s".format(stageId)) + headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId)) } def quantileRow(data: Seq[String]): Seq[Node] = {data.map(d => {d})} diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index 52b9087099..cd7f6bd9f2 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -42,7 +42,7 @@ class IndexPage(parent: BlockManagerUI) { ++ {rddTable}; - headerSparkPage(content, "Spark Storage ") + headerSparkPage(content, parent.sc, "Spark Storage ") } def rddRow(rdd: RDDInfo): Seq[Node] = { diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 1031b2e17f..059affedbd 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -65,7 +65,7 @@ class RDDPage(parent: BlockManagerUI) {
    ++ {workerTable}; - headerSparkPage(content, "RDD Info: " + rddInfo.name) + headerSparkPage(content, parent.sc, "RDD Info: " + rddInfo.name) } def blockRow(blk: (String, BlockStatus)): Seq[Node] = { -- cgit v1.2.3 From 3b7ebdeeb86868ff1ebd269c2ed832012f22e0d1 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 22 Jun 2013 09:00:42 -0700 Subject: Handling entirely failed stages --- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 18 +++++---- .../main/scala/spark/ui/jobs/JobProgressUI.scala | 43 ++++++++++++++-------- core/src/main/scala/spark/ui/storage/RDDPage.scala | 19 ++++++---- 3 files changed, 50 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 5a4658499a..2e2dcdfbc6 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -20,15 +20,18 @@ class IndexPage(parent: JobProgressUI) { def render(request: HttpServletRequest): Seq[Node] = { val stageHeaders = Seq("Stage ID", "Origin", "Submitted", "Duration", "Tasks: Complete/Total", - "Shuffle Activity", "RDDs") + "Shuffle Activity", "Stored RDD") val activeStages = listener.activeStages.toSeq - val completedStages = listener.completedStages.toSeq + val completedStages = listener.completedStages.reverse.toSeq + val failedStages = listener.failedStages.reverse.toSeq - val activeStageTable: NodeSeq = listingTable(stageHeaders, stageRow, activeStages) - val completedStageTable = listingTable(stageHeaders, stageRow, completedStages) + val activeStageTable: NodeSeq = listingTable(stageHeaders, stageRow(), activeStages) + val completedStageTable = listingTable(stageHeaders, stageRow(), completedStages) + val failedStageTable: NodeSeq = listingTable(stageHeaders, stageRow(false), failedStages) val content =

    Active Stages

    ++ activeStageTable ++ -

    Completed Stages

    ++ completedStageTable +

    Completed Stages

    ++ completedStageTable ++ +

    Failed Stages

    ++ failedStageTable headerSparkPage(content, parent.sc, "Spark Stages") } @@ -40,7 +43,7 @@ class IndexPage(parent: JobProgressUI) { } } - def stageRow(s: Stage): Seq[Node] = { + def stageRow(showLink: Boolean = true)(s: Stage): Seq[Node] = { val submissionTime = s.submissionTime match { case Some(t) => dateFmt.format(new Date(t)) case None => "Unknown" @@ -54,7 +57,8 @@ class IndexPage(parent: JobProgressUI) { } - {s.id} + {if (showLink) {{s.id}} + else {{s.id}}} {s.origin} {submissionTime} {getElapsedTime(s.submissionTime, diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 47a6bcc2c3..dea4f0e3b0 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -33,35 +33,37 @@ private[spark] class JobProgressUI(val sc: SparkContext) { } private[spark] class JobProgressListener extends SparkListener { - // TODO(pwendell) Currently does not handle entirely failed stages - // How many stages to remember val RETAINED_STAGES = 1000 val activeStages = HashSet[Stage]() + val completedStages = ListBuffer[Stage]() + val failedStages = ListBuffer[Stage]() + val stageToTasksComplete = HashMap[Int, Int]() val stageToTasksFailed = HashMap[Int, Int]() val stageToTaskInfos = HashMap[Int, ArrayBuffer[(TaskInfo, TaskMetrics)]]() - val completedStages = ListBuffer[Stage]() - override def onJobStart(jobStart: SparkListenerJobStart) { } + override def onJobStart(jobStart: SparkListenerJobStart) {} override def onStageCompleted(stageCompleted: StageCompleted) = { val stage = stageCompleted.stageInfo.stage activeStages -= stage - stage +=: completedStages - if (completedStages.size > RETAINED_STAGES) purgeStages() + completedStages += stage + trimIfNecessary(completedStages) } - /** Remove and garbage collect old stages */ - def purgeStages() { - val toRemove = RETAINED_STAGES / 10 - completedStages.takeRight(toRemove).foreach( s => { - stageToTasksComplete.remove(s.id) - stageToTasksFailed.remove(s.id) - stageToTaskInfos.remove(s.id) - }) - completedStages.trimEnd(toRemove) + /** If stages is too large, remove and garbage collect old stages */ + def trimIfNecessary(stages: ListBuffer[Stage]) { + if (stages.size > RETAINED_STAGES) { + val toRemove = RETAINED_STAGES / 10 + stages.takeRight(toRemove).foreach( s => { + stageToTasksComplete.remove(s.id) + stageToTasksFailed.remove(s.id) + stageToTaskInfos.remove(s.id) + }) + stages.trimEnd(toRemove) + } } override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = @@ -80,7 +82,16 @@ private[spark] class JobProgressListener extends SparkListener { stageToTaskInfos(sid) = taskList } - override def onJobEnd(jobEnd: SparkListenerEvents) { } + override def onJobEnd(jobEnd: SparkListenerEvents) { + jobEnd match { + case failed: SparkListenerJobFailed => + val stage = failed.failedStage + activeStages -= stage + failedStages += stage + trimIfNecessary(failedStages) + case _ => + } + } /** Is this stage's input from a shuffle read. */ def hasShuffleRead(stageID: Int): Boolean = { diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 059affedbd..73f151959e 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -56,14 +56,19 @@ class RDDPage(parent: BlockManagerUI) { -
    -
    -
    -

    RDD Summary

    -
    {blockTable} -
    +
    +
    +
    + {workerTable} +
    +
    +
    +
    +
    +

    RDD Summary

    + {blockTable}
    -
    ++ {workerTable}; +
    ; headerSparkPage(content, parent.sc, "RDD Info: " + rddInfo.name) } -- cgit v1.2.3 From 7e9f1ed0decbbb261432dcbcd2179b829dbcbc82 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 22 Jun 2013 10:17:29 -0700 Subject: Some cleanup of styling --- core/src/main/scala/spark/ui/JettyUtils.scala | 45 +++++++++++++++++++-------- core/src/main/scala/spark/ui/SparkUI.scala | 4 ++- 2 files changed, 35 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 225daf5778..96d773979f 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -41,11 +41,25 @@ private[spark] object JettyUtils extends Logging { response.setStatus(HttpServletResponse.SC_OK) baseRequest.setHandled(true) val result = responder(request) + response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") response.getWriter().println(extractFn(result)) } } } + def createRedirectHandler(newPath: String): Handler = { + new AbstractHandler { + def handle(target: String, + baseRequest: Request, + request: HttpServletRequest, + response: HttpServletResponse) { + response.setStatus(302) + response.setHeader("Location", baseRequest.getRootURL + newPath) + baseRequest.setHandled(true) + } + } + } + /** Creates a handler for serving files from a static directory. */ def createStaticHandler(resourceBase: String): ResourceHandler = { val staticHandler = new ResourceHandler @@ -66,13 +80,9 @@ private[spark] object JettyUtils extends Logging { */ def startJettyServer(ip: String, port: Int, handlers: Seq[(String, Handler)]): (Server, Int) = { val handlersToRegister = handlers.map { case(path, handler) => - if (path == "*") { - handler - } else { - val contextHandler = new ContextHandler(path) - contextHandler.setHandler(handler) - contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] - } + val contextHandler = new ContextHandler(path) + contextHandler.setHandler(handler) + contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] } val handlerList = new HandlerList @@ -99,13 +109,21 @@ object UIUtils { /** Returns a page containing the supplied content and the spark web ui headers */ def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String): Seq[Node] = { val newContent = -
    -
    +
    +
    +
    + +
    +
    +
    • Master: {sc.master}
    • Application: {sc.appName}
    • +
    • Executors: {sc.getExecutorStorageStatus.size}
    -

    Storage | Jobs


    ; @@ -128,10 +146,11 @@ object UIUtils {
    -
    +
    -

    +

    +
    +

    {title}

    diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index cfa805fcf8..66dcf07384 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -12,13 +12,15 @@ import spark.ui.JettyUtils._ /** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { + // TODO(pwendell): It would be nice to add a view that prints out environment information + val host = Utils.localHostName() val port = Option(System.getProperty("spark.ui.port")).getOrElse(SparkUI.DEFAULT_PORT).toInt var boundPort: Option[Int] = None val handlers = Seq[(String, Handler)]( ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), - ("*", (request: HttpServletRequest) => headerSparkPage(

    Test

    , sc, "Test page")) + ("/", createRedirectHandler("/stages")) ) val storage = new BlockManagerUI(sc) val jobs = new JobProgressUI(sc) -- cgit v1.2.3 From b5df1cd668e45fd0cc22c1666136d05548cae3e9 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 17:12:39 -0700 Subject: ADD_JARS environment variable for spark-shell --- docs/scala-programming-guide.md | 10 ++++++++-- repl/src/main/scala/spark/repl/SparkILoop.scala | 9 +++++++-- 2 files changed, 15 insertions(+), 4 deletions(-) diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index b0da130fcb..e9cf9ef36f 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -43,12 +43,18 @@ new SparkContext(master, appName, [sparkHome], [jars]) The `master` parameter is a string specifying a [Spark or Mesos cluster URL](#master-urls) to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later. -In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable. For example, to run on four cores, use +In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `spark-shell` on four cores, use {% highlight bash %} $ MASTER=local[4] ./spark-shell {% endhighlight %} +Or, to also add `code.jar` to its classpath, use: + +{% highlight bash %} +$ MASTER=local[4] ADD_JARS=code.jar ./spark-shell +{% endhighlight %} + ### Master URLs The master URL passed to Spark can be in one of the following formats: @@ -78,7 +84,7 @@ If you want to run your job on a cluster, you will need to specify the two optio * `sparkHome`: The path at which Spark is installed on your worker machines (it should be the same on all of them). * `jars`: A list of JAR files on the local machine containing your job's code and any dependencies, which Spark will deploy to all the worker nodes. You'll need to package your job into a set of JARs using your build system. For example, if you're using SBT, the [sbt-assembly](https://github.com/sbt/sbt-assembly) plugin is a good way to make a single JAR with your code and dependencies. -If you run `spark-shell` on a cluster, any classes you define in the shell will automatically be distributed. +If you run `spark-shell` on a cluster, you can add JARs to it by specifying the `ADD_JARS` environment variable before you launch it. This variable should contain a comma-separated list of JARs. For example, `ADD_JARS=a.jar,b.jar ./spark-shell` will launch a shell with `a.jar` and `b.jar` on its classpath. In addition, any new classes you define in the shell will automatically be distributed. # Resilient Distributed Datasets (RDDs) diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/spark/repl/SparkILoop.scala index 23556dbc8f..86eed090d0 100644 --- a/repl/src/main/scala/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/spark/repl/SparkILoop.scala @@ -822,7 +822,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: spark.repl.Main.interp.out.println("Spark context available as sc."); spark.repl.Main.interp.out.flush(); """) - command("import spark.SparkContext._"); + command("import spark.SparkContext._") } echo("Type in expressions to have them evaluated.") echo("Type :help for more information.") @@ -838,7 +838,8 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: if (prop != null) prop else "local" } } - sparkContext = new SparkContext(master, "Spark shell") + val jars = Option(System.getenv("ADD_JARS")).map(_.split(',')).getOrElse(new Array[String](0)) + sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars) sparkContext } @@ -850,6 +851,10 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: printWelcome() echo("Initializing interpreter...") + // Add JARS specified in Spark's ADD_JARS variable to classpath + val jars = Option(System.getenv("ADD_JARS")).map(_.split(',')).getOrElse(new Array[String](0)) + jars.foreach(settings.classpath.append(_)) + this.settings = settings createInterpreter() -- cgit v1.2.3 From 0e0f9d3069039f03bbf5eefe3b0637c89fddf0f1 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 17:44:04 -0700 Subject: Fix search path for REPL class loader to really find added JARs --- core/src/main/scala/spark/executor/Executor.scala | 38 +++++++++++++---------- repl/src/main/scala/spark/repl/SparkILoop.scala | 4 ++- 2 files changed, 25 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 8bebfafce4..2bf55ea9a9 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -42,7 +42,8 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert // Create our ClassLoader and set it on this thread private val urlClassLoader = createClassLoader() - Thread.currentThread.setContextClassLoader(urlClassLoader) + private val replClassLoader = addReplClassLoaderIfNeeded(urlClassLoader) + Thread.currentThread.setContextClassLoader(replClassLoader) // Make any thread terminations due to uncaught exceptions kill the entire // executor process to avoid surprising stalls. @@ -88,7 +89,7 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert override def run() { val startTime = System.currentTimeMillis() SparkEnv.set(env) - Thread.currentThread.setContextClassLoader(urlClassLoader) + Thread.currentThread.setContextClassLoader(replClassLoader) val ser = SparkEnv.get.closureSerializer.newInstance() logInfo("Running task ID " + taskId) context.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER) @@ -153,26 +154,31 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert val urls = currentJars.keySet.map { uri => new File(uri.split("/").last).toURI.toURL }.toArray - loader = new URLClassLoader(urls, loader) + new ExecutorURLClassLoader(urls, loader) + } - // If the REPL is in use, add another ClassLoader that will read - // new classes defined by the REPL as the user types code + /** + * If the REPL is in use, add another ClassLoader that will read + * new classes defined by the REPL as the user types code + */ + private def addReplClassLoaderIfNeeded(parent: ClassLoader): ClassLoader = { val classUri = System.getProperty("spark.repl.class.uri") if (classUri != null) { logInfo("Using REPL class URI: " + classUri) - loader = { - try { - val klass = Class.forName("spark.repl.ExecutorClassLoader") - .asInstanceOf[Class[_ <: ClassLoader]] - val constructor = klass.getConstructor(classOf[String], classOf[ClassLoader]) - constructor.newInstance(classUri, loader) - } catch { - case _: ClassNotFoundException => loader - } + try { + val klass = Class.forName("spark.repl.ExecutorClassLoader") + .asInstanceOf[Class[_ <: ClassLoader]] + val constructor = klass.getConstructor(classOf[String], classOf[ClassLoader]) + return constructor.newInstance(classUri, parent) + } catch { + case _: ClassNotFoundException => + logError("Could not find spark.repl.ExecutorClassLoader on classpath!") + System.exit(1) + null } + } else { + return parent } - - return new ExecutorURLClassLoader(Array(), loader) } /** diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/spark/repl/SparkILoop.scala index 86eed090d0..59f9d05683 100644 --- a/repl/src/main/scala/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/spark/repl/SparkILoop.scala @@ -838,7 +838,9 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: if (prop != null) prop else "local" } } - val jars = Option(System.getenv("ADD_JARS")).map(_.split(',')).getOrElse(new Array[String](0)) + val jars = Option(System.getenv("ADD_JARS")).map(_.split(',')) + .getOrElse(new Array[String](0)) + .map(new java.io.File(_).getAbsolutePath) sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars) sparkContext } -- cgit v1.2.3 From 78ffe164b33c6b11a2e511442605acd2f795a1b5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 23 Jun 2013 10:07:16 -0700 Subject: Clone the zero value for each key in foldByKey The old version reused the object within each task, leading to overwriting of the object when a mutable type is used, which is expected to be common in fold. Conflicts: core/src/test/scala/spark/ShuffleSuite.scala --- core/src/main/scala/spark/PairRDDFunctions.scala | 15 ++++++++++++--- core/src/test/scala/spark/ShuffleSuite.scala | 22 ++++++++++++++++++++++ 2 files changed, 34 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index fa4bbfc76f..7630fe7803 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -1,5 +1,6 @@ package spark +import java.nio.ByteBuffer import java.util.{Date, HashMap => JHashMap} import java.text.SimpleDateFormat @@ -64,8 +65,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( throw new SparkException("Default partitioner cannot partition array keys.") } } - val aggregator = - new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners) + val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners) if (self.partitioner == Some(partitioner)) { self.mapPartitions(aggregator.combineValuesByKey(_), true) } else if (mapSideCombine) { @@ -97,7 +97,16 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * list concatenation, 0 for addition, or 1 for multiplication.). */ def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): RDD[(K, V)] = { - combineByKey[V]({v: V => func(zeroValue, v)}, func, func, partitioner) + // Serialize the zero value to a byte array so that we can get a new clone of it on each key + val zeroBuffer = SparkEnv.get.closureSerializer.newInstance().serialize(zeroValue) + val zeroArray = new Array[Byte](zeroBuffer.limit) + zeroBuffer.get(zeroArray) + + // When deserializing, use a lazy val to create just one instance of the serializer per task + lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance() + def createZero() = cachedSerializer.deserialize[V](ByteBuffer.wrap(zeroArray)) + + combineByKey[V]((v: V) => func(createZero(), v), func, func, partitioner) } /** diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 1916885a73..0c1ec29f96 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -392,6 +392,28 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { assert(nonEmptyBlocks.size <= 4) } + test("foldByKey") { + sc = new SparkContext("local", "test") + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val sums = pairs.foldByKey(0)(_+_).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("foldByKey with mutable result type") { + sc = new SparkContext("local", "test") + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val bufs = pairs.mapValues(v => ArrayBuffer(v)).cache() + // Fold the values using in-place mutation + val sums = bufs.foldByKey(new ArrayBuffer[Int])(_ ++= _).collect() + assert(sums.toSet === Set((1, ArrayBuffer(1, 2, 3, 1)), (2, ArrayBuffer(1)))) + // Check that the mutable objects in the original RDD were not changed + assert(bufs.collect().toSet === Set( + (1, ArrayBuffer(1)), + (1, ArrayBuffer(2)), + (1, ArrayBuffer(3)), + (1, ArrayBuffer(1)), + (2, ArrayBuffer(1)))) + } } object ShuffleSuite { -- cgit v1.2.3 From 8955787a596216a35ad4ec52b57331aa40444bef Mon Sep 17 00:00:00 2001 From: James Phillpotts Date: Mon, 24 Jun 2013 09:15:17 +0100 Subject: Twitter API v1 is retired - username/password auth no longer possible --- .../main/scala/spark/streaming/StreamingContext.scala | 16 +--------------- 1 file changed, 1 insertion(+), 15 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index f97e47ada0..05be6bd58a 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.apache.hadoop.fs.Path import twitter4j.Status -import twitter4j.auth.{Authorization, BasicAuthorization} +import twitter4j.auth.Authorization /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic @@ -361,20 +361,6 @@ class StreamingContext private ( fileStream[LongWritable, Text, TextInputFormat](directory).map(_._2.toString) } - /** - * Create a input stream that returns tweets received from Twitter. - * @param username Twitter username - * @param password Twitter password - * @param filters Set of filter strings to get only those tweets that match them - * @param storageLevel Storage level to use for storing the received objects - */ - def twitterStream( - username: String, - password: String, - filters: Seq[String] = Nil, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[Status] = twitterStream(new BasicAuthorization(username, password), filters, storageLevel) - /** * Create a input stream that returns tweets received from Twitter. * @param twitterAuth Twitter4J authentication -- cgit v1.2.3 From f6e64b5cd6e6ac5ae3bee05af2832e1f71992310 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 22 Jun 2013 10:40:41 -0700 Subject: Updating based on changes to JobLogger (and one small change to JobLogger) --- .../main/scala/spark/scheduler/DAGScheduler.scala | 6 +++--- core/src/main/scala/spark/scheduler/JobLogger.scala | 2 +- core/src/main/scala/spark/scheduler/JobResult.scala | 2 +- core/src/main/scala/spark/scheduler/JobWaiter.scala | 2 +- .../main/scala/spark/ui/jobs/JobProgressUI.scala | 21 ++++++++++++--------- 5 files changed, 18 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index bdd8792ce9..82d419453b 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -256,7 +256,7 @@ class DAGScheduler( eventQueue.put(toSubmit) waiter.awaitResult() match { case JobSucceeded => {} - case JobFailed(exception: Exception) => + case JobFailed(exception: Exception, _) => logInfo("Failed to run " + callSite) throw exception } @@ -324,7 +324,7 @@ class DAGScheduler( for (job <- activeJobs) { val error = new SparkException("Job cancelled because SparkContext was shut down") job.listener.jobFailed(error) - sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error)))) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error, None)))) } return true } @@ -671,7 +671,7 @@ class DAGScheduler( val job = resultStageToJob(resultStage) val error = new SparkException("Job failed: " + reason) job.listener.jobFailed(error) - sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error)))) + sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error, Some(failedStage))))) activeJobs -= job resultStageToJob -= resultStage } diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index 178bfaba3d..6a9d52f356 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -275,7 +275,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { var info = "JOB_ID=" + job.runId reason match { case JobSucceeded => info += " STATUS=SUCCESS" - case JobFailed(exception) => + case JobFailed(exception, _) => info += " STATUS=FAILED REASON=" exception.getMessage.split("\\s+").foreach(info += _ + "_") case _ => diff --git a/core/src/main/scala/spark/scheduler/JobResult.scala b/core/src/main/scala/spark/scheduler/JobResult.scala index 654131ee84..a0fdf391e6 100644 --- a/core/src/main/scala/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/spark/scheduler/JobResult.scala @@ -6,4 +6,4 @@ package spark.scheduler private[spark] sealed trait JobResult private[spark] case object JobSucceeded extends JobResult -private[spark] case class JobFailed(exception: Exception) extends JobResult +private[spark] case class JobFailed(exception: Exception, failedStage: Option[Stage]) extends JobResult diff --git a/core/src/main/scala/spark/scheduler/JobWaiter.scala b/core/src/main/scala/spark/scheduler/JobWaiter.scala index 3cc6a86345..6ff2e29434 100644 --- a/core/src/main/scala/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/spark/scheduler/JobWaiter.scala @@ -35,7 +35,7 @@ private[spark] class JobWaiter[T](totalTasks: Int, resultHandler: (Int, T) => Un throw new UnsupportedOperationException("jobFailed() called on a finished JobWaiter") } jobFinished = true - jobResult = JobFailed(exception) + jobResult = JobFailed(exception, None) this.notifyAll() } } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index dea4f0e3b0..70f8e431f2 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -70,25 +70,28 @@ private[spark] class JobProgressListener extends SparkListener { activeStages += stageSubmitted.stage override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val sid = taskEnd.event.task.stageId - taskEnd.event.reason match { + val sid = taskEnd.task.stageId + taskEnd.reason match { case Success => stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 case _ => stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 } val taskList = stageToTaskInfos.getOrElse(sid, ArrayBuffer[(TaskInfo, TaskMetrics)]()) - taskList += ((taskEnd.event.taskInfo, taskEnd.event.taskMetrics)) + taskList += ((taskEnd.taskInfo, taskEnd.taskMetrics)) stageToTaskInfos(sid) = taskList } - override def onJobEnd(jobEnd: SparkListenerEvents) { + override def onJobEnd(jobEnd: SparkListenerJobEnd) { jobEnd match { - case failed: SparkListenerJobFailed => - val stage = failed.failedStage - activeStages -= stage - failedStages += stage - trimIfNecessary(failedStages) + case end: SparkListenerJobEnd => + end.jobResult match { + case JobFailed(ex, Some(stage)) => + activeStages -= stage + failedStages += stage + trimIfNecessary(failedStages) + case _ => + } case _ => } } -- cgit v1.2.3 From 93e8ed85aa1fbcb6428934b30d01f2b4090538b9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Jun 2013 13:11:18 -0700 Subject: Work around for initalization issue --- core/src/main/scala/spark/SparkContext.scala | 5 +++-- core/src/main/scala/spark/ui/SparkUI.scala | 10 +++++++++- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 2 +- core/src/main/scala/spark/ui/jobs/JobProgressUI.scala | 9 +++++++-- core/src/main/scala/spark/ui/jobs/StagePage.scala | 2 +- 5 files changed, 21 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 901cda4174..1f420d73f7 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -102,6 +102,9 @@ class SparkContext( private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]] private[spark] val metadataCleaner = new MetadataCleaner("SparkContext", this.cleanup) + // Initalize the Spark UI + private[spark] val ui = new SparkUI(this) + ui.bind() // Add each JAR given through the constructor if (jars != null) { @@ -211,8 +214,6 @@ class SparkContext( @volatile private var dagScheduler = new DAGScheduler(taskScheduler) dagScheduler.start() - // Start the Spark UI - private[spark] val ui = new SparkUI(this) ui.start() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 66dcf07384..2d5a328015 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -26,7 +26,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { val jobs = new JobProgressUI(sc) val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ handlers - def start() { + def bind() { /** Start an HTTP server to run the Web interface */ try { val (server, usedPort) = JettyUtils.startJettyServer("0.0.0.0", port, allHandlers) @@ -38,6 +38,14 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { System.exit(1) } } + /** Initialize all components of the server */ + def start() { + // NOTE: This is decoupled from bind() because of the following dependency cycle: + // DAGScheduler() requires that the port of this server is known + // This server must register all handlers, including JobProgressUI, before binding + // JobProgressUI registers a listener with SparkContext, which requires sc to initialize + jobs.start() + } private[spark] def appUIAddress = "http://" + host + ":" + boundPort.getOrElse("-1") } diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 2e2dcdfbc6..134c93091d 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -15,7 +15,7 @@ import spark.storage.StorageLevel /** Page showing list of all ongoing and recently finished stages */ class IndexPage(parent: JobProgressUI) { - val listener = parent.listener + def listener = parent.listener val dateFmt = parent.dateFmt def render(request: HttpServletRequest): Seq[Node] = { diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 70f8e431f2..99f9f2d9f6 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -18,10 +18,15 @@ import spark.Success /** Web UI showing progress status of all jobs in the given SparkContext. */ private[spark] class JobProgressUI(val sc: SparkContext) { - val listener = new JobProgressListener + private var _listener: Option[JobProgressListener] = None + def listener = _listener.get + val dateFmt = new SimpleDateFormat("EEE, MMM d yyyy HH:mm:ss") - sc.addSparkListener(listener) + def start() { + _listener = Some(new JobProgressListener) + sc.addSparkListener(listener) + } private val indexPage = new IndexPage(this) private val stagePage = new StagePage(this) diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 42eb1f9eef..8a488498d9 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -13,7 +13,7 @@ import spark.executor.TaskMetrics /** Page showing statistics and task list for a given stage */ class StagePage(parent: JobProgressUI) { - val listener = parent.listener + def listener = parent.listener val dateFmt = parent.dateFmt def render(request: HttpServletRequest): Seq[Node] = { -- cgit v1.2.3 From b5e6e8bcc8ca05cd97f35a502a89c686aa4a5a12 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Jun 2013 14:13:24 -0700 Subject: Cleaning up some code for Job Progress --- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 4 +-- .../main/scala/spark/ui/jobs/JobProgressUI.scala | 9 ++++-- core/src/main/scala/spark/ui/jobs/StagePage.scala | 37 ++++++++++++++-------- 3 files changed, 31 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 134c93091d..c7ee9dc1c1 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -38,7 +38,7 @@ class IndexPage(parent: JobProgressUI) { def getElapsedTime(submitted: Option[Long], completed: Long): String = { submitted match { - case Some(t) => Duration(completed - t, "milliseconds").printHMS + case Some(t) => parent.formatDuration(completed - t) case _ => "Unknown" } } @@ -53,7 +53,7 @@ class IndexPage(parent: JobProgressUI) { case (true, true) => "Read/Write" case (true, false) => "Read" case (false, true) => "Write" - case _ => "None" + case _ => "" } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 99f9f2d9f6..027eadde3a 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -1,5 +1,7 @@ package spark.ui.jobs +import akka.util.Duration + import java.text.SimpleDateFormat import javax.servlet.http.HttpServletRequest @@ -20,16 +22,17 @@ import spark.Success private[spark] class JobProgressUI(val sc: SparkContext) { private var _listener: Option[JobProgressListener] = None def listener = _listener.get - val dateFmt = new SimpleDateFormat("EEE, MMM d yyyy HH:mm:ss") + private val indexPage = new IndexPage(this) + private val stagePage = new StagePage(this) + def start() { _listener = Some(new JobProgressListener) sc.addSparkListener(listener) } - private val indexPage = new IndexPage(this) - private val stagePage = new StagePage(this) + def formatDuration(ms: Long) = Duration(ms, "milliseconds").printHMS def getHandlers = Seq[(String, Handler)]( ("/stages/stage", (request: HttpServletRequest) => stagePage.render(request)), diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 8a488498d9..65dbd389b1 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -8,6 +8,7 @@ import scala.xml.Node import spark.ui.UIUtils._ import spark.util.Distribution +import spark.Utils import spark.scheduler.cluster.TaskInfo import spark.executor.TaskMetrics @@ -24,24 +25,30 @@ class StagePage(parent: JobProgressUI) { val shuffleWrite = listener.hasShuffleWrite(stageId) val taskHeaders: Seq[String] = - Seq("Task ID", "Service Time (ms)", "Locality Level", "Worker", "Launch Time") ++ - {if (shuffleRead) Seq("Shuffle Read (bytes)") else Nil} ++ - {if (shuffleWrite) Seq("Shuffle Write (bytes)") else Nil} + Seq("Task ID", "Service Time", "Locality Level", "Worker", "Launch Time") ++ + {if (shuffleRead) Seq("Shuffle Read") else Nil} ++ + {if (shuffleWrite) Seq("Shuffle Write") else Nil} val taskTable = listingTable(taskHeaders, taskRow, tasks) - // TODO(pwendell): Consider factoring this more nicely with the functions in SparkListener val serviceTimes = tasks.map{case (info, metrics) => metrics.executorRunTime.toDouble} - val serviceQuantiles = "Service Time" +: Distribution(serviceTimes).get.getQuantiles().map(_.toString) + val serviceQuantiles = "Service Time" +: Distribution(serviceTimes).get.getQuantiles().map( + ms => parent.formatDuration(ms.toLong)) - val shuffleReadSizes = tasks.map{ - case(info, metrics) => metrics.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble} - val shuffleReadQuantiles = "Shuffle Read" +: Distribution(shuffleReadSizes).get.getQuantiles().map(_.toString) + def getQuantileCols(data: Seq[Double]) = + Distribution(data).get.getQuantiles().map(d => Utils.memoryBytesToString(d.toLong)) - val shuffleWriteSizes = tasks.map{ - case(info, metrics) => metrics.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble} - val shuffleWriteQuantiles = "Shuffle Write" +: Distribution(shuffleWriteSizes).get.getQuantiles().map(_.toString) + val shuffleReadSizes = tasks.map { + case(info, metrics) => + metrics.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble + } + val shuffleReadQuantiles = "Shuffle Read (Remote)" +: getQuantileCols(shuffleReadSizes) + val shuffleWriteSizes = tasks.map { + case(info, metrics) => + metrics.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble + } + val shuffleWriteQuantiles = "Shuffle Write" +: getQuantileCols(shuffleWriteSizes) val listings: Seq[Seq[String]] = Seq(serviceQuantiles, if (shuffleRead) shuffleReadQuantiles else Nil, @@ -62,12 +69,14 @@ class StagePage(parent: JobProgressUI) { val (info, metrics) = taskData {info.taskId} - {metrics.executorRunTime} + {parent.formatDuration(metrics.executorRunTime)} {info.taskLocality} {info.hostPort} {dateFmt.format(new Date(info.launchTime))} - {metrics.shuffleReadMetrics.map{m => {m.remoteBytesRead}}.getOrElse("") } - {metrics.shuffleWriteMetrics.map{m => {m.shuffleBytesWritten}}.getOrElse("") } + {metrics.shuffleReadMetrics.map{m => + {Utils.memoryBytesToString(m.remoteBytesRead)}}.getOrElse("") } + {metrics.shuffleWriteMetrics.map{m => + {Utils.memoryBytesToString(m.shuffleBytesWritten)}}.getOrElse("") } } } -- cgit v1.2.3 From a4248138b4c4b374e61f474886463c58f5bb3915 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Jun 2013 14:22:28 -0700 Subject: Minor style cleanup --- core/src/main/scala/spark/ui/JettyUtils.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 96d773979f..b5270e6062 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -47,6 +47,7 @@ private[spark] object JettyUtils extends Logging { } } + /** Creates a handler that always redirects the user to a given path */ def createRedirectHandler(newPath: String): Handler = { new AbstractHandler { def handle(target: String, @@ -151,7 +152,7 @@ object UIUtils {

    - {title} + {title}

    -- cgit v1.2.3 From 4cda8f865a003ab354890c4915ea1b5a7674f5b0 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 24 Jun 2013 15:14:48 -0700 Subject: Add simple usage to start-slave script --- bin/start-slave.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/bin/start-slave.sh b/bin/start-slave.sh index 26b5b9d462..1082c09eb1 100755 --- a/bin/start-slave.sh +++ b/bin/start-slave.sh @@ -1,4 +1,7 @@ #!/usr/bin/env bash +# +# Usage: start-slave.sh +# where is like "spark://localhost:7077" bin=`dirname "$0"` bin=`cd "$bin"; pwd` -- cgit v1.2.3 From 0bcaf036050c3d2b4389339927239e0e35bf02ff Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 24 Jun 2013 15:39:52 -0700 Subject: Split out source distro CLASSPATH logic to a separate script --- run | 123 ++++++++------------------------------------------- set-dev-classpath.sh | 112 ++++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 131 insertions(+), 104 deletions(-) create mode 100644 set-dev-classpath.sh diff --git a/run b/run index c0065c53f1..30a2885a4d 100755 --- a/run +++ b/run @@ -1,7 +1,5 @@ #!/bin/bash -SCALA_VERSION=2.9.3 - # Figure out where the Scala framework is installed FWDIR="$(cd `dirname $0`; pwd)" @@ -46,36 +44,6 @@ case "$1" in ;; esac -if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then - if [ "$SCALA_HOME" ]; then - RUNNER="${SCALA_HOME}/bin/scala" - else - if [ `command -v scala` ]; then - RUNNER="scala" - else - echo "SCALA_HOME is not set and scala is not in PATH" >&2 - exit 1 - fi - fi -else - if [ `command -v java` ]; then - RUNNER="java" - else - if [ -z "$JAVA_HOME" ]; then - echo "JAVA_HOME is not set" >&2 - exit 1 - fi - RUNNER="${JAVA_HOME}/bin/java" - fi - if [ -z "$SCALA_LIBRARY_PATH" ]; then - if [ -z "$SCALA_HOME" ]; then - echo "SCALA_HOME is not set" >&2 - exit 1 - fi - SCALA_LIBRARY_PATH="$SCALA_HOME/lib" - fi -fi - # Figure out how much memory to use per executor and set it as an environment # variable so that our process sees it and can report it to Mesos if [ -z "$SPARK_MEM" ] ; then @@ -93,64 +61,28 @@ if [ -e $FWDIR/conf/java-opts ] ; then fi export JAVA_OPTS -CORE_DIR="$FWDIR/core" -REPL_DIR="$FWDIR/repl" -REPL_BIN_DIR="$FWDIR/repl-bin" -EXAMPLES_DIR="$FWDIR/examples" -BAGEL_DIR="$FWDIR/bagel" -STREAMING_DIR="$FWDIR/streaming" -PYSPARK_DIR="$FWDIR/python" - -# Exit if the user hasn't compiled Spark -if [ ! -e "$CORE_DIR/target" ]; then - echo "Failed to find Spark classes in $CORE_DIR/target" >&2 - echo "You need to compile Spark before running this program" >&2 - exit 1 -fi +# Check if this is a binary distribution or source distribution +# and build up the classpath appropriately +if [ -f "$FWDIR/RELEASE" ]; then + echo "This is a binary distribution" -if [[ "$@" = *repl* && ! -e "$REPL_DIR/target" ]]; then - echo "Failed to find Spark classes in $REPL_DIR/target" >&2 - echo "You need to compile Spark repl module before running this program" >&2 - exit 1 -fi + if [ `command -v java` ]; then + RUNNER="java" + else + if [ -z "$JAVA_HOME" ]; then + echo "JAVA_HOME is not set" >&2 + exit 1 + fi + RUNNER="${JAVA_HOME}/bin/java" + fi -# Build up classpath -CLASSPATH="$SPARK_CLASSPATH" -CLASSPATH="$CLASSPATH:$FWDIR/conf" -CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/classes" -if [ -n "$SPARK_TESTING" ] ; then - CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes" -fi -CLASSPATH="$CLASSPATH:$CORE_DIR/src/main/resources" -CLASSPATH="$CLASSPATH:$REPL_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH="$CLASSPATH:$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH="$CLASSPATH:$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar -if [ -e "$FWDIR/lib_managed" ]; then - CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/jars/*" - CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*" -fi -CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*" -if [ -e $REPL_BIN_DIR/target ]; then - for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do - CLASSPATH="$CLASSPATH:$jar" - done -fi -CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" -for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do - CLASSPATH="$CLASSPATH:$jar" -done + CLASSPATH="$SPARK_CLASSPATH:$FWDIR/jars/*" -# Figure out the JAR file that our examples were packaged into. This includes a bit of a hack -# to avoid the -sources and -doc packages that are built by publish-local. -if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then - # Use the JAR from the SBT build - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` -fi -if [ -e "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar ]; then - # Use the JAR from the Maven build - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar` + # The JVM doesn't read JAVA_OPTS by default so we need to pass it in + EXTRA_ARGS="$JAVA_OPTS" +else + echo "This is a source distribution" + . "$FWDIR/set-dev-classpath.sh" fi # Add hadoop conf dir - else FileSystem.*, etc fail ! @@ -163,22 +95,5 @@ if [ "x" != "x$YARN_CONF_DIR" ]; then CLASSPATH="$CLASSPATH:$YARN_CONF_DIR" fi - -# Figure out whether to run our class with java or with the scala launcher. -# In most cases, we'd prefer to execute our process with java because scala -# creates a shell script as the parent of its Java process, which makes it -# hard to kill the child with stuff like Process.destroy(). However, for -# the Spark shell, the wrapper is necessary to properly reset the terminal -# when we exit, so we allow it to set a variable to launch with scala. -if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then - EXTRA_ARGS="" # Java options will be passed to scala as JAVA_OPTS -else - CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-library.jar" - CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-compiler.jar" - CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/jline.jar" - # The JVM doesn't read JAVA_OPTS by default so we need to pass it in - EXTRA_ARGS="$JAVA_OPTS" -fi - export CLASSPATH # Needed for spark-shell exec "$RUNNER" -cp "$CLASSPATH" $EXTRA_ARGS "$@" diff --git a/set-dev-classpath.sh b/set-dev-classpath.sh new file mode 100644 index 0000000000..4d09bd4416 --- /dev/null +++ b/set-dev-classpath.sh @@ -0,0 +1,112 @@ +# A BASH script to set the classpath for running Spark out of the developer/github tree + +SCALA_VERSION=2.9.3 + +# Figure out where the Scala framework is installed +FWDIR="$(cd `dirname $0`; pwd)" + +if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then + if [ "$SCALA_HOME" ]; then + RUNNER="${SCALA_HOME}/bin/scala" + else + if [ `command -v scala` ]; then + RUNNER="scala" + else + echo "SCALA_HOME is not set and scala is not in PATH" >&2 + exit 1 + fi + fi +else + if [ `command -v java` ]; then + RUNNER="java" + else + if [ -z "$JAVA_HOME" ]; then + echo "JAVA_HOME is not set" >&2 + exit 1 + fi + RUNNER="${JAVA_HOME}/bin/java" + fi + if [ -z "$SCALA_LIBRARY_PATH" ]; then + if [ -z "$SCALA_HOME" ]; then + echo "SCALA_HOME is not set" >&2 + exit 1 + fi + SCALA_LIBRARY_PATH="$SCALA_HOME/lib" + fi +fi + +CORE_DIR="$FWDIR/core" +REPL_DIR="$FWDIR/repl" +REPL_BIN_DIR="$FWDIR/repl-bin" +EXAMPLES_DIR="$FWDIR/examples" +BAGEL_DIR="$FWDIR/bagel" +STREAMING_DIR="$FWDIR/streaming" +PYSPARK_DIR="$FWDIR/python" + +# Exit if the user hasn't compiled Spark +if [ ! -e "$CORE_DIR/target" ]; then + echo "Failed to find Spark classes in $CORE_DIR/target" >&2 + echo "You need to compile Spark before running this program" >&2 + exit 1 +fi + +if [[ "$@" = *repl* && ! -e "$REPL_DIR/target" ]]; then + echo "Failed to find Spark classes in $REPL_DIR/target" >&2 + echo "You need to compile Spark repl module before running this program" >&2 + exit 1 +fi + +# Build up classpath +CLASSPATH="$SPARK_CLASSPATH" +CLASSPATH="$CLASSPATH:$FWDIR/conf" +CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/classes" +if [ -n "$SPARK_TESTING" ] ; then + CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes" +fi +CLASSPATH="$CLASSPATH:$CORE_DIR/src/main/resources" +CLASSPATH="$CLASSPATH:$REPL_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar +if [ -e "$FWDIR/lib_managed" ]; then + CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/jars/*" + CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*" +fi +CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*" +if [ -e $REPL_BIN_DIR/target ]; then + for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do + CLASSPATH="$CLASSPATH:$jar" + done +fi +CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" +for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do + CLASSPATH="$CLASSPATH:$jar" +done + +# Figure out the JAR file that our examples were packaged into. This includes a bit of a hack +# to avoid the -sources and -doc packages that are built by publish-local. +if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then + # Use the JAR from the SBT build + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` +fi +if [ -e "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar ]; then + # Use the JAR from the Maven build + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar` +fi + +# Figure out whether to run our class with java or with the scala launcher. +# In most cases, we'd prefer to execute our process with java because scala +# creates a shell script as the parent of its Java process, which makes it +# hard to kill the child with stuff like Process.destroy(). However, for +# the Spark shell, the wrapper is necessary to properly reset the terminal +# when we exit, so we allow it to set a variable to launch with scala. +if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then + EXTRA_ARGS="" # Java options will be passed to scala as JAVA_OPTS +else + CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-library.jar" + CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-compiler.jar" + CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/jline.jar" + # The JVM doesn't read JAVA_OPTS by default so we need to pass it in + EXTRA_ARGS="$JAVA_OPTS" +fi -- cgit v1.2.3 From 42157027f247035e9ae41efe899e27c0942f5cd8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Jun 2013 16:25:05 -0700 Subject: A few bug fixes and a unit test --- core/src/main/scala/spark/SparkContext.scala | 1 + core/src/main/scala/spark/deploy/master/Master.scala | 9 +++++---- .../main/scala/spark/deploy/master/ui/MasterWebUI.scala | 10 ++++++++-- core/src/main/scala/spark/deploy/worker/Worker.scala | 9 +++------ .../main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 10 ++++++++-- core/src/main/scala/spark/ui/JettyUtils.scala | 7 +++++-- core/src/main/scala/spark/ui/SparkUI.scala | 13 ++++++++++--- core/src/test/resources/log4j.properties | 1 + core/src/test/scala/spark/DistributedSuite.scala | 2 ++ core/src/test/scala/spark/ui/UISuite.scala | 15 +++++++++++++++ 10 files changed, 58 insertions(+), 19 deletions(-) create mode 100644 core/src/test/scala/spark/ui/UISuite.scala diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 1f420d73f7..500d25efdd 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -578,6 +578,7 @@ class SparkContext( /** Shut down the SparkContext. */ def stop() { + ui.stop() // Do this only if not stopped already - best case effort. // prevent NPE if stopped more than once. val dagSchedulerCopy = dagScheduler diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 3eca522c7e..2a49dfb486 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -36,6 +36,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act var firstApp: Option[ApplicationInfo] = None + val webUi = new MasterWebUI(self) + Utils.checkHost(host, "Expected hostname") val masterPublicAddress = { @@ -52,13 +54,12 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act logInfo("Starting Spark master at spark://" + host + ":" + port) // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) - startWebUi() + webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) } - def startWebUi() { - val webUi = new MasterWebUI(self) - webUi.start() + override def postStop() { + webUi.stop() } override def receive = { 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 5fd17f10c6..065b62a4b2 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -5,7 +5,7 @@ import akka.util.Duration import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.Handler +import org.eclipse.jetty.server.{Handler, Server} import spark.{Logging, Utils} import spark.ui.JettyUtils @@ -21,13 +21,15 @@ class MasterWebUI(val master: ActorRef) extends Logging { val host = Utils.localHostName() val port = Option(System.getProperty("master.ui.port")) .getOrElse(MasterWebUI.DEFAULT_PORT).toInt + var server: Option[Server] = None val applicationPage = new ApplicationPage(this) val indexPage = new IndexPage(this) def start() { try { - val (server, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + val (srv, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + server = Some(srv) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => @@ -42,6 +44,10 @@ class MasterWebUI(val master: ActorRef) extends Logging { ("/app", (request: HttpServletRequest) => applicationPage.render(request)), ("*", (request: HttpServletRequest) => indexPage.render(request)) ) + + def stop() { + server.foreach(_.stop()) + } } object MasterWebUI { diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 690bdfe128..28553b6c02 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -45,6 +45,7 @@ private[spark] class Worker( val envVar = System.getenv("SPARK_PUBLIC_DNS") if (envVar != null) envVar else host } + val webUi = new WorkerWebUI(self, workDir) var coresUsed = 0 var memoryUsed = 0 @@ -77,7 +78,7 @@ private[spark] class Worker( logInfo("Spark home: " + sparkHome) createWorkDir() connectToMaster() - startWebUi() + webUi.start() } def connectToMaster() { @@ -88,11 +89,6 @@ private[spark] class Worker( context.watch(master) // Doesn't work with remote actors, but useful for testing } - def startWebUi() { - val webUi = new WorkerWebUI(self, workDir) - webUi.start() - } - override def receive = { case RegisteredWorker(url) => masterWebUiUrl = url @@ -163,6 +159,7 @@ private[spark] class Worker( override def postStop() { executors.values.foreach(_.kill()) + webUi.stop() } } diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index abfc847527..ee3889192d 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -7,7 +7,7 @@ import java.io.File import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.Handler +import org.eclipse.jetty.server.{Handler, Server} import scala.io.Source @@ -25,6 +25,7 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File) extends Logging { val host = Utils.localHostName() val port = Option(System.getProperty("wroker.ui.port")) .getOrElse(WorkerWebUI.DEFAULT_PORT).toInt + var server: Option[Server] = None val indexPage = new IndexPage(this) @@ -37,7 +38,8 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File) extends Logging { def start() { try { - val (server, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + val (srv, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + server = Some(srv) logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => @@ -56,6 +58,10 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File) extends Logging { source.close() lines } + + def stop() { + server.foreach(_.stop()) + } } object WorkerWebUI { diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index b5270e6062..85d6a7e867 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -12,7 +12,8 @@ import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHa import scala.util.{Try, Success, Failure} import scala.xml.Node -import spark.{Utils, SparkContext, Logging} +import spark.{SparkContext, Logging} +import org.eclipse.jetty.util.log.Log /** Utilities for launching a web server using Jetty's HTTP Server class */ private[spark] object JettyUtils extends Logging { @@ -91,12 +92,14 @@ private[spark] object JettyUtils extends Logging { @tailrec def connect(currentPort: Int): (Server, Int) = { - val server = new Server(port) + val server = new Server(currentPort) server.setHandler(handlerList) Try { server.start() } match { case s: Success[_] => (server, currentPort) case f: Failure[_] => + server.stop() logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) + logInfo("Error was: " + f.toString) connect((currentPort + 1) % 65536) } } diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 2d5a328015..487f005615 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -2,7 +2,7 @@ package spark.ui import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.Handler +import org.eclipse.jetty.server.{Handler, Server} import spark.{Logging, SparkContext, Utils} import spark.ui.storage.BlockManagerUI @@ -17,6 +17,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { val host = Utils.localHostName() val port = Option(System.getProperty("spark.ui.port")).getOrElse(SparkUI.DEFAULT_PORT).toInt var boundPort: Option[Int] = None + var server: Option[Server] = None val handlers = Seq[(String, Handler)]( ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), @@ -26,11 +27,12 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { val jobs = new JobProgressUI(sc) val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ handlers + /** Bind the HTTP server which backs this web interface */ def bind() { - /** Start an HTTP server to run the Web interface */ try { - val (server, usedPort) = JettyUtils.startJettyServer("0.0.0.0", port, allHandlers) + val (srv, usedPort) = JettyUtils.startJettyServer("0.0.0.0", port, allHandlers) logInfo("Started Spark Web UI at http://%s:%d".format(host, usedPort)) + server = Some(srv) boundPort = Some(usedPort) } catch { case e: Exception => @@ -38,6 +40,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { System.exit(1) } } + /** Initialize all components of the server */ def start() { // NOTE: This is decoupled from bind() because of the following dependency cycle: @@ -47,6 +50,10 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { jobs.start() } + def stop() { + server.foreach(_.stop()) + } + private[spark] def appUIAddress = "http://" + host + ":" + boundPort.getOrElse("-1") } diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 6ec89c0184..d05cf3dec1 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -8,3 +8,4 @@ log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN +org.eclipse.jetty.LEVEL=WARN diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index 0866fb47b3..0024ede828 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -10,6 +10,7 @@ import org.scalatest.time.{Span, Millis} import org.scalacheck.Arbitrary._ import org.scalacheck.Gen import org.scalacheck.Prop._ +import org.eclipse.jetty.server.{Server, Request, Handler} import com.google.common.io.Files @@ -17,6 +18,7 @@ import scala.collection.mutable.ArrayBuffer import SparkContext._ import storage.{GetBlock, BlockManagerWorker, StorageLevel} +import ui.JettyUtils class NotSerializableClass diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala new file mode 100644 index 0000000000..6766b158f6 --- /dev/null +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -0,0 +1,15 @@ +package spark.ui + +import org.scalatest.FunSuite +import org.eclipse.jetty.server.Server + +class UISuite extends FunSuite { + test("jetty port increases under contention") { + val startPort = 33333 + val server = new Server(startPort) + server.start() + val (jettyServer, boundPort) = JettyUtils.startJettyServer("localhost", startPort, Seq()) + assert(boundPort === startPort + 1) + } + +} -- cgit v1.2.3 From f7389330c35a6372c4c7b455b96b8498b9a9da27 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Jun 2013 16:51:52 -0700 Subject: Allowing for requested port on construction --- core/src/main/scala/spark/deploy/master/Master.scala | 2 +- .../main/scala/spark/deploy/master/ui/MasterWebUI.scala | 11 +++++++---- core/src/main/scala/spark/deploy/worker/Worker.scala | 6 +++--- .../main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 14 +++++++++----- core/src/main/scala/spark/ui/JettyUtils.scala | 2 +- 5 files changed, 21 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 2a49dfb486..127be9e7d4 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -71,7 +71,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } else { addWorker(id, host, workerPort, cores, memory, worker_webUiPort, publicAddress) context.watch(sender) // This doesn't work with remote actors but helps for testing - sender ! RegisteredWorker("http://" + masterPublicAddress + ":" + webUiPort) + sender ! RegisteredWorker("http://" + masterPublicAddress + ":" + webUi.boundPort) schedule() } } 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 065b62a4b2..50e93cd00f 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -15,21 +15,24 @@ import spark.ui.JettyUtils._ * Web UI server for the standalone master. */ private[spark] -class MasterWebUI(val master: ActorRef) extends Logging { +class MasterWebUI(val master: ActorRef, requestedPort: Option[Int]) extends Logging { implicit val timeout = Duration.create( System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") val host = Utils.localHostName() - val port = Option(System.getProperty("master.ui.port")) - .getOrElse(MasterWebUI.DEFAULT_PORT).toInt + val port = requestedPort.getOrElse( + System.getProperty("master.ui.port", MasterWebUI.DEFAULT_PORT).toInt) + var server: Option[Server] = None + var boundPort: Option[Int] = None val applicationPage = new ApplicationPage(this) val indexPage = new IndexPage(this) def start() { try { - val (srv, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + val (srv, bPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) server = Some(srv) + boundPort = Some(bPort) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 28553b6c02..b6a26245fc 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -45,7 +45,7 @@ private[spark] class Worker( val envVar = System.getenv("SPARK_PUBLIC_DNS") if (envVar != null) envVar else host } - val webUi = new WorkerWebUI(self, workDir) + val webUi = new WorkerWebUI(self, workDir, Some(webUiPort)) var coresUsed = 0 var memoryUsed = 0 @@ -77,14 +77,14 @@ private[spark] class Worker( sparkHome = new File(Option(System.getenv("SPARK_HOME")).getOrElse(".")) logInfo("Spark home: " + sparkHome) createWorkDir() - connectToMaster() webUi.start() + connectToMaster() } def connectToMaster() { logInfo("Connecting to master " + masterUrl) master = context.actorFor(Master.toAkkaUrl(masterUrl)) - master ! RegisterWorker(workerId, host, port, cores, memory, webUiPort, publicAddress) + master ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort.get, publicAddress) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) context.watch(master) // Doesn't work with remote actors, but useful for testing } diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index ee3889192d..2615458dee 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -19,13 +19,16 @@ import spark.ui.JettyUtils._ * Web UI server for the standalone worker. */ private[spark] -class WorkerWebUI(val worker: ActorRef, val workDir: File) extends Logging { +class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option[Int] = None) + extends Logging { implicit val timeout = Timeout( Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")) val host = Utils.localHostName() - val port = Option(System.getProperty("wroker.ui.port")) - .getOrElse(WorkerWebUI.DEFAULT_PORT).toInt + val port = requestedPort.getOrElse( + System.getProperty("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) + var server: Option[Server] = None + var boundPort: Option[Int] = None val indexPage = new IndexPage(this) @@ -38,9 +41,10 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File) extends Logging { def start() { try { - val (srv, boundPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) + val (srv, bPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) server = Some(srv) - logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) + boundPort = Some(bPort) + logInfo("Started Worker web UI at http://%s:%d".format(host, bPort)) } catch { case e: Exception => logError("Failed to create Worker JettyUtils", e) diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 85d6a7e867..1f665cbb42 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -95,7 +95,7 @@ private[spark] object JettyUtils extends Logging { val server = new Server(currentPort) server.setHandler(handlerList) Try { server.start() } match { - case s: Success[_] => (server, currentPort) + case s: Success[_] => (server, server.getConnectors.head.getLocalPort) case f: Failure[_] => server.stop() logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) -- cgit v1.2.3 From 81df20e5b44407c8c321471be2faee4bf6853fd6 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 24 Jun 2013 17:05:37 -0700 Subject: Script to create binary distribution for Spark --- make-distribution.sh | 30 ++++++++++++++++++++++++++++++ 1 file changed, 30 insertions(+) create mode 100755 make-distribution.sh diff --git a/make-distribution.sh b/make-distribution.sh new file mode 100755 index 0000000000..34f6d4a36b --- /dev/null +++ b/make-distribution.sh @@ -0,0 +1,30 @@ +#!/bin/bash +# +# Script to create a binary distribution for easy deploys of Spark. +# The distribution directory defaults to dist/ but can be overridden below. +# The distribution contains fat (assembly) jars that include the Scala library, +# so it is completely self contained. + +# Figure out where the Spark framework is installed +FWDIR="$(cd `dirname $0`; pwd)" +DISTDIR="$FWDIR/dist" + +# Get version from SBT +VERSION=$($FWDIR/sbt/sbt "show version" | tail -1 | cut -f 2) +echo "Making distribution for Spark $VERSION in $DISTDIR..." + +# Build fat JAR +$FWDIR/sbt/sbt "repl/assembly" + +# Make directories +rm -rf "$DISTDIR" +mkdir -p "$DISTDIR/jars" +echo "$VERSION" >$DISTDIR/RELEASE + +# Copy jars +cp $FWDIR/repl/target/*.jar "$DISTDIR/jars/" + +# Copy other things +cp -r "$FWDIR/bin" "$DISTDIR" +cp -r "$FWDIR/conf" "$DISTDIR" +cp "$FWDIR/run" "$FWDIR/spark-shell" "$DISTDIR" \ No newline at end of file -- cgit v1.2.3 From d66bd6f8851e87c8f35cca86ee45e26a8f547040 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Jun 2013 17:08:58 -0700 Subject: Adding another unit test to Web UI suite --- .../scala/spark/deploy/master/ui/MasterWebUI.scala | 4 ++-- core/src/main/scala/spark/ui/JettyUtils.scala | 2 +- core/src/test/scala/spark/ui/UISuite.scala | 18 ++++++++++++++++-- 3 files changed, 19 insertions(+), 5 deletions(-) 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 50e93cd00f..dfefd6571b 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -15,7 +15,7 @@ import spark.ui.JettyUtils._ * Web UI server for the standalone master. */ private[spark] -class MasterWebUI(val master: ActorRef, requestedPort: Option[Int]) extends Logging { +class MasterWebUI(val master: ActorRef, requestedPort: Option[Int] = None) extends Logging { implicit val timeout = Duration.create( System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") val host = Utils.localHostName() @@ -33,7 +33,7 @@ class MasterWebUI(val master: ActorRef, requestedPort: Option[Int]) extends Logg val (srv, bPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers) server = Some(srv) boundPort = Some(bPort) - logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) + logInfo("Started Master web UI at http://%s:%d".format(host, boundPort.get)) } catch { case e: Exception => logError("Failed to create Master JettyUtils", e) diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 1f665cbb42..45c0805b22 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -62,7 +62,7 @@ private[spark] object JettyUtils extends Logging { } } - /** Creates a handler for serving files from a static directory. */ + /** Creates a handler for serving files from a static directory */ def createStaticHandler(resourceBase: String): ResourceHandler = { val staticHandler = new ResourceHandler Option(getClass.getClassLoader.getResource(resourceBase)) match { diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index 6766b158f6..7df7356a6e 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -2,14 +2,28 @@ package spark.ui import org.scalatest.FunSuite import org.eclipse.jetty.server.Server +import util.{Try, Success, Failure} +import java.net.ServerSocket class UISuite extends FunSuite { test("jetty port increases under contention") { val startPort = 33333 val server = new Server(startPort) server.start() - val (jettyServer, boundPort) = JettyUtils.startJettyServer("localhost", startPort, Seq()) - assert(boundPort === startPort + 1) + val (jettyServer1, boundPort1) = JettyUtils.startJettyServer("localhost", startPort, Seq()) + val (jettyServer2, boundPort2) = JettyUtils.startJettyServer("localhost", startPort, Seq()) + + assert(boundPort1 === startPort + 1) + assert(boundPort2 === startPort + 2) } + test("jetty binds to port 0 correctly") { + val (jettyServer, boundPort) = JettyUtils.startJettyServer("localhost", 0, Seq()) + assert(jettyServer.getState === "STARTED") + assert(boundPort != 0) + Try {new ServerSocket(boundPort)} match { + case Success(s) => fail("Port %s doesn't seem used by jetty server".format(boundPort)) + case Failure(e) => + } + } } -- cgit v1.2.3 From 48c7e373c62b2e8cf48157ceb0d92c38c3a40652 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 24 Jun 2013 23:11:04 -0700 Subject: Minor formatting fixes --- .../src/main/scala/spark/streaming/DStream.scala | 9 +++++-- .../scala/spark/streaming/StreamingContext.scala | 29 +++++++++++++--------- .../streaming/api/java/JavaStreamingContext.scala | 15 +++++++---- 3 files changed, 34 insertions(+), 19 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index e125310861..9be7926a4a 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -441,7 +441,12 @@ abstract class DStream[T: ClassManifest] ( * Return a new DStream in which each RDD has a single element generated by counting each RDD * of this DStream. */ - def count(): DStream[Long] = this.map(_ => (null, 1L)).transform(_.union(context.sparkContext.makeRDD(Seq((null, 0L)), 1))).reduceByKey(_ + _).map(_._2) + def count(): DStream[Long] = { + this.map(_ => (null, 1L)) + .transform(_.union(context.sparkContext.makeRDD(Seq((null, 0L)), 1))) + .reduceByKey(_ + _) + .map(_._2) + } /** * Return a new DStream in which each RDD contains the counts of each distinct value in @@ -457,7 +462,7 @@ abstract class DStream[T: ClassManifest] ( * this DStream will be registered as an output stream and therefore materialized. */ def foreach(foreachFunc: RDD[T] => Unit) { - foreach((r: RDD[T], t: Time) => foreachFunc(r)) + this.foreach((r: RDD[T], t: Time) => foreachFunc(r)) } /** diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 2c6326943d..03d2907323 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -171,10 +171,11 @@ class StreamingContext private ( * should be same. */ def actorStream[T: ClassManifest]( - props: Props, - name: String, - storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2, - supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy): DStream[T] = { + props: Props, + name: String, + storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2, + supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy + ): DStream[T] = { networkStream(new ActorReceiver[T](props, name, storageLevel, supervisorStrategy)) } @@ -182,9 +183,10 @@ class StreamingContext private ( * Create an input stream that receives messages pushed by a zeromq publisher. * @param publisherUrl Url of remote zeromq publisher * @param subscribe topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence - * of byte thus it needs the converter(which might be deserializer of bytes) - * to translate from sequence of sequence of bytes, where sequence refer to a frame + * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic + * and each frame has sequence of byte thus it needs the converter + * (which might be deserializer of bytes) to translate from sequence + * of sequence of bytes, where sequence refer to a frame * and sub sequence refer to its payload. * @param storageLevel RDD storage level. Defaults to memory-only. */ @@ -204,7 +206,7 @@ class StreamingContext private ( * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. + * in its own thread. * @param storageLevel Storage level to use for storing the received objects * (default: StorageLevel.MEMORY_AND_DISK_SER_2) */ @@ -214,15 +216,17 @@ class StreamingContext private ( topics: Map[String, Int], storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2 ): DStream[String] = { - val kafkaParams = Map[String, String]("zk.connect" -> zkQuorum, "groupid" -> groupId, "zk.connectiontimeout.ms" -> "10000"); + val kafkaParams = Map[String, String]( + "zk.connect" -> zkQuorum, "groupid" -> groupId, "zk.connectiontimeout.ms" -> "10000") kafkaStream[String, kafka.serializer.StringDecoder](kafkaParams, topics, storageLevel) } /** * Create an input stream that pulls messages from a Kafka Broker. - * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html + * @param kafkaParams Map of kafka configuration paramaters. + * See: http://kafka.apache.org/configuration.html * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. + * in its own thread. * @param storageLevel Storage level to use for storing the received objects */ def kafkaStream[T: ClassManifest, D <: kafka.serializer.Decoder[_]: Manifest]( @@ -395,7 +399,8 @@ class StreamingContext private ( * it will process either one or all of the RDDs returned by the queue. * @param queue Queue of RDDs * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval - * @param defaultRDD Default RDD is returned by the DStream when the queue is empty. Set as null if no RDD should be returned when empty + * @param defaultRDD Default RDD is returned by the DStream when the queue is empty. + * Set as null if no RDD should be returned when empty * @tparam T Type of objects in the RDD */ def queueStream[T: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index b35d9032f1..fd5e06b50f 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -75,7 +75,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { : JavaDStream[String] = { implicit val cmt: ClassManifest[String] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[String]] - ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), StorageLevel.MEMORY_ONLY_SER_2) + ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), + StorageLevel.MEMORY_ONLY_SER_2) } /** @@ -83,8 +84,9 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..). * @param groupId The group id for this consumer. * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread. * @param storageLevel RDD storage level. Defaults to memory-only - * in its own thread. + * */ def kafkaStream( zkQuorum: String, @@ -94,14 +96,16 @@ class JavaStreamingContext(val ssc: StreamingContext) { : JavaDStream[String] = { implicit val cmt: ClassManifest[String] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[String]] - ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) + ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), + storageLevel) } /** * Create an input stream that pulls messages form a Kafka Broker. * @param typeClass Type of RDD * @param decoderClass Type of kafka decoder - * @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html + * @param kafkaParams Map of kafka configuration paramaters. + * See: http://kafka.apache.org/configuration.html * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. * @param storageLevel RDD storage level. Defaults to memory-only @@ -113,7 +117,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { topics: JMap[String, JInt], storageLevel: StorageLevel) : JavaDStream[T] = { - implicit val cmt: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] + implicit val cmt: ClassManifest[T] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] implicit val cmd: Manifest[D] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[D]] ssc.kafkaStream[T, D]( kafkaParams.toMap, -- cgit v1.2.3 From cee05a174897af294f52cbda551da09cb869557e Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 24 Jun 2013 23:55:09 -0700 Subject: Copy restore-TTY functions from Scala script so binary distros don't need 'scala' installed --- spark-shell | 39 ++++++++++++++++++++++++++++++++++++--- 1 file changed, 36 insertions(+), 3 deletions(-) diff --git a/spark-shell b/spark-shell index 574ae2104d..afbb7a9a8e 100755 --- a/spark-shell +++ b/spark-shell @@ -1,4 +1,37 @@ -#!/bin/sh +#!/bin/bash --posix FWDIR="`dirname $0`" -export SPARK_LAUNCH_WITH_SCALA=1 -exec $FWDIR/run spark.repl.Main "$@" + +# Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in +# binary distribution of Spark where Scala is not installed +exit_status=127 +saved_stty="" + +# restore stty settings (echo in particular) +function restoreSttySettings() { + stty $saved_stty + saved_stty="" +} + +function onExit() { + if [[ "$saved_stty" != "" ]]; then + restoreSttySettings + fi + exit $exit_status +} + +# to reenable echo if we are interrupted before completing. +trap onExit INT + +# save terminal settings +saved_stty=$(stty -g 2>/dev/null) +# clear on error so we don't later try to restore them +if [[ ! $? ]]; then + saved_stty="" +fi + +$FWDIR/run spark.repl.Main "$@" + +# record the exit status lest it be overwritten: +# then reenable echo and propagate the code. +exit_status=$? +onExit -- cgit v1.2.3 From c3d11d0d57739ec3f08783f71bf4d0efdec3d627 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 25 Jun 2013 00:02:00 -0700 Subject: Get rid of debugging statements --- run | 3 --- 1 file changed, 3 deletions(-) diff --git a/run b/run index 30a2885a4d..646d12c1eb 100755 --- a/run +++ b/run @@ -64,8 +64,6 @@ export JAVA_OPTS # Check if this is a binary distribution or source distribution # and build up the classpath appropriately if [ -f "$FWDIR/RELEASE" ]; then - echo "This is a binary distribution" - if [ `command -v java` ]; then RUNNER="java" else @@ -81,7 +79,6 @@ if [ -f "$FWDIR/RELEASE" ]; then # The JVM doesn't read JAVA_OPTS by default so we need to pass it in EXTRA_ARGS="$JAVA_OPTS" else - echo "This is a source distribution" . "$FWDIR/set-dev-classpath.sh" fi -- cgit v1.2.3 From 243d71cb066d888f3d1a9c613859522438b69ba8 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 25 Jun 2013 00:15:58 -0700 Subject: Add deploy/testing procedure --- make-distribution.sh | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/make-distribution.sh b/make-distribution.sh index 34f6d4a36b..855475864d 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -4,6 +4,14 @@ # The distribution directory defaults to dist/ but can be overridden below. # The distribution contains fat (assembly) jars that include the Scala library, # so it is completely self contained. +# It does not contain source or *.class files. +# +# Recommended deploy/testing procedure (standalone mode): +# 1) Rsync / deploy the dist/ dir to one host +# 2) cd to deploy dir; ./bin/start-master.sh +# 3) Verify master is up by visiting web page, ie http://master-ip:8080. Note the spark:// URL. +# 4) ./bin/start-slave.sh 1 <> +# 5) MASTER="spark://my-master-ip:7077" ./spark-shell # Figure out where the Spark framework is installed FWDIR="$(cd `dirname $0`; pwd)" -- cgit v1.2.3 From 982a686ff858399067de960fa62cc80f60c6fa32 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 25 Jun 2013 11:14:21 -0700 Subject: Add -m and -c options to spark-shell for convenience --- spark-shell | 29 ++++++++++++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/spark-shell b/spark-shell index afbb7a9a8e..ea67a3e6b8 100755 --- a/spark-shell +++ b/spark-shell @@ -1,6 +1,33 @@ #!/bin/bash --posix +# +# Shell script for starting the Spark Shell REPL +# Options: +# -m Set MASTER to spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT +# -c Set the number of cores for REPL to use +# FWDIR="`dirname $0`" +for o in "$@"; do + if [ "$1" = "-m" -o "$1" = "--master" ]; then + shift + if [ -e "$FWDIR/conf/spark-env.sh" ]; then + . "$FWDIR/conf/spark-env.sh" + fi + if [ -z "$MASTER" ]; then + MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" + fi + export MASTER + fi + + if [ "$1" = "-c" -o "$1" = "--cores" ]; then + shift + if [ -n "$1" ]; then + OPTIONS="-Dspark.cores.max=$1" + shift + fi + fi +done + # Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in # binary distribution of Spark where Scala is not installed exit_status=127 @@ -29,7 +56,7 @@ if [[ ! $? ]]; then saved_stty="" fi -$FWDIR/run spark.repl.Main "$@" +$FWDIR/run $OPTIONS spark.repl.Main "$@" # record the exit status lest it be overwritten: # then reenable echo and propagate the code. -- cgit v1.2.3 From 7680ce0bd65fc44716c5bc03d5909a3ddbd43501 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 16:11:44 -0400 Subject: Fixed deprecated use of expect in SizeEstimatorSuite --- core/src/test/scala/spark/SizeEstimatorSuite.scala | 72 +++++++++++----------- 1 file changed, 36 insertions(+), 36 deletions(-) diff --git a/core/src/test/scala/spark/SizeEstimatorSuite.scala b/core/src/test/scala/spark/SizeEstimatorSuite.scala index e235ef2f67..b5c8525f91 100644 --- a/core/src/test/scala/spark/SizeEstimatorSuite.scala +++ b/core/src/test/scala/spark/SizeEstimatorSuite.scala @@ -35,7 +35,7 @@ class SizeEstimatorSuite var oldOops: String = _ override def beforeAll() { - // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case + // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case oldArch = System.setProperty("os.arch", "amd64") oldOops = System.setProperty("spark.test.useCompressedOops", "true") } @@ -46,54 +46,54 @@ class SizeEstimatorSuite } test("simple classes") { - expect(16)(SizeEstimator.estimate(new DummyClass1)) - expect(16)(SizeEstimator.estimate(new DummyClass2)) - expect(24)(SizeEstimator.estimate(new DummyClass3)) - expect(24)(SizeEstimator.estimate(new DummyClass4(null))) - expect(48)(SizeEstimator.estimate(new DummyClass4(new DummyClass3))) + assert(SizeEstimator.estimate(new DummyClass1) === 16) + assert(SizeEstimator.estimate(new DummyClass2) === 16) + assert(SizeEstimator.estimate(new DummyClass3) === 24) + assert(SizeEstimator.estimate(new DummyClass4(null)) === 24) + assert(SizeEstimator.estimate(new DummyClass4(new DummyClass3)) === 48) } // NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors // (Sun vs IBM). Use a DummyString class to make tests deterministic. test("strings") { - expect(40)(SizeEstimator.estimate(DummyString(""))) - expect(48)(SizeEstimator.estimate(DummyString("a"))) - expect(48)(SizeEstimator.estimate(DummyString("ab"))) - expect(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) + assert(SizeEstimator.estimate(DummyString("")) === 40) + assert(SizeEstimator.estimate(DummyString("a")) === 48) + assert(SizeEstimator.estimate(DummyString("ab")) === 48) + assert(SizeEstimator.estimate(DummyString("abcdefgh")) === 56) } test("primitive arrays") { - expect(32)(SizeEstimator.estimate(new Array[Byte](10))) - expect(40)(SizeEstimator.estimate(new Array[Char](10))) - expect(40)(SizeEstimator.estimate(new Array[Short](10))) - expect(56)(SizeEstimator.estimate(new Array[Int](10))) - expect(96)(SizeEstimator.estimate(new Array[Long](10))) - expect(56)(SizeEstimator.estimate(new Array[Float](10))) - expect(96)(SizeEstimator.estimate(new Array[Double](10))) - expect(4016)(SizeEstimator.estimate(new Array[Int](1000))) - expect(8016)(SizeEstimator.estimate(new Array[Long](1000))) + assert(SizeEstimator.estimate(new Array[Byte](10)) === 32) + assert(SizeEstimator.estimate(new Array[Char](10)) === 40) + assert(SizeEstimator.estimate(new Array[Short](10)) === 40) + assert(SizeEstimator.estimate(new Array[Int](10)) === 56) + assert(SizeEstimator.estimate(new Array[Long](10)) === 96) + assert(SizeEstimator.estimate(new Array[Float](10)) === 56) + assert(SizeEstimator.estimate(new Array[Double](10)) === 96) + assert(SizeEstimator.estimate(new Array[Int](1000)) === 4016) + assert(SizeEstimator.estimate(new Array[Long](1000)) === 8016) } test("object arrays") { // Arrays containing nulls should just have one pointer per element - expect(56)(SizeEstimator.estimate(new Array[String](10))) - expect(56)(SizeEstimator.estimate(new Array[AnyRef](10))) + assert(SizeEstimator.estimate(new Array[String](10)) === 56) + assert(SizeEstimator.estimate(new Array[AnyRef](10)) === 56) // For object arrays with non-null elements, each object should take one pointer plus // however many bytes that class takes. (Note that Array.fill calls the code in its // second parameter separately for each object, so we get distinct objects.) - expect(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass1))) - expect(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass2))) - expect(296)(SizeEstimator.estimate(Array.fill(10)(new DummyClass3))) - expect(56)(SizeEstimator.estimate(Array(new DummyClass1, new DummyClass2))) + assert(SizeEstimator.estimate(Array.fill(10)(new DummyClass1)) === 216) + assert(SizeEstimator.estimate(Array.fill(10)(new DummyClass2)) === 216) + assert(SizeEstimator.estimate(Array.fill(10)(new DummyClass3)) === 296) + assert(SizeEstimator.estimate(Array(new DummyClass1, new DummyClass2)) === 56) // Past size 100, our samples 100 elements, but we should still get the right size. - expect(28016)(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3))) + assert(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3)) === 28016) // If an array contains the *same* element many times, we should only count it once. val d1 = new DummyClass1 - expect(72)(SizeEstimator.estimate(Array.fill(10)(d1))) // 10 pointers plus 8-byte object - expect(432)(SizeEstimator.estimate(Array.fill(100)(d1))) // 100 pointers plus 8-byte object + assert(SizeEstimator.estimate(Array.fill(10)(d1)) === 72) // 10 pointers plus 8-byte object + assert(SizeEstimator.estimate(Array.fill(100)(d1)) === 432) // 100 pointers plus 8-byte object // Same thing with huge array containing the same element many times. Note that this won't // return exactly 4032 because it can't tell that *all* the elements will equal the first @@ -111,10 +111,10 @@ class SizeEstimatorSuite val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() - expect(40)(SizeEstimator.estimate(DummyString(""))) - expect(48)(SizeEstimator.estimate(DummyString("a"))) - expect(48)(SizeEstimator.estimate(DummyString("ab"))) - expect(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) + assert(SizeEstimator.estimate(DummyString("")) === 40) + assert(SizeEstimator.estimate(DummyString("a")) === 48) + assert(SizeEstimator.estimate(DummyString("ab")) === 48) + assert(SizeEstimator.estimate(DummyString("abcdefgh")) === 56) resetOrClear("os.arch", arch) } @@ -128,10 +128,10 @@ class SizeEstimatorSuite val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() - expect(56)(SizeEstimator.estimate(DummyString(""))) - expect(64)(SizeEstimator.estimate(DummyString("a"))) - expect(64)(SizeEstimator.estimate(DummyString("ab"))) - expect(72)(SizeEstimator.estimate(DummyString("abcdefgh"))) + assert(SizeEstimator.estimate(DummyString("")) === 56) + assert(SizeEstimator.estimate(DummyString("a")) === 64) + assert(SizeEstimator.estimate(DummyString("ab")) === 64) + assert(SizeEstimator.estimate(DummyString("abcdefgh")) === 72) resetOrClear("os.arch", arch) resetOrClear("spark.test.useCompressedOops", oops) -- cgit v1.2.3 From 15b00914c53f1f4f00a3313968f68a8f032e7cb7 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 17:17:27 -0400 Subject: Some fixes to the launch-java-directly change: - Split SPARK_JAVA_OPTS into multiple command-line arguments if it contains spaces; this splitting follows quoting rules in bash - Add the Scala JARs to the classpath if they're not in the CLASSPATH variable because the ExecutorRunner is launched with "scala" (this can happen when using local-cluster URLs in spark-shell) --- core/src/main/scala/spark/Utils.scala | 65 +++++++++++++++++++++- .../scala/spark/deploy/worker/ExecutorRunner.scala | 51 +++++++++++------ core/src/test/scala/spark/UtilsSuite.scala | 53 +++++++++++++----- 3 files changed, 138 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index f3621c6bee..bdc1494cc9 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -522,7 +522,7 @@ private object Utils extends Logging { execute(command, new File(".")) } - private[spark] class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String, + private[spark] class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String, val firstUserLine: Int, val firstUserClass: String) /** * When called inside a class in the spark package, returns the name of the user code class @@ -610,4 +610,67 @@ private object Utils extends Logging { } return false } + + def isSpace(c: Char): Boolean = { + " \t\r\n".indexOf(c) != -1 + } + + /** + * Split a string of potentially quoted arguments from the command line the way that a shell + * would do it to determine arguments to a command. For example, if the string is 'a "b c" d', + * then it would be parsed as three arguments: 'a', 'b c' and 'd'. + */ + def splitCommandString(s: String): Seq[String] = { + val buf = new ArrayBuffer[String] + var inWord = false + var inSingleQuote = false + var inDoubleQuote = false + var curWord = new StringBuilder + def endWord() { + buf += curWord.toString + curWord.clear() + } + var i = 0 + while (i < s.length) { + var nextChar = s.charAt(i) + if (inDoubleQuote) { + if (nextChar == '"') { + inDoubleQuote = false + } else if (nextChar == '\\') { + if (i < s.length - 1) { + // Append the next character directly, because only " and \ may be escaped in + // double quotes after the shell's own expansion + curWord.append(s.charAt(i + 1)) + i += 1 + } + } else { + curWord.append(nextChar) + } + } else if (inSingleQuote) { + if (nextChar == '\'') { + inSingleQuote = false + } else { + curWord.append(nextChar) + } + // Backslashes are not treated specially in single quotes + } else if (nextChar == '"') { + inWord = true + inDoubleQuote = true + } else if (nextChar == '\'') { + inWord = true + inSingleQuote = true + } else if (!isSpace(nextChar)) { + curWord.append(nextChar) + inWord = true + } else if (inWord && isSpace(nextChar)) { + endWord() + inWord = false + } + i += 1 + } + if (inWord || inDoubleQuote || inSingleQuote) { + endWord() + } + return buf + } } diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 4d31657d9e..db580e39ab 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -40,7 +40,7 @@ private[spark] class ExecutorRunner( workerThread.start() // Shutdown hook that kills actors on shutdown. - shutdownHook = new Thread() { + shutdownHook = new Thread() { override def run() { if (process != null) { logInfo("Shutdown hook killing child process.") @@ -87,25 +87,43 @@ private[spark] class ExecutorRunner( Seq(runner) ++ buildJavaOpts() ++ Seq(command.mainClass) ++ command.arguments.map(substituteVariables) } - - /* - * Attention: this must always be aligned with the environment variables in the run scripts and the - * way the JAVA_OPTS are assembled there. + + /** + * Attention: this must always be aligned with the environment variables in the run scripts and + * the way the JAVA_OPTS are assembled there. */ def buildJavaOpts(): Seq[String] = { - val _javaLibPath = if (System.getenv("SPARK_LIBRARY_PATH") == null) { - "" + val libraryOpts = if (System.getenv("SPARK_LIBRARY_PATH") == null) { + Nil + } else { + List("-Djava.library.path=" + System.getenv("SPARK_LIBRARY_PATH")) + } + + val userOpts = if (System.getenv("SPARK_JAVA_OPTS") == null) { + Nil } else { - "-Djava.library.path=" + System.getenv("SPARK_LIBRARY_PATH") + Utils.splitCommandString(System.getenv("SPARK_JAVA_OPTS")) } - - Seq("-cp", - System.getenv("CLASSPATH"), - System.getenv("SPARK_JAVA_OPTS"), - _javaLibPath, - "-Xms" + memory.toString + "M", - "-Xmx" + memory.toString + "M") - .filter(_ != null) + + val memoryOpts = Seq("-Xms" + memory + "M", "-Xmx" + memory + "M") + + var classPath = System.getenv("CLASSPATH") + if (System.getenv("SPARK_LAUNCH_WITH_SCALA") == "1") { + // Add the Scala library JARs to the classpath; this is needed when the ExecutorRunner + // was launched with "scala" as the runner (e.g. in spark-shell in local-cluster mode) + // and the Scala libraries won't be in the CLASSPATH environment variable by defalt. + if (System.getenv("SCALA_LIBRARY_PATH") == null && System.getenv("SCALA_HOME") == null) { + logError("Cloud not launch executors: neither SCALA_LIBRARY_PATH nor SCALA_HOME are set") + System.exit(1) + } + val scalaLib = Option(System.getenv("SCALA_LIBRARY_PATH")).getOrElse( + System.getenv("SCALA_HOME") + "/lib") + classPath += ":" + scalaLib + "/scala-library.jar" + + ":" + scalaLib + "/scala-compiler.jar" + + ":" + scalaLib + "/jline.jar" + } + + Seq("-cp", classPath) ++ libraryOpts ++ userOpts ++ memoryOpts } /** Spawn a thread that will redirect a given stream to a file */ @@ -136,6 +154,7 @@ private[spark] class ExecutorRunner( // Launch the process val command = buildCommandSeq() + println("COMMAND: " + command.mkString(" ")) val builder = new ProcessBuilder(command: _*).directory(executorDir) val env = builder.environment() for ((key, value) <- appDesc.command.environment) { diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala index ed4701574f..4a113e16bf 100644 --- a/core/src/test/scala/spark/UtilsSuite.scala +++ b/core/src/test/scala/spark/UtilsSuite.scala @@ -27,24 +27,49 @@ class UtilsSuite extends FunSuite { assert(os.toByteArray.toList.equals(bytes.toList)) } - test("memoryStringToMb"){ - assert(Utils.memoryStringToMb("1") == 0) - assert(Utils.memoryStringToMb("1048575") == 0) - assert(Utils.memoryStringToMb("3145728") == 3) + test("memoryStringToMb") { + assert(Utils.memoryStringToMb("1") === 0) + assert(Utils.memoryStringToMb("1048575") === 0) + assert(Utils.memoryStringToMb("3145728") === 3) - assert(Utils.memoryStringToMb("1024k") == 1) - assert(Utils.memoryStringToMb("5000k") == 4) - assert(Utils.memoryStringToMb("4024k") == Utils.memoryStringToMb("4024K")) + assert(Utils.memoryStringToMb("1024k") === 1) + assert(Utils.memoryStringToMb("5000k") === 4) + assert(Utils.memoryStringToMb("4024k") === Utils.memoryStringToMb("4024K")) - assert(Utils.memoryStringToMb("1024m") == 1024) - assert(Utils.memoryStringToMb("5000m") == 5000) - assert(Utils.memoryStringToMb("4024m") == Utils.memoryStringToMb("4024M")) + assert(Utils.memoryStringToMb("1024m") === 1024) + assert(Utils.memoryStringToMb("5000m") === 5000) + assert(Utils.memoryStringToMb("4024m") === Utils.memoryStringToMb("4024M")) - assert(Utils.memoryStringToMb("2g") == 2048) - assert(Utils.memoryStringToMb("3g") == Utils.memoryStringToMb("3G")) + assert(Utils.memoryStringToMb("2g") === 2048) + assert(Utils.memoryStringToMb("3g") === Utils.memoryStringToMb("3G")) - assert(Utils.memoryStringToMb("2t") == 2097152) - assert(Utils.memoryStringToMb("3t") == Utils.memoryStringToMb("3T")) + assert(Utils.memoryStringToMb("2t") === 2097152) + assert(Utils.memoryStringToMb("3t") === Utils.memoryStringToMb("3T")) + } + + test("splitCommandString") { + assert(Utils.splitCommandString("") === Seq()) + assert(Utils.splitCommandString("a") === Seq("a")) + assert(Utils.splitCommandString("aaa") === Seq("aaa")) + assert(Utils.splitCommandString("a b c") === Seq("a", "b", "c")) + assert(Utils.splitCommandString(" a b\t c ") === Seq("a", "b", "c")) + assert(Utils.splitCommandString("a 'b c'") === Seq("a", "b c")) + assert(Utils.splitCommandString("a 'b c' d") === Seq("a", "b c", "d")) + assert(Utils.splitCommandString("'b c'") === Seq("b c")) + assert(Utils.splitCommandString("a \"b c\"") === Seq("a", "b c")) + assert(Utils.splitCommandString("a \"b c\" d") === Seq("a", "b c", "d")) + assert(Utils.splitCommandString("\"b c\"") === Seq("b c")) + assert(Utils.splitCommandString("a 'b\" c' \"d' e\"") === Seq("a", "b\" c", "d' e")) + assert(Utils.splitCommandString("a\t'b\nc'\nd") === Seq("a", "b\nc", "d")) + assert(Utils.splitCommandString("a \"b\\\\c\"") === Seq("a", "b\\c")) + assert(Utils.splitCommandString("a \"b\\\"c\"") === Seq("a", "b\"c")) + assert(Utils.splitCommandString("a 'b\\\"c'") === Seq("a", "b\\\"c")) + assert(Utils.splitCommandString("'a'b") === Seq("ab")) + assert(Utils.splitCommandString("'a''b'") === Seq("ab")) + assert(Utils.splitCommandString("\"a\"b") === Seq("ab")) + assert(Utils.splitCommandString("\"a\"\"b\"") === Seq("ab")) + assert(Utils.splitCommandString("''") === Seq("")) + assert(Utils.splitCommandString("\"\"") === Seq("")) } } -- cgit v1.2.3 From 366572edcab87701fd795ca0142ac9829b312d36 Mon Sep 17 00:00:00 2001 From: James Phillpotts Date: Tue, 25 Jun 2013 22:59:34 +0100 Subject: Include a default OAuth implementation, and update examples and JavaStreamingContext --- .../streaming/examples/TwitterAlgebirdCMS.scala | 2 +- .../streaming/examples/TwitterAlgebirdHLL.scala | 2 +- .../streaming/examples/TwitterPopularTags.scala | 2 +- .../scala/spark/streaming/StreamingContext.scala | 2 +- .../streaming/api/java/JavaStreamingContext.scala | 69 +++++++++++++++------- .../streaming/dstream/TwitterInputDStream.scala | 32 +++++++++- 6 files changed, 81 insertions(+), 28 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala index a9642100e3..548190309e 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -45,7 +45,7 @@ object TwitterAlgebirdCMS { val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) - val stream = ssc.twitterStream(username, password, filters, StorageLevel.MEMORY_ONLY_SER) + val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala index f3288bfb85..5a86c6318d 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -34,7 +34,7 @@ object TwitterAlgebirdHLL { val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) - val stream = ssc.twitterStream(username, password, filters, StorageLevel.MEMORY_ONLY_SER) + val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala index 9d4494c6f2..076c3878c8 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala @@ -23,7 +23,7 @@ object TwitterPopularTags { val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) - val stream = ssc.twitterStream(username, password, filters) + val stream = ssc.twitterStream(None, filters) val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#"))) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 05be6bd58a..0f36504c0d 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -368,7 +368,7 @@ class StreamingContext private ( * @param storageLevel Storage level to use for storing the received objects */ def twitterStream( - twitterAuth: Authorization, + twitterAuth: Option[Authorization] = None, filters: Seq[String] = Nil, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 ): DStream[Status] = { diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index 3d149a742c..85390ef57e 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -4,23 +4,18 @@ import spark.streaming._ import receivers.{ActorReceiver, ReceiverSupervisorStrategy} import spark.streaming.dstream._ import spark.storage.StorageLevel - import spark.api.java.function.{Function => JFunction, Function2 => JFunction2} import spark.api.java.{JavaSparkContext, JavaRDD} - import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} - import twitter4j.Status - import akka.actor.Props import akka.actor.SupervisorStrategy import akka.zeromq.Subscribe - import scala.collection.JavaConversions._ - import java.lang.{Long => JLong, Integer => JInt} import java.io.InputStream import java.util.{Map => JMap} +import twitter4j.auth.Authorization /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic @@ -315,46 +310,78 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Create a input stream that returns tweets received from Twitter. - * @param username Twitter username - * @param password Twitter password + * @param twitterAuth Twitter4J Authorization * @param filters Set of filter strings to get only those tweets that match them * @param storageLevel Storage level to use for storing the received objects */ def twitterStream( - username: String, - password: String, + twitterAuth: Authorization, filters: Array[String], storageLevel: StorageLevel ): JavaDStream[Status] = { - ssc.twitterStream(username, password, filters, storageLevel) + ssc.twitterStream(Some(twitterAuth), filters, storageLevel) + } + + /** + * Create a input stream that returns tweets received from Twitter using + * java.util.Preferences to store OAuth token. OAuth key and secret should + * be provided using system properties twitter4j.oauth.consumerKey and + * twitter4j.oauth.consumerSecret + * @param filters Set of filter strings to get only those tweets that match them + * @param storageLevel Storage level to use for storing the received objects + */ + def twitterStream( + filters: Array[String], + storageLevel: StorageLevel + ): JavaDStream[Status] = { + ssc.twitterStream(None, filters, storageLevel) } /** * Create a input stream that returns tweets received from Twitter. - * @param username Twitter username - * @param password Twitter password + * @param twitterAuth Twitter4J Authorization * @param filters Set of filter strings to get only those tweets that match them */ def twitterStream( - username: String, - password: String, + twitterAuth: Authorization, filters: Array[String] ): JavaDStream[Status] = { - ssc.twitterStream(username, password, filters) + ssc.twitterStream(Some(twitterAuth), filters) + } + + /** + * Create a input stream that returns tweets received from Twitter using + * java.util.Preferences to store OAuth token. OAuth key and secret should + * be provided using system properties twitter4j.oauth.consumerKey and + * twitter4j.oauth.consumerSecret + * @param filters Set of filter strings to get only those tweets that match them + */ + def twitterStream( + filters: Array[String] + ): JavaDStream[Status] = { + ssc.twitterStream(None, filters) } /** * Create a input stream that returns tweets received from Twitter. - * @param username Twitter username - * @param password Twitter password + * @param twitterAuth Twitter4J Authorization */ def twitterStream( - username: String, - password: String + twitterAuth: Authorization ): JavaDStream[Status] = { - ssc.twitterStream(username, password) + ssc.twitterStream(Some(twitterAuth)) } + /** + * Create a input stream that returns tweets received from Twitter using + * java.util.Preferences to store OAuth token. OAuth key and secret should + * be provided using system properties twitter4j.oauth.consumerKey and + * twitter4j.oauth.consumerSecret + */ + def twitterStream(): JavaDStream[Status] = { + ssc.twitterStream() + } + /** * Create an input stream with any arbitrary user implemented actor receiver. * @param props Props object defining creation of the actor diff --git a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala index 0b01091a52..e0c654d385 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala @@ -3,27 +3,53 @@ package spark.streaming.dstream import spark._ import spark.streaming._ import storage.StorageLevel - import twitter4j._ import twitter4j.auth.BasicAuthorization import twitter4j.auth.Authorization +import java.util.prefs.Preferences +import twitter4j.conf.PropertyConfiguration +import twitter4j.auth.OAuthAuthorization +import twitter4j.auth.AccessToken /* A stream of Twitter statuses, potentially filtered by one or more keywords. * * @constructor create a new Twitter stream using the supplied username and password to authenticate. * An optional set of string filters can be used to restrict the set of tweets. The Twitter API is * such that this may return a sampled subset of all tweets during each interval. +* +* Includes a simple implementation of OAuth using consumer key and secret provided using system +* properties twitter4j.oauth.consumerKey and twitter4j.oauth.consumerSecret */ private[streaming] class TwitterInputDStream( @transient ssc_ : StreamingContext, - twitterAuth: Authorization, + twitterAuth: Option[Authorization], filters: Seq[String], storageLevel: StorageLevel ) extends NetworkInputDStream[Status](ssc_) { + lazy val createOAuthAuthorization: Authorization = { + val userRoot = Preferences.userRoot(); + val token = Option(userRoot.get(PropertyConfiguration.OAUTH_ACCESS_TOKEN, null)) + val tokenSecret = Option(userRoot.get(PropertyConfiguration.OAUTH_ACCESS_TOKEN_SECRET, null)) + val oAuth = new OAuthAuthorization(new PropertyConfiguration(System.getProperties())) + if (token.isEmpty || tokenSecret.isEmpty) { + val requestToken = oAuth.getOAuthRequestToken() + println("Authorize application using URL: "+requestToken.getAuthorizationURL()) + println("Enter PIN: ") + val pin = Console.readLine + val accessToken = if (pin.length() > 0) oAuth.getOAuthAccessToken(requestToken, pin) else oAuth.getOAuthAccessToken() + userRoot.put(PropertyConfiguration.OAUTH_ACCESS_TOKEN, accessToken.getToken()) + userRoot.put(PropertyConfiguration.OAUTH_ACCESS_TOKEN_SECRET, accessToken.getTokenSecret()) + userRoot.flush() + } else { + oAuth.setOAuthAccessToken(new AccessToken(token.get, tokenSecret.get)); + } + oAuth + } + override def getReceiver(): NetworkReceiver[Status] = { - new TwitterReceiver(twitterAuth, filters, storageLevel) + new TwitterReceiver(if (twitterAuth.isEmpty) createOAuthAuthorization else twitterAuth.get, filters, storageLevel) } } -- cgit v1.2.3 From 8dd78f80b578751df164772a01772aad26540ac9 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 25 Jun 2013 15:01:06 -0700 Subject: Manually merge spark/master commit d92d3f7938dec954ea31de232f50cafd4b644065 --- set-dev-classpath.sh | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/set-dev-classpath.sh b/set-dev-classpath.sh index 4d09bd4416..d031c56baf 100644 --- a/set-dev-classpath.sh +++ b/set-dev-classpath.sh @@ -74,10 +74,15 @@ if [ -e "$FWDIR/lib_managed" ]; then CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*" fi CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*" + +# Add the shaded JAR for Maven builds if [ -e $REPL_BIN_DIR/target ]; then for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do CLASSPATH="$CLASSPATH:$jar" done + # The shaded JAR doesn't contain examples, so include those separately + EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` + CLASSPATH+=":$EXAMPLES_JAR" fi CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do @@ -90,11 +95,12 @@ if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; # Use the JAR from the SBT build export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` fi -if [ -e "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar ]; then +if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then # Use the JAR from the Maven build - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar` + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` fi + # Figure out whether to run our class with java or with the scala launcher. # In most cases, we'd prefer to execute our process with java because scala # creates a shell script as the parent of its Java process, which makes it -- cgit v1.2.3 From 176193b1e8acdbe2f1cfaed16b8f42f89e226f79 Mon Sep 17 00:00:00 2001 From: James Phillpotts Date: Tue, 25 Jun 2013 23:06:15 +0100 Subject: Fix usage and parameter extraction --- .../main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala | 7 +++---- .../main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala | 7 +++---- .../main/scala/spark/streaming/examples/TwitterPopularTags.scala | 7 +++---- 3 files changed, 9 insertions(+), 12 deletions(-) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala index 548190309e..528778ed72 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -26,8 +26,8 @@ import spark.SparkContext._ */ object TwitterAlgebirdCMS { def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: TwitterAlgebirdCMS " + + if (args.length < 1) { + System.err.println("Usage: TwitterAlgebirdCMS " + " [filter1] [filter2] ... [filter n]") System.exit(1) } @@ -40,8 +40,7 @@ object TwitterAlgebirdCMS { // K highest frequency elements to take val TOPK = 10 - val Array(master, username, password) = args.slice(0, 3) - val filters = args.slice(3, args.length) + val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala index 5a86c6318d..896e9fd8af 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -21,16 +21,15 @@ import spark.streaming.dstream.TwitterInputDStream */ object TwitterAlgebirdHLL { def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: TwitterAlgebirdHLL " + + if (args.length < 1) { + System.err.println("Usage: TwitterAlgebirdHLL " + " [filter1] [filter2] ... [filter n]") System.exit(1) } /** Bit size parameter for HyperLogLog, trades off accuracy vs size */ val BIT_SIZE = 12 - val Array(master, username, password) = args.slice(0, 3) - val filters = args.slice(3, args.length) + val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala index 076c3878c8..65f0b6d352 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala @@ -12,14 +12,13 @@ import spark.SparkContext._ */ object TwitterPopularTags { def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: TwitterPopularTags " + + if (args.length < 1) { + System.err.println("Usage: TwitterPopularTags " + " [filter1] [filter2] ... [filter n]") System.exit(1) } - val Array(master, username, password) = args.slice(0, 3) - val filters = args.slice(3, args.length) + val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) -- cgit v1.2.3 From 6c8d1b2ca618a1a17566ede46821c0807a1b11f5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 18:21:00 -0400 Subject: Fix computation of classpath when we launch java directly The previous version assumed that a CLASSPATH environment variable was set by the "run" script when launching the process that starts the ExecutorRunner, but unfortunately this is not true in tests. Instead, we factor the classpath calculation into an extenral script and call that. NOTE: This includes a Windows version but hasn't yet been tested there. --- bin/compute-classpath.cmd | 52 +++++++++++++ bin/compute-classpath.sh | 89 ++++++++++++++++++++++ core/src/main/scala/spark/Utils.scala | 31 ++++++++ .../scala/spark/deploy/worker/ExecutorRunner.scala | 19 +---- run | 67 +++------------- run2.cmd | 38 +-------- 6 files changed, 189 insertions(+), 107 deletions(-) create mode 100644 bin/compute-classpath.cmd create mode 100755 bin/compute-classpath.sh diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd new file mode 100644 index 0000000000..1dff8fea22 --- /dev/null +++ b/bin/compute-classpath.cmd @@ -0,0 +1,52 @@ +@echo off + +rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run" +rem script and the ExecutorRunner in standalone cluster mode. + +set SCALA_VERSION=2.9.3 + +rem Figure out where the Spark framework is installed +set FWDIR=%~dp0\.. + +rem Load environment variables from conf\spark-env.cmd, if it exists +if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" + +set CORE_DIR=%FWDIR%core +set REPL_DIR=%FWDIR%repl +set EXAMPLES_DIR=%FWDIR%examples +set BAGEL_DIR=%FWDIR%bagel +set STREAMING_DIR=%FWDIR%streaming +set PYSPARK_DIR=%FWDIR%python + +rem Build up classpath +set CLASSPATH=%SPARK_CLASSPATH%;%MESOS_CLASSPATH%;%FWDIR%conf;%CORE_DIR%\target\scala-%SCALA_VERSION%\classes +set CLASSPATH=%CLASSPATH%;%CORE_DIR%\target\scala-%SCALA_VERSION%\test-classes;%CORE_DIR%\src\main\resources +set CLASSPATH=%CLASSPATH%;%STREAMING_DIR%\target\scala-%SCALA_VERSION%\classes;%STREAMING_DIR%\target\scala-%SCALA_VERSION%\test-classes +set CLASSPATH=%CLASSPATH%;%STREAMING_DIR%\lib\org\apache\kafka\kafka\0.7.2-spark\* +set CLASSPATH=%CLASSPATH%;%REPL_DIR%\target\scala-%SCALA_VERSION%\classes;%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\classes +set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\jars\* +set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\bundles\* +set CLASSPATH=%CLASSPATH%;%FWDIR%repl\lib\* +set CLASSPATH=%CLASSPATH%;%FWDIR%python\lib\* +set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes + +rem Add hadoop conf dir - else FileSystem.*, etc fail +rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts +rem the configurtion files. +if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir + set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% +:no_hadoop_conf_dir + +if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir + set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% +:no_yarn_conf_dir + +rem Add Scala standard library +set CLASSPATH=%CLASSPATH%;%SCALA_HOME%\lib\scala-library.jar;%SCALA_HOME%\lib\scala-compiler.jar;%SCALA_HOME%\lib\jline.jar + +rem A bit of a hack to allow calling this script within run2.cmd without seeing output +if "x%DONT_PRINT_CLASSPATH%"=="x1" goto exit + +echo %CLASSPATH% + +:exit diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh new file mode 100755 index 0000000000..3a78880290 --- /dev/null +++ b/bin/compute-classpath.sh @@ -0,0 +1,89 @@ +#!/bin/bash + +# This script computes Spark's classpath and prints it to stdout; it's used by both the "run" +# script and the ExecutorRunner in standalone cluster mode. + +SCALA_VERSION=2.9.3 + +# Figure out where Spark is installed +FWDIR="$(cd `dirname $0`/..; pwd)" + +# Load environment variables from conf/spark-env.sh, if it exists +if [ -e $FWDIR/conf/spark-env.sh ] ; then + . $FWDIR/conf/spark-env.sh +fi + +CORE_DIR="$FWDIR/core" +REPL_DIR="$FWDIR/repl" +REPL_BIN_DIR="$FWDIR/repl-bin" +EXAMPLES_DIR="$FWDIR/examples" +BAGEL_DIR="$FWDIR/bagel" +STREAMING_DIR="$FWDIR/streaming" +PYSPARK_DIR="$FWDIR/python" + +# Build up classpath +CLASSPATH="$SPARK_CLASSPATH" +CLASSPATH="$CLASSPATH:$FWDIR/conf" +CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/classes" +if [ -n "$SPARK_TESTING" ] ; then + CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes" +fi +CLASSPATH="$CLASSPATH:$CORE_DIR/src/main/resources" +CLASSPATH="$CLASSPATH:$REPL_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar +if [ -e "$FWDIR/lib_managed" ]; then + CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/jars/*" + CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*" +fi +CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*" +# Add the shaded JAR for Maven builds +if [ -e $REPL_BIN_DIR/target ]; then + for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do + CLASSPATH="$CLASSPATH:$jar" + done + # The shaded JAR doesn't contain examples, so include those separately + EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` + CLASSPATH+=":$EXAMPLES_JAR" +fi +CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" +for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do + CLASSPATH="$CLASSPATH:$jar" +done + +# Figure out the JAR file that our examples were packaged into. This includes a bit of a hack +# to avoid the -sources and -doc packages that are built by publish-local. +if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then + # Use the JAR from the SBT build + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` +fi +if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then + # Use the JAR from the Maven build + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` +fi + +# Add hadoop conf dir - else FileSystem.*, etc fail ! +# Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts +# the configurtion files. +if [ "x" != "x$HADOOP_CONF_DIR" ]; then + CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR" +fi +if [ "x" != "x$YARN_CONF_DIR" ]; then + CLASSPATH="$CLASSPATH:$YARN_CONF_DIR" +fi + +# Add Scala standard library +if [ -z "$SCALA_LIBRARY_PATH" ]; then + if [ -z "$SCALA_HOME" ]; then + echo "SCALA_HOME is not set" >&2 + exit 1 + fi + SCALA_LIBRARY_PATH="$SCALA_HOME/lib" +fi +CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-library.jar" +CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-compiler.jar" +CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/jline.jar" + +echo "$CLASSPATH" diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index bdc1494cc9..f41efa9d29 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -522,6 +522,37 @@ private object Utils extends Logging { execute(command, new File(".")) } + /** + * Execute a command and get its output, throwing an exception if it yields a code other than 0. + */ + def executeAndGetOutput(command: Seq[String], workingDir: File = new File(".")): String = { + val process = new ProcessBuilder(command: _*) + .directory(workingDir) + .start() + new Thread("read stderr for " + command(0)) { + override def run() { + for (line <- Source.fromInputStream(process.getErrorStream).getLines) { + System.err.println(line) + } + } + }.start() + val output = new StringBuffer + val stdoutThread = new Thread("read stdout for " + command(0)) { + override def run() { + for (line <- Source.fromInputStream(process.getInputStream).getLines) { + output.append(line) + } + } + } + stdoutThread.start() + val exitCode = process.waitFor() + stdoutThread.join() // Wait for it to finish reading output + if (exitCode != 0) { + throw new SparkException("Process " + command + " exited with code " + exitCode) + } + output.toString + } + private[spark] class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String, val firstUserLine: Int, val firstUserClass: String) /** diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index db580e39ab..4f8e1dcb26 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -107,21 +107,9 @@ private[spark] class ExecutorRunner( val memoryOpts = Seq("-Xms" + memory + "M", "-Xmx" + memory + "M") - var classPath = System.getenv("CLASSPATH") - if (System.getenv("SPARK_LAUNCH_WITH_SCALA") == "1") { - // Add the Scala library JARs to the classpath; this is needed when the ExecutorRunner - // was launched with "scala" as the runner (e.g. in spark-shell in local-cluster mode) - // and the Scala libraries won't be in the CLASSPATH environment variable by defalt. - if (System.getenv("SCALA_LIBRARY_PATH") == null && System.getenv("SCALA_HOME") == null) { - logError("Cloud not launch executors: neither SCALA_LIBRARY_PATH nor SCALA_HOME are set") - System.exit(1) - } - val scalaLib = Option(System.getenv("SCALA_LIBRARY_PATH")).getOrElse( - System.getenv("SCALA_HOME") + "/lib") - classPath += ":" + scalaLib + "/scala-library.jar" + - ":" + scalaLib + "/scala-compiler.jar" + - ":" + scalaLib + "/jline.jar" - } + // Figure out our classpath with the external compute-classpath script + val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh" + val classPath = Utils.executeAndGetOutput(Seq(sparkHome + "/bin/compute-classpath" + ext)) Seq("-cp", classPath) ++ libraryOpts ++ userOpts ++ memoryOpts } @@ -154,7 +142,6 @@ private[spark] class ExecutorRunner( // Launch the process val command = buildCommandSeq() - println("COMMAND: " + command.mkString(" ")) val builder = new ProcessBuilder(command: _*).directory(executorDir) val env = builder.environment() for ((key, value) <- appDesc.command.environment) { diff --git a/run b/run index 0fb15f8b24..7c06a55062 100755 --- a/run +++ b/run @@ -49,6 +49,12 @@ case "$1" in ;; esac +# Figure out whether to run our class with java or with the scala launcher. +# In most cases, we'd prefer to execute our process with java because scala +# creates a shell script as the parent of its Java process, which makes it +# hard to kill the child with stuff like Process.destroy(). However, for +# the Spark shell, the wrapper is necessary to properly reset the terminal +# when we exit, so we allow it to set a variable to launch with scala. if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then if [ "$SCALA_HOME" ]; then RUNNER="${SCALA_HOME}/bin/scala" @@ -98,12 +104,8 @@ export JAVA_OPTS # Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! CORE_DIR="$FWDIR/core" -REPL_DIR="$FWDIR/repl" -REPL_BIN_DIR="$FWDIR/repl-bin" EXAMPLES_DIR="$FWDIR/examples" -BAGEL_DIR="$FWDIR/bagel" -STREAMING_DIR="$FWDIR/streaming" -PYSPARK_DIR="$FWDIR/python" +REPL_DIR="$FWDIR/repl" # Exit if the user hasn't compiled Spark if [ ! -e "$CORE_DIR/target" ]; then @@ -118,37 +120,9 @@ if [[ "$@" = *repl* && ! -e "$REPL_DIR/target" ]]; then exit 1 fi -# Build up classpath -CLASSPATH="$SPARK_CLASSPATH" -CLASSPATH="$CLASSPATH:$FWDIR/conf" -CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/classes" -if [ -n "$SPARK_TESTING" ] ; then - CLASSPATH="$CLASSPATH:$CORE_DIR/target/scala-$SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/test-classes" -fi -CLASSPATH="$CLASSPATH:$CORE_DIR/src/main/resources" -CLASSPATH="$CLASSPATH:$REPL_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH="$CLASSPATH:$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH="$CLASSPATH:$STREAMING_DIR/target/scala-$SCALA_VERSION/classes" -CLASSPATH="$CLASSPATH:$STREAMING_DIR/lib/org/apache/kafka/kafka/0.7.2-spark/*" # <-- our in-project Kafka Jar -if [ -e "$FWDIR/lib_managed" ]; then - CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/jars/*" - CLASSPATH="$CLASSPATH:$FWDIR/lib_managed/bundles/*" -fi -CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*" -# Add the shaded JAR for Maven builds -if [ -e $REPL_BIN_DIR/target ]; then - for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do - CLASSPATH="$CLASSPATH:$jar" - done - # The shaded JAR doesn't contain examples, so include those separately - EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` - CLASSPATH+=":$EXAMPLES_JAR" -fi -CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" -for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do - CLASSPATH="$CLASSPATH:$jar" -done +# Compute classpath using external script +CLASSPATH=`$FWDIR/bin/compute-classpath.sh` +export CLASSPATH # Figure out the JAR file that our examples were packaged into. This includes a bit of a hack # to avoid the -sources and -doc packages that are built by publish-local. @@ -161,32 +135,11 @@ if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` fi -# Add hadoop conf dir - else FileSystem.*, etc fail ! -# Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts -# the configurtion files. -if [ "x" != "x$HADOOP_CONF_DIR" ]; then - CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR" -fi -if [ "x" != "x$YARN_CONF_DIR" ]; then - CLASSPATH="$CLASSPATH:$YARN_CONF_DIR" -fi - - -# Figure out whether to run our class with java or with the scala launcher. -# In most cases, we'd prefer to execute our process with java because scala -# creates a shell script as the parent of its Java process, which makes it -# hard to kill the child with stuff like Process.destroy(). However, for -# the Spark shell, the wrapper is necessary to properly reset the terminal -# when we exit, so we allow it to set a variable to launch with scala. if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then EXTRA_ARGS="" # Java options will be passed to scala as JAVA_OPTS else - CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-library.jar" - CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-compiler.jar" - CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/jline.jar" # The JVM doesn't read JAVA_OPTS by default so we need to pass it in EXTRA_ARGS="$JAVA_OPTS" fi -export CLASSPATH # Needed for spark-shell exec "$RUNNER" -cp "$CLASSPATH" $EXTRA_ARGS "$@" diff --git a/run2.cmd b/run2.cmd index bf76844d11..25e4f3b57c 100644 --- a/run2.cmd +++ b/run2.cmd @@ -33,51 +33,21 @@ if not "x%SCALA_HOME%"=="x" goto scala_exists goto exit :scala_exists -rem If the user specifies a Mesos JAR, put it before our included one on the classpath -set MESOS_CLASSPATH= -if not "x%MESOS_JAR%"=="x" set MESOS_CLASSPATH=%MESOS_JAR% - rem Figure out how much memory to use per executor and set it as an environment rem variable so that our process sees it and can report it to Mesos if "x%SPARK_MEM%"=="x" set SPARK_MEM=512m rem Set JAVA_OPTS to be able to load native libraries and to set heap size set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%SPARK_MEM% -Xmx%SPARK_MEM% -rem Load extra JAVA_OPTS from conf/java-opts, if it exists -if exist "%FWDIR%conf\java-opts.cmd" call "%FWDIR%conf\java-opts.cmd" rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! set CORE_DIR=%FWDIR%core -set REPL_DIR=%FWDIR%repl set EXAMPLES_DIR=%FWDIR%examples -set BAGEL_DIR=%FWDIR%bagel -set STREAMING_DIR=%FWDIR%streaming -set PYSPARK_DIR=%FWDIR%python - -rem Build up classpath -set CLASSPATH=%SPARK_CLASSPATH%;%MESOS_CLASSPATH%;%FWDIR%conf;%CORE_DIR%\target\scala-%SCALA_VERSION%\classes -set CLASSPATH=%CLASSPATH%;%CORE_DIR%\target\scala-%SCALA_VERSION%\test-classes;%CORE_DIR%\src\main\resources -set CLASSPATH=%CLASSPATH%;%STREAMING_DIR%\target\scala-%SCALA_VERSION%\classes;%STREAMING_DIR%\target\scala-%SCALA_VERSION%\test-classes -set CLASSPATH=%CLASSPATH%;%STREAMING_DIR%\lib\org\apache\kafka\kafka\0.7.2-spark\* -set CLASSPATH=%CLASSPATH%;%REPL_DIR%\target\scala-%SCALA_VERSION%\classes;%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\classes -set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\jars\* -set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\bundles\* -set CLASSPATH=%CLASSPATH%;%FWDIR%repl\lib\* -set CLASSPATH=%CLASSPATH%;%FWDIR%python\lib\* -set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes - -rem Add hadoop conf dir - else FileSystem.*, etc fail -rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts -rem the configurtion files. -if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir - set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% -:no_hadoop_conf_dir - -if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir - set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% -:no_yarn_conf_dir - +set REPL_DIR=%FWDIR%repl +rem Compute classpath using external script +set DONT_PRINT_CLASSPATH=1 +call "%FWDIR%bin\compute-classpath.cmd" rem Figure out the JAR file that our examples were packaged into. rem First search in the build path from SBT: -- cgit v1.2.3 From f2263350eda780aba45f383b722e20702c310e6a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 18:35:35 -0400 Subject: Added a local-cluster mode test to ReplSuite --- repl/src/test/scala/spark/repl/ReplSuite.scala | 31 +++++++++++++++++++++----- 1 file changed, 26 insertions(+), 5 deletions(-) diff --git a/repl/src/test/scala/spark/repl/ReplSuite.scala b/repl/src/test/scala/spark/repl/ReplSuite.scala index 1c64f9b98d..72ed8aca5b 100644 --- a/repl/src/test/scala/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/spark/repl/ReplSuite.scala @@ -35,17 +35,17 @@ class ReplSuite extends FunSuite { System.clearProperty("spark.hostPort") return out.toString } - + def assertContains(message: String, output: String) { assert(output contains message, "Interpreter output did not contain '" + message + "':\n" + output) } - + def assertDoesNotContain(message: String, output: String) { assert(!(output contains message), "Interpreter output contained '" + message + "':\n" + output) } - + test ("simple foreach with accumulator") { val output = runInterpreter("local", """ val accum = sc.accumulator(0) @@ -56,7 +56,7 @@ class ReplSuite extends FunSuite { assertDoesNotContain("Exception", output) assertContains("res1: Int = 55", output) } - + test ("external vars") { val output = runInterpreter("local", """ var v = 7 @@ -105,7 +105,7 @@ class ReplSuite extends FunSuite { assertContains("res0: Int = 70", output) assertContains("res1: Int = 100", output) } - + test ("broadcast vars") { // Test that the value that a broadcast var had when it was created is used, // even if that variable is then modified in the driver program @@ -143,6 +143,27 @@ class ReplSuite extends FunSuite { assertContains("res2: Long = 3", output) } + test ("local-cluster mode") { + val output = runInterpreter("local-cluster[1,1,512]", """ + var v = 7 + def getV() = v + sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + v = 10 + sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + var array = new Array[Int](5) + val broadcastArray = sc.broadcast(array) + sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + array(0) = 5 + sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + """) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + assertContains("res0: Int = 70", output) + assertContains("res1: Int = 100", output) + assertContains("res2: Array[Int] = Array(0, 0, 0, 0, 0)", output) + assertContains("res4: Array[Int] = Array(0, 0, 0, 0, 0)", output) + } + if (System.getenv("MESOS_NATIVE_LIBRARY") != null) { test ("running on Mesos") { val output = runInterpreter("localquiet", """ -- cgit v1.2.3 From 2bd04c3513ffa6deabc290a3931be946b1c18713 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 18:37:14 -0400 Subject: Formatting --- repl/src/test/scala/spark/repl/ReplSuite.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/repl/src/test/scala/spark/repl/ReplSuite.scala b/repl/src/test/scala/spark/repl/ReplSuite.scala index 72ed8aca5b..f46e6d8be4 100644 --- a/repl/src/test/scala/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/spark/repl/ReplSuite.scala @@ -28,8 +28,9 @@ class ReplSuite extends FunSuite { val separator = System.getProperty("path.separator") interp.process(Array("-classpath", paths.mkString(separator))) spark.repl.Main.interp = null - if (interp.sparkContext != null) + if (interp.sparkContext != null) { interp.sparkContext.stop() + } // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown System.clearProperty("spark.driver.port") System.clearProperty("spark.hostPort") @@ -37,12 +38,12 @@ class ReplSuite extends FunSuite { } def assertContains(message: String, output: String) { - assert(output contains message, + assert(output.contains(message), "Interpreter output did not contain '" + message + "':\n" + output) } def assertDoesNotContain(message: String, output: String) { - assert(!(output contains message), + assert(!output.contains(message), "Interpreter output contained '" + message + "':\n" + output) } -- cgit v1.2.3 From 9f0d91329516c829c86fc8e95d02071ca7d1f186 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 19:18:30 -0400 Subject: Refactored tests to share SparkContexts in some of them Creating these seems to take a while and clutters the output with Akka stuff, so it would be nice to share them. --- core/src/test/scala/spark/CheckpointSuite.scala | 10 + .../test/scala/spark/PairRDDFunctionsSuite.scala | 287 ++++++++++++++++++++ core/src/test/scala/spark/PartitioningSuite.scala | 19 +- core/src/test/scala/spark/PipedRDDSuite.scala | 16 +- core/src/test/scala/spark/RDDSuite.scala | 87 +----- core/src/test/scala/spark/SharedSparkContext.scala | 25 ++ core/src/test/scala/spark/ShuffleSuite.scala | 298 +-------------------- core/src/test/scala/spark/SortingSuite.scala | 23 +- core/src/test/scala/spark/UnpersistSuite.scala | 30 +++ .../test/scala/spark/ZippedPartitionsSuite.scala | 3 +- 10 files changed, 373 insertions(+), 425 deletions(-) create mode 100644 core/src/test/scala/spark/PairRDDFunctionsSuite.scala create mode 100644 core/src/test/scala/spark/SharedSparkContext.scala create mode 100644 core/src/test/scala/spark/UnpersistSuite.scala diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala index ca385972fb..28a7b21b92 100644 --- a/core/src/test/scala/spark/CheckpointSuite.scala +++ b/core/src/test/scala/spark/CheckpointSuite.scala @@ -27,6 +27,16 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { } } + test("basic checkpointing") { + val parCollection = sc.makeRDD(1 to 4) + val flatMappedRDD = parCollection.flatMap(x => 1 to x) + flatMappedRDD.checkpoint() + assert(flatMappedRDD.dependencies.head.rdd == parCollection) + val result = flatMappedRDD.collect() + assert(flatMappedRDD.dependencies.head.rdd != parCollection) + assert(flatMappedRDD.collect() === result) + } + test("RDDs with one-to-one dependencies") { testCheckpointing(_.map(x => x.toString)) testCheckpointing(_.flatMap(x => 1 to x)) diff --git a/core/src/test/scala/spark/PairRDDFunctionsSuite.scala b/core/src/test/scala/spark/PairRDDFunctionsSuite.scala new file mode 100644 index 0000000000..682d2745bf --- /dev/null +++ b/core/src/test/scala/spark/PairRDDFunctionsSuite.scala @@ -0,0 +1,287 @@ +package spark + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashSet + +import org.scalatest.FunSuite +import org.scalatest.prop.Checkers +import org.scalacheck.Arbitrary._ +import org.scalacheck.Gen +import org.scalacheck.Prop._ + +import com.google.common.io.Files + +import spark.rdd.ShuffledRDD +import spark.SparkContext._ + +class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { + test("groupByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))) + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with duplicates") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with negative key hash codes") { + val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1))) + val groups = pairs.groupByKey().collect() + assert(groups.size === 2) + val valuesForMinus1 = groups.find(_._1 == -1).get._2 + assert(valuesForMinus1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("groupByKey with many output partitions") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))) + val groups = pairs.groupByKey(10).collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) + } + + test("reduceByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val sums = pairs.reduceByKey(_+_).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("reduceByKey with collectAsMap") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val sums = pairs.reduceByKey(_+_).collectAsMap() + assert(sums.size === 2) + assert(sums(1) === 7) + assert(sums(2) === 1) + } + + test("reduceByKey with many output partitons") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val sums = pairs.reduceByKey(_+_, 10).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("reduceByKey with partitioner") { + val p = new Partitioner() { + def numPartitions = 2 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).partitionBy(p) + val sums = pairs.reduceByKey(_+_) + assert(sums.collect().toSet === Set((1, 4), (0, 1))) + assert(sums.partitioner === Some(p)) + // count the dependencies to make sure there is only 1 ShuffledRDD + val deps = new HashSet[RDD[_]]() + def visit(r: RDD[_]) { + for (dep <- r.dependencies) { + deps += dep.rdd + visit(dep.rdd) + } + } + visit(sums) + assert(deps.size === 2) // ShuffledRDD, ParallelCollection + } + + test("join") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("join all-to-all") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))) + val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 6) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (1, 'y')), + (1, (2, 'x')), + (1, (2, 'y')), + (1, (3, 'x')), + (1, (3, 'y')) + )) + } + + test("leftOuterJoin") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (3, (1, None)) + )) + } + + test("rightOuterJoin") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.rightOuterJoin(rdd2).collect() + assert(joined.size === 5) + assert(joined.toSet === Set( + (1, (Some(1), 'x')), + (1, (Some(2), 'x')), + (2, (Some(1), 'y')), + (2, (Some(1), 'z')), + (4, (None, 'w')) + )) + } + + test("join with no matches") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))) + val joined = rdd1.join(rdd2).collect() + assert(joined.size === 0) + } + + test("join with many output partitions") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.join(rdd2, 10).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')) + )) + } + + test("groupWith") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.groupWith(rdd2).collect() + assert(joined.size === 4) + assert(joined.toSet === Set( + (1, (ArrayBuffer(1, 2), ArrayBuffer('x'))), + (2, (ArrayBuffer(1), ArrayBuffer('y', 'z'))), + (3, (ArrayBuffer(1), ArrayBuffer())), + (4, (ArrayBuffer(), ArrayBuffer('w'))) + )) + } + + test("zero-partition RDD") { + val emptyDir = Files.createTempDir() + val file = sc.textFile(emptyDir.getAbsolutePath) + assert(file.partitions.size == 0) + assert(file.collect().toList === Nil) + // Test that a shuffle on the file works, because this used to be a bug + assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) + } + + test("keys and values") { + val rdd = sc.parallelize(Array((1, "a"), (2, "b"))) + assert(rdd.keys.collect().toList === List(1, 2)) + assert(rdd.values.collect().toList === List("a", "b")) + } + + test("default partitioner uses partition size") { + // specify 2000 partitions + val a = sc.makeRDD(Array(1, 2, 3, 4), 2000) + // do a map, which loses the partitioner + val b = a.map(a => (a, (a * 2).toString)) + // then a group by, and see we didn't revert to 2 partitions + val c = b.groupByKey() + assert(c.partitions.size === 2000) + } + + test("default partitioner uses largest partitioner") { + val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2) + val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000) + val c = a.join(b) + assert(c.partitions.size === 2000) + } + + test("subtract") { + val a = sc.parallelize(Array(1, 2, 3), 2) + val b = sc.parallelize(Array(2, 3, 4), 4) + val c = a.subtract(b) + assert(c.collect().toSet === Set(1)) + assert(c.partitions.size === a.partitions.size) + } + + test("subtract with narrow dependency") { + // use a deterministic partitioner + val p = new Partitioner() { + def numPartitions = 5 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + // partitionBy so we have a narrow dependency + val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).partitionBy(p) + // more partitions/no partitioner so a shuffle dependency + val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) + val c = a.subtract(b) + assert(c.collect().toSet === Set((1, "a"), (3, "c"))) + // Ideally we could keep the original partitioner... + assert(c.partitioner === None) + } + + test("subtractByKey") { + val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2) + val b = sc.parallelize(Array((2, 20), (3, 30), (4, 40)), 4) + val c = a.subtractByKey(b) + assert(c.collect().toSet === Set((1, "a"), (1, "a"))) + assert(c.partitions.size === a.partitions.size) + } + + test("subtractByKey with narrow dependency") { + // use a deterministic partitioner + val p = new Partitioner() { + def numPartitions = 5 + def getPartition(key: Any) = key.asInstanceOf[Int] + } + // partitionBy so we have a narrow dependency + val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).partitionBy(p) + // more partitions/no partitioner so a shuffle dependency + val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) + val c = a.subtractByKey(b) + assert(c.collect().toSet === Set((1, "a"), (1, "a"))) + assert(c.partitioner.get === p) + } + + test("foldByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val sums = pairs.foldByKey(0)(_+_).collect() + assert(sums.toSet === Set((1, 7), (2, 1))) + } + + test("foldByKey with mutable result type") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) + val bufs = pairs.mapValues(v => ArrayBuffer(v)).cache() + // Fold the values using in-place mutation + val sums = bufs.foldByKey(new ArrayBuffer[Int])(_ ++= _).collect() + assert(sums.toSet === Set((1, ArrayBuffer(1, 2, 3, 1)), (2, ArrayBuffer(1)))) + // Check that the mutable objects in the original RDD were not changed + assert(bufs.collect().toSet === Set( + (1, ArrayBuffer(1)), + (1, ArrayBuffer(2)), + (1, ArrayBuffer(3)), + (1, ArrayBuffer(1)), + (2, ArrayBuffer(1)))) + } +} diff --git a/core/src/test/scala/spark/PartitioningSuite.scala b/core/src/test/scala/spark/PartitioningSuite.scala index 16f93e71a3..99e433e3bd 100644 --- a/core/src/test/scala/spark/PartitioningSuite.scala +++ b/core/src/test/scala/spark/PartitioningSuite.scala @@ -6,8 +6,8 @@ import SparkContext._ import spark.util.StatCounter import scala.math.abs -class PartitioningSuite extends FunSuite with LocalSparkContext { - +class PartitioningSuite extends FunSuite with SharedSparkContext { + test("HashPartitioner equality") { val p2 = new HashPartitioner(2) val p4 = new HashPartitioner(4) @@ -21,8 +21,6 @@ class PartitioningSuite extends FunSuite with LocalSparkContext { } test("RangePartitioner equality") { - sc = new SparkContext("local", "test") - // Make an RDD where all the elements are the same so that the partition range bounds // are deterministically all the same. val rdd = sc.parallelize(Seq(1, 1, 1, 1)).map(x => (x, x)) @@ -50,7 +48,6 @@ class PartitioningSuite extends FunSuite with LocalSparkContext { } test("HashPartitioner not equal to RangePartitioner") { - sc = new SparkContext("local", "test") val rdd = sc.parallelize(1 to 10).map(x => (x, x)) val rangeP2 = new RangePartitioner(2, rdd) val hashP2 = new HashPartitioner(2) @@ -61,8 +58,6 @@ class PartitioningSuite extends FunSuite with LocalSparkContext { } test("partitioner preservation") { - sc = new SparkContext("local", "test") - val rdd = sc.parallelize(1 to 10, 4).map(x => (x, x)) val grouped2 = rdd.groupByKey(2) @@ -101,7 +96,6 @@ class PartitioningSuite extends FunSuite with LocalSparkContext { } test("partitioning Java arrays should fail") { - sc = new SparkContext("local", "test") val arrs: RDD[Array[Int]] = sc.parallelize(Array(1, 2, 3, 4), 2).map(x => Array(x)) val arrPairs: RDD[(Array[Int], Int)] = sc.parallelize(Array(1, 2, 3, 4), 2).map(x => (Array(x), x)) @@ -120,21 +114,20 @@ class PartitioningSuite extends FunSuite with LocalSparkContext { assert(intercept[SparkException]{ arrPairs.reduceByKeyLocally(_ + _) }.getMessage.contains("array")) assert(intercept[SparkException]{ arrPairs.reduceByKey(_ + _) }.getMessage.contains("array")) } - - test("Zero-length partitions should be correctly handled") { + + test("zero-length partitions should be correctly handled") { // Create RDD with some consecutive empty partitions (including the "first" one) - sc = new SparkContext("local", "test") val rdd: RDD[Double] = sc .parallelize(Array(-1.0, -1.0, -1.0, -1.0, 2.0, 4.0, -1.0, -1.0), 8) .filter(_ >= 0.0) - + // Run the partitions, including the consecutive empty ones, through StatCounter val stats: StatCounter = rdd.stats(); assert(abs(6.0 - stats.sum) < 0.01); assert(abs(6.0/2 - rdd.mean) < 0.01); assert(abs(1.0 - rdd.variance) < 0.01); assert(abs(1.0 - rdd.stdev) < 0.01); - + // Add other tests here for classes that should be able to handle empty partitions correctly } } diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/spark/PipedRDDSuite.scala index ed075f93ec..1c9ca50811 100644 --- a/core/src/test/scala/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/spark/PipedRDDSuite.scala @@ -3,10 +3,9 @@ package spark import org.scalatest.FunSuite import SparkContext._ -class PipedRDDSuite extends FunSuite with LocalSparkContext { - +class PipedRDDSuite extends FunSuite with SharedSparkContext { + test("basic pipe") { - sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) val piped = nums.pipe(Seq("cat")) @@ -20,12 +19,11 @@ class PipedRDDSuite extends FunSuite with LocalSparkContext { } test("advanced pipe") { - sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) val bl = sc.broadcast(List("0")) - val piped = nums.pipe(Seq("cat"), - Map[String, String](), + val piped = nums.pipe(Seq("cat"), + Map[String, String](), (f: String => Unit) => {bl.value.map(f(_));f("\u0001")}, (i:Int, f: String=> Unit) => f(i + "_")) @@ -43,8 +41,8 @@ class PipedRDDSuite extends FunSuite with LocalSparkContext { val nums1 = sc.makeRDD(Array("a\t1", "b\t2", "a\t3", "b\t4"), 2) val d = nums1.groupBy(str=>str.split("\t")(0)). - pipe(Seq("cat"), - Map[String, String](), + pipe(Seq("cat"), + Map[String, String](), (f: String => Unit) => {bl.value.map(f(_));f("\u0001")}, (i:Tuple2[String, Seq[String]], f: String=> Unit) => {for (e <- i._2){ f(e + "_")}}).collect() assert(d.size === 8) @@ -59,7 +57,6 @@ class PipedRDDSuite extends FunSuite with LocalSparkContext { } test("pipe with env variable") { - sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) val piped = nums.pipe(Seq("printenv", "MY_TEST_ENV"), Map("MY_TEST_ENV" -> "LALALA")) val c = piped.collect() @@ -69,7 +66,6 @@ class PipedRDDSuite extends FunSuite with LocalSparkContext { } test("pipe with non-zero exit status") { - sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) val piped = nums.pipe("cat nonexistent_file") intercept[SparkException] { diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 67f3332d44..d8db69b1c9 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -7,10 +7,9 @@ import org.scalatest.time.{Span, Millis} import spark.SparkContext._ import spark.rdd.{CoalescedRDD, CoGroupedRDD, EmptyRDD, PartitionPruningRDD, ShuffledRDD} -class RDDSuite extends FunSuite with LocalSparkContext { +class RDDSuite extends FunSuite with SharedSparkContext { test("basic operations") { - sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) assert(nums.collect().toList === List(1, 2, 3, 4)) val dups = sc.makeRDD(Array(1, 1, 2, 2, 3, 3, 4, 4), 2) @@ -46,7 +45,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { } test("SparkContext.union") { - sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) assert(sc.union(nums).collect().toList === List(1, 2, 3, 4)) assert(sc.union(nums, nums).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4)) @@ -55,7 +53,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { } test("aggregate") { - sc = new SparkContext("local", "test") val pairs = sc.makeRDD(Array(("a", 1), ("b", 2), ("a", 2), ("c", 5), ("a", 3))) type StringMap = HashMap[String, Int] val emptyMap = new StringMap { @@ -75,57 +72,14 @@ class RDDSuite extends FunSuite with LocalSparkContext { assert(result.toSet === Set(("a", 6), ("b", 2), ("c", 5))) } - test("basic checkpointing") { - import java.io.File - val checkpointDir = File.createTempFile("temp", "") - checkpointDir.delete() - - sc = new SparkContext("local", "test") - sc.setCheckpointDir(checkpointDir.toString) - val parCollection = sc.makeRDD(1 to 4) - val flatMappedRDD = parCollection.flatMap(x => 1 to x) - flatMappedRDD.checkpoint() - assert(flatMappedRDD.dependencies.head.rdd == parCollection) - val result = flatMappedRDD.collect() - Thread.sleep(1000) - assert(flatMappedRDD.dependencies.head.rdd != parCollection) - assert(flatMappedRDD.collect() === result) - - checkpointDir.deleteOnExit() - } - test("basic caching") { - sc = new SparkContext("local", "test") val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() assert(rdd.collect().toList === List(1, 2, 3, 4)) assert(rdd.collect().toList === List(1, 2, 3, 4)) assert(rdd.collect().toList === List(1, 2, 3, 4)) } - test("unpersist RDD") { - sc = new SparkContext("local", "test") - val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() - rdd.count - assert(sc.persistentRdds.isEmpty === false) - rdd.unpersist() - assert(sc.persistentRdds.isEmpty === true) - - failAfter(Span(3000, Millis)) { - try { - while (! sc.getRDDStorageInfo.isEmpty) { - Thread.sleep(200) - } - } catch { - case _ => { Thread.sleep(10) } - // Do nothing. We might see exceptions because block manager - // is racing this thread to remove entries from the driver. - } - } - assert(sc.getRDDStorageInfo.isEmpty === true) - } - test("caching with failures") { - sc = new SparkContext("local", "test") val onlySplit = new Partition { override def index: Int = 0 } var shouldFail = true val rdd = new RDD[Int](sc, Nil) { @@ -148,7 +102,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { } test("empty RDD") { - sc = new SparkContext("local", "test") val empty = new EmptyRDD[Int](sc) assert(empty.count === 0) assert(empty.collect().size === 0) @@ -168,37 +121,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { } test("cogrouped RDDs") { - sc = new SparkContext("local", "test") - val rdd1 = sc.makeRDD(Array((1, "one"), (1, "another one"), (2, "two"), (3, "three")), 2) - val rdd2 = sc.makeRDD(Array((1, "one1"), (1, "another one1"), (2, "two1")), 2) - - // Use cogroup function - val cogrouped = rdd1.cogroup(rdd2).collectAsMap() - assert(cogrouped(1) === (Seq("one", "another one"), Seq("one1", "another one1"))) - assert(cogrouped(2) === (Seq("two"), Seq("two1"))) - assert(cogrouped(3) === (Seq("three"), Seq())) - - // Construct CoGroupedRDD directly, with map side combine enabled - val cogrouped1 = new CoGroupedRDD[Int]( - Seq(rdd1.asInstanceOf[RDD[(Int, Any)]], rdd2.asInstanceOf[RDD[(Int, Any)]]), - new HashPartitioner(3), - true).collectAsMap() - assert(cogrouped1(1).toSeq === Seq(Seq("one", "another one"), Seq("one1", "another one1"))) - assert(cogrouped1(2).toSeq === Seq(Seq("two"), Seq("two1"))) - assert(cogrouped1(3).toSeq === Seq(Seq("three"), Seq())) - - // Construct CoGroupedRDD directly, with map side combine disabled - val cogrouped2 = new CoGroupedRDD[Int]( - Seq(rdd1.asInstanceOf[RDD[(Int, Any)]], rdd2.asInstanceOf[RDD[(Int, Any)]]), - new HashPartitioner(3), - false).collectAsMap() - assert(cogrouped2(1).toSeq === Seq(Seq("one", "another one"), Seq("one1", "another one1"))) - assert(cogrouped2(2).toSeq === Seq(Seq("two"), Seq("two1"))) - assert(cogrouped2(3).toSeq === Seq(Seq("three"), Seq())) - } - - test("coalesced RDDs") { - sc = new SparkContext("local", "test") val data = sc.parallelize(1 to 10, 10) val coalesced1 = data.coalesce(2) @@ -236,7 +158,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { } test("zipped RDDs") { - sc = new SparkContext("local", "test") val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) val zipped = nums.zip(nums.map(_ + 1.0)) assert(zipped.glom().map(_.toList).collect().toList === @@ -248,7 +169,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { } test("partition pruning") { - sc = new SparkContext("local", "test") val data = sc.parallelize(1 to 10, 10) // Note that split number starts from 0, so > 8 means only 10th partition left. val prunedRdd = new PartitionPruningRDD(data, splitNum => splitNum > 8) @@ -260,7 +180,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { test("mapWith") { import java.util.Random - sc = new SparkContext("local", "test") val ones = sc.makeRDD(Array(1, 1, 1, 1, 1, 1), 2) val randoms = ones.mapWith( (index: Int) => new Random(index + 42)) @@ -279,7 +198,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { test("flatMapWith") { import java.util.Random - sc = new SparkContext("local", "test") val ones = sc.makeRDD(Array(1, 1, 1, 1, 1, 1), 2) val randoms = ones.flatMapWith( (index: Int) => new Random(index + 42)) @@ -301,7 +219,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { test("filterWith") { import java.util.Random - sc = new SparkContext("local", "test") val ints = sc.makeRDD(Array(1, 2, 3, 4, 5, 6), 2) val sample = ints.filterWith( (index: Int) => new Random(index + 42)) @@ -319,7 +236,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { } test("top with predefined ordering") { - sc = new SparkContext("local", "test") val nums = Array.range(1, 100000) val ints = sc.makeRDD(scala.util.Random.shuffle(nums), 2) val topK = ints.top(5) @@ -328,7 +244,6 @@ class RDDSuite extends FunSuite with LocalSparkContext { } test("top with custom ordering") { - sc = new SparkContext("local", "test") val words = Vector("a", "b", "c", "d") implicit val ord = implicitly[Ordering[String]].reverse val rdd = sc.makeRDD(words, 2) diff --git a/core/src/test/scala/spark/SharedSparkContext.scala b/core/src/test/scala/spark/SharedSparkContext.scala new file mode 100644 index 0000000000..1da79f9824 --- /dev/null +++ b/core/src/test/scala/spark/SharedSparkContext.scala @@ -0,0 +1,25 @@ +package spark + +import org.scalatest.Suite +import org.scalatest.BeforeAndAfterAll + +/** Shares a local `SparkContext` between all tests in a suite and closes it at the end */ +trait SharedSparkContext extends BeforeAndAfterAll { self: Suite => + + @transient private var _sc: SparkContext = _ + + def sc: SparkContext = _sc + + override def beforeAll() { + _sc = new SparkContext("local", "test") + super.beforeAll() + } + + override def afterAll() { + if (_sc != null) { + LocalSparkContext.stop(_sc) + _sc = null + } + super.afterAll() + } +} diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 0c1ec29f96..950218fa28 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -16,54 +16,9 @@ import spark.rdd.ShuffledRDD import spark.SparkContext._ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { - - test("groupByKey") { - sc = new SparkContext("local", "test") - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))) - val groups = pairs.groupByKey().collect() - assert(groups.size === 2) - val valuesFor1 = groups.find(_._1 == 1).get._2 - assert(valuesFor1.toList.sorted === List(1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - - test("groupByKey with duplicates") { - sc = new SparkContext("local", "test") - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val groups = pairs.groupByKey().collect() - assert(groups.size === 2) - val valuesFor1 = groups.find(_._1 == 1).get._2 - assert(valuesFor1.toList.sorted === List(1, 1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - - test("groupByKey with negative key hash codes") { - sc = new SparkContext("local", "test") - val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1))) - val groups = pairs.groupByKey().collect() - assert(groups.size === 2) - val valuesForMinus1 = groups.find(_._1 == -1).get._2 - assert(valuesForMinus1.toList.sorted === List(1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - - test("groupByKey with many output partitions") { - sc = new SparkContext("local", "test") - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))) - val groups = pairs.groupByKey(10).collect() - assert(groups.size === 2) - val valuesFor1 = groups.find(_._1 == 1).get._2 - assert(valuesFor1.toList.sorted === List(1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } - test("groupByKey with compression") { try { - System.setProperty("spark.blockManager.compress", "true") + System.setProperty("spark.shuffle.compress", "true") sc = new SparkContext("local", "test") val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4) val groups = pairs.groupByKey(4).collect() @@ -77,234 +32,6 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { } } - test("reduceByKey") { - sc = new SparkContext("local", "test") - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.reduceByKey(_+_).collect() - assert(sums.toSet === Set((1, 7), (2, 1))) - } - - test("reduceByKey with collectAsMap") { - sc = new SparkContext("local", "test") - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.reduceByKey(_+_).collectAsMap() - assert(sums.size === 2) - assert(sums(1) === 7) - assert(sums(2) === 1) - } - - test("reduceByKey with many output partitons") { - sc = new SparkContext("local", "test") - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.reduceByKey(_+_, 10).collect() - assert(sums.toSet === Set((1, 7), (2, 1))) - } - - test("reduceByKey with partitioner") { - sc = new SparkContext("local", "test") - val p = new Partitioner() { - def numPartitions = 2 - def getPartition(key: Any) = key.asInstanceOf[Int] - } - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).partitionBy(p) - val sums = pairs.reduceByKey(_+_) - assert(sums.collect().toSet === Set((1, 4), (0, 1))) - assert(sums.partitioner === Some(p)) - // count the dependencies to make sure there is only 1 ShuffledRDD - val deps = new HashSet[RDD[_]]() - def visit(r: RDD[_]) { - for (dep <- r.dependencies) { - deps += dep.rdd - visit(dep.rdd) - } - } - visit(sums) - assert(deps.size === 2) // ShuffledRDD, ParallelCollection - } - - test("join") { - sc = new SparkContext("local", "test") - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (2, 'x')), - (2, (1, 'y')), - (2, (1, 'z')) - )) - } - - test("join all-to-all") { - sc = new SparkContext("local", "test") - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))) - val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 6) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (1, 'y')), - (1, (2, 'x')), - (1, (2, 'y')), - (1, (3, 'x')), - (1, (3, 'y')) - )) - } - - test("leftOuterJoin") { - sc = new SparkContext("local", "test") - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.leftOuterJoin(rdd2).collect() - assert(joined.size === 5) - assert(joined.toSet === Set( - (1, (1, Some('x'))), - (1, (2, Some('x'))), - (2, (1, Some('y'))), - (2, (1, Some('z'))), - (3, (1, None)) - )) - } - - test("rightOuterJoin") { - sc = new SparkContext("local", "test") - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.rightOuterJoin(rdd2).collect() - assert(joined.size === 5) - assert(joined.toSet === Set( - (1, (Some(1), 'x')), - (1, (Some(2), 'x')), - (2, (Some(1), 'y')), - (2, (Some(1), 'z')), - (4, (None, 'w')) - )) - } - - test("join with no matches") { - sc = new SparkContext("local", "test") - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))) - val joined = rdd1.join(rdd2).collect() - assert(joined.size === 0) - } - - test("join with many output partitions") { - sc = new SparkContext("local", "test") - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.join(rdd2, 10).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (1, 'x')), - (1, (2, 'x')), - (2, (1, 'y')), - (2, (1, 'z')) - )) - } - - test("groupWith") { - sc = new SparkContext("local", "test") - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.groupWith(rdd2).collect() - assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (ArrayBuffer(1, 2), ArrayBuffer('x'))), - (2, (ArrayBuffer(1), ArrayBuffer('y', 'z'))), - (3, (ArrayBuffer(1), ArrayBuffer())), - (4, (ArrayBuffer(), ArrayBuffer('w'))) - )) - } - - test("zero-partition RDD") { - sc = new SparkContext("local", "test") - val emptyDir = Files.createTempDir() - val file = sc.textFile(emptyDir.getAbsolutePath) - assert(file.partitions.size == 0) - assert(file.collect().toList === Nil) - // Test that a shuffle on the file works, because this used to be a bug - assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) - } - - test("keys and values") { - sc = new SparkContext("local", "test") - val rdd = sc.parallelize(Array((1, "a"), (2, "b"))) - assert(rdd.keys.collect().toList === List(1, 2)) - assert(rdd.values.collect().toList === List("a", "b")) - } - - test("default partitioner uses partition size") { - sc = new SparkContext("local", "test") - // specify 2000 partitions - val a = sc.makeRDD(Array(1, 2, 3, 4), 2000) - // do a map, which loses the partitioner - val b = a.map(a => (a, (a * 2).toString)) - // then a group by, and see we didn't revert to 2 partitions - val c = b.groupByKey() - assert(c.partitions.size === 2000) - } - - test("default partitioner uses largest partitioner") { - sc = new SparkContext("local", "test") - val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2) - val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000) - val c = a.join(b) - assert(c.partitions.size === 2000) - } - - test("subtract") { - sc = new SparkContext("local", "test") - val a = sc.parallelize(Array(1, 2, 3), 2) - val b = sc.parallelize(Array(2, 3, 4), 4) - val c = a.subtract(b) - assert(c.collect().toSet === Set(1)) - assert(c.partitions.size === a.partitions.size) - } - - test("subtract with narrow dependency") { - sc = new SparkContext("local", "test") - // use a deterministic partitioner - val p = new Partitioner() { - def numPartitions = 5 - def getPartition(key: Any) = key.asInstanceOf[Int] - } - // partitionBy so we have a narrow dependency - val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).partitionBy(p) - // more partitions/no partitioner so a shuffle dependency - val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) - val c = a.subtract(b) - assert(c.collect().toSet === Set((1, "a"), (3, "c"))) - // Ideally we could keep the original partitioner... - assert(c.partitioner === None) - } - - test("subtractByKey") { - sc = new SparkContext("local", "test") - val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2) - val b = sc.parallelize(Array((2, 20), (3, 30), (4, 40)), 4) - val c = a.subtractByKey(b) - assert(c.collect().toSet === Set((1, "a"), (1, "a"))) - assert(c.partitions.size === a.partitions.size) - } - - test("subtractByKey with narrow dependency") { - sc = new SparkContext("local", "test") - // use a deterministic partitioner - val p = new Partitioner() { - def numPartitions = 5 - def getPartition(key: Any) = key.asInstanceOf[Int] - } - // partitionBy so we have a narrow dependency - val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).partitionBy(p) - // more partitions/no partitioner so a shuffle dependency - val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4) - val c = a.subtractByKey(b) - assert(c.collect().toSet === Set((1, "a"), (1, "a"))) - assert(c.partitioner.get === p) - } - test("shuffle non-zero block size") { sc = new SparkContext("local-cluster[2,1,512]", "test") val NUM_BLOCKS = 3 @@ -391,29 +118,6 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { // We should have at most 4 non-zero sized partitions assert(nonEmptyBlocks.size <= 4) } - - test("foldByKey") { - sc = new SparkContext("local", "test") - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.foldByKey(0)(_+_).collect() - assert(sums.toSet === Set((1, 7), (2, 1))) - } - - test("foldByKey with mutable result type") { - sc = new SparkContext("local", "test") - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val bufs = pairs.mapValues(v => ArrayBuffer(v)).cache() - // Fold the values using in-place mutation - val sums = bufs.foldByKey(new ArrayBuffer[Int])(_ ++= _).collect() - assert(sums.toSet === Set((1, ArrayBuffer(1, 2, 3, 1)), (2, ArrayBuffer(1)))) - // Check that the mutable objects in the original RDD were not changed - assert(bufs.collect().toSet === Set( - (1, ArrayBuffer(1)), - (1, ArrayBuffer(2)), - (1, ArrayBuffer(3)), - (1, ArrayBuffer(1)), - (2, ArrayBuffer(1)))) - } } object ShuffleSuite { diff --git a/core/src/test/scala/spark/SortingSuite.scala b/core/src/test/scala/spark/SortingSuite.scala index 495f957e53..f7bf207c68 100644 --- a/core/src/test/scala/spark/SortingSuite.scala +++ b/core/src/test/scala/spark/SortingSuite.scala @@ -5,16 +5,14 @@ import org.scalatest.BeforeAndAfter import org.scalatest.matchers.ShouldMatchers import SparkContext._ -class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers with Logging { - +class SortingSuite extends FunSuite with SharedSparkContext with ShouldMatchers with Logging { + test("sortByKey") { - sc = new SparkContext("local", "test") val pairs = sc.parallelize(Array((1, 0), (2, 0), (0, 0), (3, 0)), 2) - assert(pairs.sortByKey().collect() === Array((0,0), (1,0), (2,0), (3,0))) + assert(pairs.sortByKey().collect() === Array((0,0), (1,0), (2,0), (3,0))) } test("large array") { - sc = new SparkContext("local", "test") val rand = new scala.util.Random() val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } val pairs = sc.parallelize(pairArr, 2) @@ -24,7 +22,6 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w } test("large array with one split") { - sc = new SparkContext("local", "test") val rand = new scala.util.Random() val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } val pairs = sc.parallelize(pairArr, 2) @@ -32,9 +29,8 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w assert(sorted.partitions.size === 1) assert(sorted.collect() === pairArr.sortBy(_._1)) } - + test("large array with many partitions") { - sc = new SparkContext("local", "test") val rand = new scala.util.Random() val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } val pairs = sc.parallelize(pairArr, 2) @@ -42,9 +38,8 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w assert(sorted.partitions.size === 20) assert(sorted.collect() === pairArr.sortBy(_._1)) } - + test("sort descending") { - sc = new SparkContext("local", "test") val rand = new scala.util.Random() val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } val pairs = sc.parallelize(pairArr, 2) @@ -52,15 +47,13 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w } test("sort descending with one split") { - sc = new SparkContext("local", "test") val rand = new scala.util.Random() val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } val pairs = sc.parallelize(pairArr, 1) assert(pairs.sortByKey(false, 1).collect() === pairArr.sortWith((x, y) => x._1 > y._1)) } - + test("sort descending with many partitions") { - sc = new SparkContext("local", "test") val rand = new scala.util.Random() val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) } val pairs = sc.parallelize(pairArr, 2) @@ -68,7 +61,6 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w } test("more partitions than elements") { - sc = new SparkContext("local", "test") val rand = new scala.util.Random() val pairArr = Array.fill(10) { (rand.nextInt(), rand.nextInt()) } val pairs = sc.parallelize(pairArr, 30) @@ -76,14 +68,12 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w } test("empty RDD") { - sc = new SparkContext("local", "test") val pairArr = new Array[(Int, Int)](0) val pairs = sc.parallelize(pairArr, 2) assert(pairs.sortByKey().collect() === pairArr.sortBy(_._1)) } test("partition balancing") { - sc = new SparkContext("local", "test") val pairArr = (1 to 1000).map(x => (x, x)).toArray val sorted = sc.parallelize(pairArr, 4).sortByKey() assert(sorted.collect() === pairArr.sortBy(_._1)) @@ -99,7 +89,6 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w } test("partition balancing for descending sort") { - sc = new SparkContext("local", "test") val pairArr = (1 to 1000).map(x => (x, x)).toArray val sorted = sc.parallelize(pairArr, 4).sortByKey(false) assert(sorted.collect() === pairArr.sortBy(_._1).reverse) diff --git a/core/src/test/scala/spark/UnpersistSuite.scala b/core/src/test/scala/spark/UnpersistSuite.scala new file mode 100644 index 0000000000..94776e7572 --- /dev/null +++ b/core/src/test/scala/spark/UnpersistSuite.scala @@ -0,0 +1,30 @@ +package spark + +import org.scalatest.FunSuite +import org.scalatest.concurrent.Timeouts._ +import org.scalatest.time.{Span, Millis} +import spark.SparkContext._ + +class UnpersistSuite extends FunSuite with LocalSparkContext { + test("unpersist RDD") { + sc = new SparkContext("local", "test") + val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() + rdd.count + assert(sc.persistentRdds.isEmpty === false) + rdd.unpersist() + assert(sc.persistentRdds.isEmpty === true) + + failAfter(Span(3000, Millis)) { + try { + while (! sc.getRDDStorageInfo.isEmpty) { + Thread.sleep(200) + } + } catch { + case _ => { Thread.sleep(10) } + // Do nothing. We might see exceptions because block manager + // is racing this thread to remove entries from the driver. + } + } + assert(sc.getRDDStorageInfo.isEmpty === true) + } +} diff --git a/core/src/test/scala/spark/ZippedPartitionsSuite.scala b/core/src/test/scala/spark/ZippedPartitionsSuite.scala index 5f60aa75d7..96cb295f45 100644 --- a/core/src/test/scala/spark/ZippedPartitionsSuite.scala +++ b/core/src/test/scala/spark/ZippedPartitionsSuite.scala @@ -17,9 +17,8 @@ object ZippedPartitionsSuite { } } -class ZippedPartitionsSuite extends FunSuite with LocalSparkContext { +class ZippedPartitionsSuite extends FunSuite with SharedSparkContext { test("print sizes") { - sc = new SparkContext("local", "test") val data1 = sc.makeRDD(Array(1, 2, 3, 4), 2) val data2 = sc.makeRDD(Array("1", "2", "3", "4", "5", "6"), 2) val data3 = sc.makeRDD(Array(1.0, 2.0), 2) -- cgit v1.2.3 From 32370da4e40062b88c921417cd7418d804e16f23 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 22:08:19 -0400 Subject: Don't use forward slash in exclusion for JAR signature files --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 484f97d992..07572201de 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -236,7 +236,7 @@ object SparkBuild extends Build { def extraAssemblySettings() = Seq(test in assembly := {}) ++ Seq( mergeStrategy in assembly := { case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard - case m if m.toLowerCase.matches("meta-inf/.*\\.sf$") => MergeStrategy.discard + case m if m.toLowerCase.matches("meta-inf.*\\.sf$") => MergeStrategy.discard case "reference.conf" => MergeStrategy.concat case _ => MergeStrategy.first } -- cgit v1.2.3 From d11025dc6aedd9763cfd2390e8daf24747d17258 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Wed, 26 Jun 2013 09:53:35 -0500 Subject: Be cute with Option and getenv. --- .../scala/spark/deploy/worker/ExecutorRunner.scala | 21 ++++----------------- 1 file changed, 4 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 4f8e1dcb26..a9b12421e6 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -1,6 +1,7 @@ package spark.deploy.worker import java.io._ +import java.lang.System.getenv import spark.deploy.{ExecutorState, ExecutorStateChanged, ApplicationDescription} import akka.actor.ActorRef import spark.{Utils, Logging} @@ -77,11 +78,7 @@ private[spark] class ExecutorRunner( def buildCommandSeq(): Seq[String] = { val command = appDesc.command - val runner = if (System.getenv("JAVA_HOME") == null) { - "java" - } else { - System.getenv("JAVA_HOME") + "/bin/java" - } + val runner = Option(getenv("JAVA_HOME")).map(_ + "/bin/java").getOrElse("java") // SPARK-698: do not call the run.cmd script, as process.destroy() // fails to kill a process tree on Windows Seq(runner) ++ buildJavaOpts() ++ Seq(command.mainClass) ++ @@ -93,18 +90,8 @@ private[spark] class ExecutorRunner( * the way the JAVA_OPTS are assembled there. */ def buildJavaOpts(): Seq[String] = { - val libraryOpts = if (System.getenv("SPARK_LIBRARY_PATH") == null) { - Nil - } else { - List("-Djava.library.path=" + System.getenv("SPARK_LIBRARY_PATH")) - } - - val userOpts = if (System.getenv("SPARK_JAVA_OPTS") == null) { - Nil - } else { - Utils.splitCommandString(System.getenv("SPARK_JAVA_OPTS")) - } - + val libraryOpts = Option(getenv("SPARK_LIBRARY_PATH")).map(p => List("-Djava.library.path=" + p)).getOrElse(Nil) + val userOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString).getOrElse(Nil) val memoryOpts = Seq("-Xms" + memory + "M", "-Xmx" + memory + "M") // Figure out our classpath with the external compute-classpath script -- cgit v1.2.3 From 2cbaa0734b1a23e5d6ce1543a22358f436b5b1ef Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 26 Jun 2013 08:44:55 -0700 Subject: Making all new classes package private --- core/src/main/scala/spark/deploy/JsonProtocol.scala | 2 +- core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala | 2 +- core/src/main/scala/spark/deploy/master/ui/IndexPage.scala | 2 +- core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala | 2 +- core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala | 2 +- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- core/src/main/scala/spark/ui/JettyUtils.scala | 2 +- core/src/main/scala/spark/ui/SparkUI.scala | 2 +- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 2 +- core/src/main/scala/spark/ui/jobs/StagePage.scala | 2 +- core/src/main/scala/spark/ui/storage/BlockManagerUI.scala | 3 +-- core/src/main/scala/spark/ui/storage/IndexPage.scala | 2 +- core/src/test/scala/spark/ui/UISuite.scala | 2 +- 13 files changed, 13 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index b4365d31e9..335aaeb769 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -4,7 +4,7 @@ import master.{ApplicationInfo, WorkerInfo} import net.liftweb.json.JsonDSL._ import worker.ExecutorRunner -object JsonProtocol { +private[spark] object JsonProtocol { def writeWorkerInfo(obj: WorkerInfo) = { ("id" -> obj.id) ~ ("host" -> obj.host) ~ diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index 2a810b71d2..9cb9c94e9f 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -14,7 +14,7 @@ import spark.deploy.{RequestMasterState, JsonProtocol, MasterState} import spark.deploy.master.ExecutorInfo import spark.ui.UIUtils -class ApplicationPage(parent: MasterWebUI) { +private[spark] class ApplicationPage(parent: MasterWebUI) { val master = parent.master implicit val timeout = parent.timeout diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index f833c59de8..3c3679a4b0 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -13,7 +13,7 @@ import spark.Utils import spark.ui.UIUtils import spark.deploy.master.{ApplicationInfo, WorkerInfo} -class IndexPage(parent: MasterWebUI) { +private[spark] class IndexPage(parent: MasterWebUI) { val master = parent.master implicit val timeout = parent.timeout 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 dfefd6571b..0cfe600043 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -53,7 +53,7 @@ class MasterWebUI(val master: ActorRef, requestedPort: Option[Int] = None) exten } } -object MasterWebUI { +private[spark] object MasterWebUI { val STATIC_RESOURCE_DIR = "spark/webui/static" val DEFAULT_PORT = "8080" } \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index 9e7b2bfd92..c1ca09fc97 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -15,7 +15,7 @@ import spark.deploy.worker.ExecutorRunner import spark.Utils import spark.ui.UIUtils -class IndexPage(parent: WorkerWebUI) { +private[spark] class IndexPage(parent: WorkerWebUI) { val worker = parent.worker val timeout = parent.timeout diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 2615458dee..65bdba4735 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -68,7 +68,7 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option } } -object WorkerWebUI { +private[spark] object WorkerWebUI { val STATIC_RESOURCE_DIR = "spark/webui/static" val DEFAULT_PORT="8081" } diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 45c0805b22..1f311bb75f 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -108,7 +108,7 @@ private[spark] object JettyUtils extends Logging { } /** Utility functions for generating XML pages with spark content. */ -object UIUtils { +private[spark] object UIUtils { /** Returns a page containing the supplied content and the spark web ui headers */ def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String): Seq[Node] = { diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 487f005615..86b394c09f 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -57,7 +57,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { private[spark] def appUIAddress = "http://" + host + ":" + boundPort.getOrElse("-1") } -object SparkUI { +private[spark] object SparkUI { val DEFAULT_PORT = "33000" val STATIC_RESOURCE_DIR = "spark/webui/static" } diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index c7ee9dc1c1..152e1a79b0 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -14,7 +14,7 @@ import spark.ui.UIUtils._ import spark.storage.StorageLevel /** Page showing list of all ongoing and recently finished stages */ -class IndexPage(parent: JobProgressUI) { +private[spark] class IndexPage(parent: JobProgressUI) { def listener = parent.listener val dateFmt = parent.dateFmt diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 65dbd389b1..03200861e0 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -13,7 +13,7 @@ import spark.scheduler.cluster.TaskInfo import spark.executor.TaskMetrics /** Page showing statistics and task list for a given stage */ -class StagePage(parent: JobProgressUI) { +private[spark] class StagePage(parent: JobProgressUI) { def listener = parent.listener val dateFmt = parent.dateFmt diff --git a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala index 4f1928fac9..3d67888e2c 100644 --- a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala @@ -10,8 +10,7 @@ import spark.{Logging, SparkContext} import spark.ui.JettyUtils._ /** Web UI showing storage status of all RDD's in the given SparkContext. */ -private[spark] -class BlockManagerUI(val sc: SparkContext) extends Logging { +private[spark] class BlockManagerUI(val sc: SparkContext) extends Logging { implicit val timeout = Duration.create( System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index cd7f6bd9f2..772c669a1a 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -9,7 +9,7 @@ import spark.Utils import spark.ui.UIUtils._ /** Page showing list of RDD's currently stored in the cluster */ -class IndexPage(parent: BlockManagerUI) { +private[spark] class IndexPage(parent: BlockManagerUI) { val sc = parent.sc def render(request: HttpServletRequest): Seq[Node] = { diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index 7df7356a6e..6519bf77c6 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -5,7 +5,7 @@ import org.eclipse.jetty.server.Server import util.{Try, Success, Failure} import java.net.ServerSocket -class UISuite extends FunSuite { +private[spark] class UISuite extends FunSuite { test("jetty port increases under contention") { val startPort = 33333 val server = new Server(startPort) -- cgit v1.2.3 From b14ad509ba4c00d57bd4a52a7b70a18325064a7b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 26 Jun 2013 08:46:51 -0700 Subject: Moving static ui package --- .../spark/ui/static/bootstrap-responsive.min.css | 9 + .../resources/spark/ui/static/bootstrap.min.css | 9 + .../main/resources/spark/ui/static/sorttable.js | 495 +++++++++++++++++++++ .../main/resources/spark/ui/static/spark_logo.png | Bin 0 -> 14233 bytes .../webui/static/bootstrap-responsive.min.css | 9 - .../resources/spark/webui/static/bootstrap.min.css | 9 - .../main/resources/spark/webui/static/sorttable.js | 495 --------------------- .../resources/spark/webui/static/spark_logo.png | Bin 14233 -> 0 bytes .../scala/spark/deploy/master/ui/MasterWebUI.scala | 2 +- .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- core/src/main/scala/spark/ui/SparkUI.scala | 2 +- 11 files changed, 516 insertions(+), 516 deletions(-) create mode 100644 core/src/main/resources/spark/ui/static/bootstrap-responsive.min.css create mode 100644 core/src/main/resources/spark/ui/static/bootstrap.min.css create mode 100644 core/src/main/resources/spark/ui/static/sorttable.js create mode 100644 core/src/main/resources/spark/ui/static/spark_logo.png delete mode 100644 core/src/main/resources/spark/webui/static/bootstrap-responsive.min.css delete mode 100644 core/src/main/resources/spark/webui/static/bootstrap.min.css delete mode 100644 core/src/main/resources/spark/webui/static/sorttable.js delete mode 100644 core/src/main/resources/spark/webui/static/spark_logo.png diff --git a/core/src/main/resources/spark/ui/static/bootstrap-responsive.min.css b/core/src/main/resources/spark/ui/static/bootstrap-responsive.min.css new file mode 100644 index 0000000000..1f55036a07 --- /dev/null +++ b/core/src/main/resources/spark/ui/static/bootstrap-responsive.min.css @@ -0,0 +1,9 @@ +/*! + * Bootstrap Responsive v2.0.4 + * + * Copyright 2012 Twitter, Inc + * Licensed under the Apache License v2.0 + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Designed and built with all the love in the world @twitter by @mdo and @fat. + */.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:28px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box}.hidden{display:none;visibility:hidden}.visible-phone{display:none!important}.visible-tablet{display:none!important}.hidden-desktop{display:none!important}@media(max-width:767px){.visible-phone{display:inherit!important}.hidden-phone{display:none!important}.hidden-desktop{display:inherit!important}.visible-desktop{display:none!important}}@media(min-width:768px) and (max-width:979px){.visible-tablet{display:inherit!important}.hidden-tablet{display:none!important}.hidden-desktop{display:inherit!important}.visible-desktop{display:none!important}}@media(max-width:480px){.nav-collapse{-webkit-transform:translate3d(0,0,0)}.page-header h1 small{display:block;line-height:18px}input[type="checkbox"],input[type="radio"]{border:1px solid #ccc}.form-horizontal .control-group>label{float:none;width:auto;padding-top:0;text-align:left}.form-horizontal .controls{margin-left:0}.form-horizontal .control-list{padding-top:0}.form-horizontal .form-actions{padding-right:10px;padding-left:10px}.modal{position:absolute;top:10px;right:10px;left:10px;width:auto;margin:0}.modal.fade.in{top:auto}.modal-header .close{padding:10px;margin:-10px}.carousel-caption{position:static}}@media(max-width:767px){body{padding-right:20px;padding-left:20px}.navbar-fixed-top,.navbar-fixed-bottom{margin-right:-20px;margin-left:-20px}.container-fluid{padding:0}.dl-horizontal dt{float:none;width:auto;clear:none;text-align:left}.dl-horizontal dd{margin-left:0}.container{width:auto}.row-fluid{width:100%}.row,.thumbnails{margin-left:0}[class*="span"],.row-fluid [class*="span"]{display:block;float:none;width:auto;margin-left:0}.input-large,.input-xlarge,.input-xxlarge,input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input{display:block;width:100%;min-height:28px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box}.input-prepend input,.input-append input,.input-prepend input[class*="span"],.input-append input[class*="span"]{display:inline-block;width:auto}}@media(min-width:768px) and (max-width:979px){.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:724px}.span12{width:724px}.span11{width:662px}.span10{width:600px}.span9{width:538px}.span8{width:476px}.span7{width:414px}.span6{width:352px}.span5{width:290px}.span4{width:228px}.span3{width:166px}.span2{width:104px}.span1{width:42px}.offset12{margin-left:764px}.offset11{margin-left:702px}.offset10{margin-left:640px}.offset9{margin-left:578px}.offset8{margin-left:516px}.offset7{margin-left:454px}.offset6{margin-left:392px}.offset5{margin-left:330px}.offset4{margin-left:268px}.offset3{margin-left:206px}.offset2{margin-left:144px}.offset1{margin-left:82px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:28px;margin-left:2.762430939%;*margin-left:2.709239449638298%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:99.999999993%;*width:99.9468085036383%}.row-fluid .span11{width:91.436464082%;*width:91.38327259263829%}.row-fluid .span10{width:82.87292817100001%;*width:82.8197366816383%}.row-fluid .span9{width:74.30939226%;*width:74.25620077063829%}.row-fluid .span8{width:65.74585634900001%;*width:65.6926648596383%}.row-fluid .span7{width:57.182320438000005%;*width:57.129128948638304%}.row-fluid .span6{width:48.618784527%;*width:48.5655930376383%}.row-fluid .span5{width:40.055248616%;*width:40.0020571266383%}.row-fluid .span4{width:31.491712705%;*width:31.4385212156383%}.row-fluid .span3{width:22.928176794%;*width:22.874985304638297%}.row-fluid .span2{width:14.364640883%;*width:14.311449393638298%}.row-fluid .span1{width:5.801104972%;*width:5.747913482638298%}input,textarea,.uneditable-input{margin-left:0}input.span12,textarea.span12,.uneditable-input.span12{width:714px}input.span11,textarea.span11,.uneditable-input.span11{width:652px}input.span10,textarea.span10,.uneditable-input.span10{width:590px}input.span9,textarea.span9,.uneditable-input.span9{width:528px}input.span8,textarea.span8,.uneditable-input.span8{width:466px}input.span7,textarea.span7,.uneditable-input.span7{width:404px}input.span6,textarea.span6,.uneditable-input.span6{width:342px}input.span5,textarea.span5,.uneditable-input.span5{width:280px}input.span4,textarea.span4,.uneditable-input.span4{width:218px}input.span3,textarea.span3,.uneditable-input.span3{width:156px}input.span2,textarea.span2,.uneditable-input.span2{width:94px}input.span1,textarea.span1,.uneditable-input.span1{width:32px}}@media(min-width:1200px){.row{margin-left:-30px;*zoom:1}.row:before,.row:after{display:table;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:30px}.container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:1170px}.span12{width:1170px}.span11{width:1070px}.span10{width:970px}.span9{width:870px}.span8{width:770px}.span7{width:670px}.span6{width:570px}.span5{width:470px}.span4{width:370px}.span3{width:270px}.span2{width:170px}.span1{width:70px}.offset12{margin-left:1230px}.offset11{margin-left:1130px}.offset10{margin-left:1030px}.offset9{margin-left:930px}.offset8{margin-left:830px}.offset7{margin-left:730px}.offset6{margin-left:630px}.offset5{margin-left:530px}.offset4{margin-left:430px}.offset3{margin-left:330px}.offset2{margin-left:230px}.offset1{margin-left:130px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:28px;margin-left:2.564102564%;*margin-left:2.510911074638298%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.45299145300001%;*width:91.3997999636383%}.row-fluid .span10{width:82.905982906%;*width:82.8527914166383%}.row-fluid .span9{width:74.358974359%;*width:74.30578286963829%}.row-fluid .span8{width:65.81196581200001%;*width:65.7587743226383%}.row-fluid .span7{width:57.264957265%;*width:57.2117657756383%}.row-fluid .span6{width:48.717948718%;*width:48.6647572286383%}.row-fluid .span5{width:40.170940171000005%;*width:40.117748681638304%}.row-fluid .span4{width:31.623931624%;*width:31.5707401346383%}.row-fluid .span3{width:23.076923077%;*width:23.0237315876383%}.row-fluid .span2{width:14.529914530000001%;*width:14.4767230406383%}.row-fluid .span1{width:5.982905983%;*width:5.929714493638298%}input,textarea,.uneditable-input{margin-left:0}input.span12,textarea.span12,.uneditable-input.span12{width:1160px}input.span11,textarea.span11,.uneditable-input.span11{width:1060px}input.span10,textarea.span10,.uneditable-input.span10{width:960px}input.span9,textarea.span9,.uneditable-input.span9{width:860px}input.span8,textarea.span8,.uneditable-input.span8{width:760px}input.span7,textarea.span7,.uneditable-input.span7{width:660px}input.span6,textarea.span6,.uneditable-input.span6{width:560px}input.span5,textarea.span5,.uneditable-input.span5{width:460px}input.span4,textarea.span4,.uneditable-input.span4{width:360px}input.span3,textarea.span3,.uneditable-input.span3{width:260px}input.span2,textarea.span2,.uneditable-input.span2{width:160px}input.span1,textarea.span1,.uneditable-input.span1{width:60px}.thumbnails{margin-left:-30px}.thumbnails>li{margin-left:30px}.row-fluid .thumbnails{margin-left:0}}@media(max-width:979px){body{padding-top:0}.navbar-fixed-top,.navbar-fixed-bottom{position:static}.navbar-fixed-top{margin-bottom:18px}.navbar-fixed-bottom{margin-top:18px}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding:5px}.navbar .container{width:auto;padding:0}.navbar .brand{padding-right:10px;padding-left:10px;margin:0 0 0 -5px}.nav-collapse{clear:both}.nav-collapse .nav{float:none;margin:0 0 9px}.nav-collapse .nav>li{float:none}.nav-collapse .nav>li>a{margin-bottom:2px}.nav-collapse .nav>.divider-vertical{display:none}.nav-collapse .nav .nav-header{color:#999;text-shadow:none}.nav-collapse .nav>li>a,.nav-collapse .dropdown-menu a{padding:6px 15px;font-weight:bold;color:#999;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.nav-collapse .btn{padding:4px 10px 4px;font-weight:normal;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.nav-collapse .dropdown-menu li+li a{margin-bottom:2px}.nav-collapse .nav>li>a:hover,.nav-collapse .dropdown-menu a:hover{background-color:#222}.nav-collapse.in .btn-group{padding:0;margin-top:5px}.nav-collapse .dropdown-menu{position:static;top:auto;left:auto;display:block;float:none;max-width:none;padding:0;margin:0 15px;background-color:transparent;border:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.nav-collapse .dropdown-menu:before,.nav-collapse .dropdown-menu:after{display:none}.nav-collapse .dropdown-menu .divider{display:none}.nav-collapse .navbar-form,.nav-collapse .navbar-search{float:none;padding:9px 15px;margin:9px 0;border-top:1px solid #222;border-bottom:1px solid #222;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.1);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.1);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.1)}.navbar .nav-collapse .nav.pull-right{float:none;margin-left:0}.nav-collapse,.nav-collapse.collapse{height:0;overflow:hidden}.navbar .btn-navbar{display:block}.navbar-static .navbar-inner{padding-right:10px;padding-left:10px}}@media(min-width:980px){.nav-collapse.collapse{height:auto!important;overflow:visible!important}} diff --git a/core/src/main/resources/spark/ui/static/bootstrap.min.css b/core/src/main/resources/spark/ui/static/bootstrap.min.css new file mode 100644 index 0000000000..b74b4546a6 --- /dev/null +++ b/core/src/main/resources/spark/ui/static/bootstrap.min.css @@ -0,0 +1,9 @@ +/*! + * Bootstrap v2.0.4 + * + * Copyright 2012 Twitter, Inc + * Licensed under the Apache License v2.0 + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Designed and built with all the love in the world @twitter by @mdo and @fat. + */article,aside,details,figcaption,figure,footer,header,hgroup,nav,section{display:block}audio,canvas,video{display:inline-block;*display:inline;*zoom:1}audio:not([controls]){display:none}html{font-size:100%;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%}a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}a:hover,a:active{outline:0}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sup{top:-0.5em}sub{bottom:-0.25em}img{max-width:100%;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic}#map_canvas img{max-width:none}button,input,select,textarea{margin:0;font-size:100%;vertical-align:middle}button,input{*overflow:visible;line-height:normal}button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0}button,input[type="button"],input[type="reset"],input[type="submit"]{cursor:pointer;-webkit-appearance:button}input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;-webkit-appearance:textfield}input[type="search"]::-webkit-search-decoration,input[type="search"]::-webkit-search-cancel-button{-webkit-appearance:none}textarea{overflow:auto;vertical-align:top}.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:28px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;line-height:18px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.span12{width:940px}.span11{width:860px}.span10{width:780px}.span9{width:700px}.span8{width:620px}.span7{width:540px}.span6{width:460px}.span5{width:380px}.span4{width:300px}.span3{width:220px}.span2{width:140px}.span1{width:60px}.offset12{margin-left:980px}.offset11{margin-left:900px}.offset10{margin-left:820px}.offset9{margin-left:740px}.offset8{margin-left:660px}.offset7{margin-left:580px}.offset6{margin-left:500px}.offset5{margin-left:420px}.offset4{margin-left:340px}.offset3{margin-left:260px}.offset2{margin-left:180px}.offset1{margin-left:100px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:28px;margin-left:2.127659574%;*margin-left:2.0744680846382977%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:99.99999998999999%;*width:99.94680850063828%}.row-fluid .span11{width:91.489361693%;*width:91.4361702036383%}.row-fluid .span10{width:82.97872339599999%;*width:82.92553190663828%}.row-fluid .span9{width:74.468085099%;*width:74.4148936096383%}.row-fluid .span8{width:65.95744680199999%;*width:65.90425531263828%}.row-fluid .span7{width:57.446808505%;*width:57.3936170156383%}.row-fluid .span6{width:48.93617020799999%;*width:48.88297871863829%}.row-fluid .span5{width:40.425531911%;*width:40.3723404216383%}.row-fluid .span4{width:31.914893614%;*width:31.8617021246383%}.row-fluid .span3{width:23.404255317%;*width:23.3510638276383%}.row-fluid .span2{width:14.89361702%;*width:14.8404255306383%}.row-fluid .span1{width:6.382978723%;*width:6.329787233638298%}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;content:""}.container-fluid:after{clear:both}p{margin:0 0 9px}p small{font-size:11px;color:#999}.lead{margin-bottom:18px;font-size:20px;font-weight:200;line-height:27px}h1,h2,h3,h4,h5,h6{margin:0;font-family:inherit;font-weight:bold;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;color:#999}h1{font-size:30px;line-height:36px}h1 small{font-size:18px}h2{font-size:24px;line-height:36px}h2 small{font-size:18px}h3{font-size:18px;line-height:27px}h3 small{font-size:14px}h4,h5,h6{line-height:18px}h4{font-size:14px}h4 small{font-size:12px}h5{font-size:12px}h6{font-size:11px;color:#999;text-transform:uppercase}.page-header{padding-bottom:17px;margin:18px 0;border-bottom:1px solid #eee}.page-header h1{line-height:1}ul,ol{padding:0;margin:0 0 9px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}ul{list-style:disc}ol{list-style:decimal}li{line-height:18px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:18px}dt,dd{line-height:18px}dt{font-weight:bold;line-height:17px}dd{margin-left:9px}.dl-horizontal dt{float:left;width:120px;overflow:hidden;clear:left;text-align:right;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:130px}hr{margin:18px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}strong{font-weight:bold}em{font-style:italic}.muted{color:#999}abbr[title]{cursor:help;border-bottom:1px dotted #999}abbr.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:0 0 0 15px;margin:0 0 18px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:22.5px}blockquote small{display:block;line-height:18px;color:#999}blockquote small:before{content:'\2014 \00A0'}blockquote.pull-right{float:right;padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0}blockquote.pull-right p,blockquote.pull-right small{text-align:right}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:18px;font-style:normal;line-height:18px}small{font-size:100%}cite{font-style:normal}code,pre{padding:0 3px 2px;font-family:Menlo,Monaco,Consolas,"Courier New",monospace;font-size:12px;color:#333;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}code{padding:2px 4px;color:#d14;background-color:#f7f7f9;border:1px solid #e1e1e8}pre{display:block;padding:8.5px;margin:0 0 9px;font-size:12.025px;line-height:18px;word-break:break-all;word-wrap:break-word;white-space:pre;white-space:pre-wrap;background-color:#f5f5f5;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.15);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}pre.prettyprint{margin-bottom:18px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 18px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:27px;font-size:19.5px;line-height:36px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:13.5px;color:#999}label,input,button,select,textarea{font-size:13px;font-weight:normal;line-height:18px}input,button,select,textarea{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif}label{display:block;margin-bottom:5px}select,textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{display:inline-block;height:18px;padding:4px;margin-bottom:9px;font-size:13px;line-height:18px;color:#555}input,textarea{width:210px}textarea{height:auto}textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{background-color:#fff;border:1px solid #ccc;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-webkit-transition:border linear .2s,box-shadow linear .2s;-moz-transition:border linear .2s,box-shadow linear .2s;-ms-transition:border linear .2s,box-shadow linear .2s;-o-transition:border linear .2s,box-shadow linear .2s;transition:border linear .2s,box-shadow linear .2s}textarea:focus,input[type="text"]:focus,input[type="password"]:focus,input[type="datetime"]:focus,input[type="datetime-local"]:focus,input[type="date"]:focus,input[type="month"]:focus,input[type="time"]:focus,input[type="week"]:focus,input[type="number"]:focus,input[type="email"]:focus,input[type="url"]:focus,input[type="search"]:focus,input[type="tel"]:focus,input[type="color"]:focus,.uneditable-input:focus{border-color:rgba(82,168,236,0.8);outline:0;outline:thin dotted \9;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6)}input[type="radio"],input[type="checkbox"]{margin:3px 0;*margin-top:0;line-height:normal;cursor:pointer}input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}.uneditable-textarea{width:auto;height:auto}select,input[type="file"]{height:28px;*margin-top:4px;line-height:28px}select{width:220px;border:1px solid #bbb}select[multiple],select[size]{height:auto}select:focus,input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.radio,.checkbox{min-height:18px;padding-left:18px}.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-18px}.controls>.radio:first-child,.controls>.checkbox:first-child{padding-top:5px}.radio.inline,.checkbox.inline{display:inline-block;padding-top:5px;margin-bottom:0;vertical-align:middle}.radio.inline+.radio.inline,.checkbox.inline+.checkbox.inline{margin-left:10px}.input-mini{width:60px}.input-small{width:90px}.input-medium{width:150px}.input-large{width:210px}.input-xlarge{width:270px}.input-xxlarge{width:530px}input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"]{float:none;margin-left:0}.input-append input[class*="span"],.input-append .uneditable-input[class*="span"],.input-prepend input[class*="span"],.input-prepend .uneditable-input[class*="span"],.row-fluid .input-prepend [class*="span"],.row-fluid .input-append [class*="span"]{display:inline-block}input,textarea,.uneditable-input{margin-left:0}input.span12,textarea.span12,.uneditable-input.span12{width:930px}input.span11,textarea.span11,.uneditable-input.span11{width:850px}input.span10,textarea.span10,.uneditable-input.span10{width:770px}input.span9,textarea.span9,.uneditable-input.span9{width:690px}input.span8,textarea.span8,.uneditable-input.span8{width:610px}input.span7,textarea.span7,.uneditable-input.span7{width:530px}input.span6,textarea.span6,.uneditable-input.span6{width:450px}input.span5,textarea.span5,.uneditable-input.span5{width:370px}input.span4,textarea.span4,.uneditable-input.span4{width:290px}input.span3,textarea.span3,.uneditable-input.span3{width:210px}input.span2,textarea.span2,.uneditable-input.span2{width:130px}input.span1,textarea.span1,.uneditable-input.span1{width:50px}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee;border-color:#ddd}input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent}.control-group.warning>label,.control-group.warning .help-block,.control-group.warning .help-inline{color:#c09853}.control-group.warning .checkbox,.control-group.warning .radio,.control-group.warning input,.control-group.warning select,.control-group.warning textarea{color:#c09853;border-color:#c09853}.control-group.warning .checkbox:focus,.control-group.warning .radio:focus,.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:0 0 6px #dbc59e;-moz-box-shadow:0 0 6px #dbc59e;box-shadow:0 0 6px #dbc59e}.control-group.warning .input-prepend .add-on,.control-group.warning .input-append .add-on{color:#c09853;background-color:#fcf8e3;border-color:#c09853}.control-group.error>label,.control-group.error .help-block,.control-group.error .help-inline{color:#b94a48}.control-group.error .checkbox,.control-group.error .radio,.control-group.error input,.control-group.error select,.control-group.error textarea{color:#b94a48;border-color:#b94a48}.control-group.error .checkbox:focus,.control-group.error .radio:focus,.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:0 0 6px #d59392;-moz-box-shadow:0 0 6px #d59392;box-shadow:0 0 6px #d59392}.control-group.error .input-prepend .add-on,.control-group.error .input-append .add-on{color:#b94a48;background-color:#f2dede;border-color:#b94a48}.control-group.success>label,.control-group.success .help-block,.control-group.success .help-inline{color:#468847}.control-group.success .checkbox,.control-group.success .radio,.control-group.success input,.control-group.success select,.control-group.success textarea{color:#468847;border-color:#468847}.control-group.success .checkbox:focus,.control-group.success .radio:focus,.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:0 0 6px #7aba7b;-moz-box-shadow:0 0 6px #7aba7b;box-shadow:0 0 6px #7aba7b}.control-group.success .input-prepend .add-on,.control-group.success .input-append .add-on{color:#468847;background-color:#dff0d8;border-color:#468847}input:focus:required:invalid,textarea:focus:required:invalid,select:focus:required:invalid{color:#b94a48;border-color:#ee5f5b}input:focus:required:invalid:focus,textarea:focus:required:invalid:focus,select:focus:required:invalid:focus{border-color:#e9322d;-webkit-box-shadow:0 0 6px #f8b9b7;-moz-box-shadow:0 0 6px #f8b9b7;box-shadow:0 0 6px #f8b9b7}.form-actions{padding:17px 20px 18px;margin-top:18px;margin-bottom:18px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;content:""}.form-actions:after{clear:both}.uneditable-input{overflow:hidden;white-space:nowrap;cursor:not-allowed;background-color:#fff;border-color:#eee;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);box-shadow:inset 0 1px 2px rgba(0,0,0,0.025)}:-moz-placeholder{color:#999}:-ms-input-placeholder{color:#999}::-webkit-input-placeholder{color:#999}.help-block,.help-inline{color:#555}.help-block{display:block;margin-bottom:9px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-prepend,.input-append{margin-bottom:5px}.input-prepend input,.input-append input,.input-prepend select,.input-append select,.input-prepend .uneditable-input,.input-append .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;vertical-align:middle;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-prepend input:focus,.input-append input:focus,.input-prepend select:focus,.input-append select:focus,.input-prepend .uneditable-input:focus,.input-append .uneditable-input:focus{z-index:2}.input-prepend .uneditable-input,.input-append .uneditable-input{border-left-color:#ccc}.input-prepend .add-on,.input-append .add-on{display:inline-block;width:auto;height:18px;min-width:16px;padding:4px 5px;font-weight:normal;line-height:18px;text-align:center;text-shadow:0 1px 0 #fff;vertical-align:middle;background-color:#eee;border:1px solid #ccc}.input-prepend .add-on,.input-append .add-on,.input-prepend .btn,.input-append .btn{margin-left:-1px;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend .active,.input-append .active{background-color:#a9dba9;border-color:#46a546}.input-prepend .add-on,.input-prepend .btn{margin-right:-1px}.input-prepend .add-on:first-child,.input-prepend .btn:first-child{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append .uneditable-input{border-right-color:#ccc;border-left-color:#eee}.input-append .add-on:last-child,.input-append .btn:last-child{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-prepend.input-append input,.input-prepend.input-append select,.input-prepend.input-append .uneditable-input{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:14px;-moz-border-radius:14px;border-radius:14px}.form-search input,.form-inline input,.form-horizontal input,.form-search textarea,.form-inline textarea,.form-horizontal textarea,.form-search select,.form-inline select,.form-horizontal select,.form-search .help-inline,.form-inline .help-inline,.form-horizontal .help-inline,.form-search .uneditable-input,.form-inline .uneditable-input,.form-horizontal .uneditable-input,.form-search .input-prepend,.form-inline .input-prepend,.form-horizontal .input-prepend,.form-search .input-append,.form-inline .input-append,.form-horizontal .input-append{display:inline-block;*display:inline;margin-bottom:0;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label{display:inline-block}.form-search .input-append,.form-inline .input-append,.form-search .input-prepend,.form-inline .input-prepend{margin-bottom:0}.form-search .radio,.form-search .checkbox,.form-inline .radio,.form-inline .checkbox{padding-left:0;margin-bottom:0;vertical-align:middle}.form-search .radio input[type="radio"],.form-search .checkbox input[type="checkbox"],.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:left;margin-right:3px;margin-left:0}.control-group{margin-bottom:9px}legend+.control-group{margin-top:18px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:18px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;content:""}.form-horizontal .control-group:after{clear:both}.form-horizontal .control-label{float:left;width:140px;padding-top:5px;text-align:right}.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:160px;*margin-left:0}.form-horizontal .controls:first-child{*padding-left:160px}.form-horizontal .help-block{margin-top:9px;margin-bottom:0}.form-horizontal .form-actions{padding-left:160px}table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0}.table{width:100%;margin-bottom:18px}.table th,.table td{padding:8px;line-height:18px;text-align:left;vertical-align:top;border-top:1px solid #ddd}.table th{font-weight:bold}.table thead th{vertical-align:bottom}.table caption+thead tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child th,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child th,.table thead:first-child tr:first-child td{border-top:0}.table tbody+tbody{border-top:2px solid #ddd}.table-condensed th,.table-condensed td{padding:4px 5px}.table-bordered{border:1px solid #ddd;border-collapse:separate;*border-collapse:collapsed;border-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.table-bordered th,.table-bordered td{border-left:1px solid #ddd}.table-bordered caption+thead tr:first-child th,.table-bordered caption+tbody tr:first-child th,.table-bordered caption+tbody tr:first-child td,.table-bordered colgroup+thead tr:first-child th,.table-bordered colgroup+tbody tr:first-child th,.table-bordered colgroup+tbody tr:first-child td,.table-bordered thead:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child td{border-top:0}.table-bordered thead:first-child tr:first-child th:first-child,.table-bordered tbody:first-child tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered thead:first-child tr:first-child th:last-child,.table-bordered tbody:first-child tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-bordered thead:last-child tr:last-child th:first-child,.table-bordered tbody:last-child tr:last-child td:first-child{-webkit-border-radius:0 0 0 4px;-moz-border-radius:0 0 0 4px;border-radius:0 0 0 4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px}.table-bordered thead:last-child tr:last-child th:last-child,.table-bordered tbody:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px}.table-striped tbody tr:nth-child(odd) td,.table-striped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table tbody tr:hover td,.table tbody tr:hover th{background-color:#f5f5f5}table .span1{float:none;width:44px;margin-left:0}table .span2{float:none;width:124px;margin-left:0}table .span3{float:none;width:204px;margin-left:0}table .span4{float:none;width:284px;margin-left:0}table .span5{float:none;width:364px;margin-left:0}table .span6{float:none;width:444px;margin-left:0}table .span7{float:none;width:524px;margin-left:0}table .span8{float:none;width:604px;margin-left:0}table .span9{float:none;width:684px;margin-left:0}table .span10{float:none;width:764px;margin-left:0}table .span11{float:none;width:844px;margin-left:0}table .span12{float:none;width:924px;margin-left:0}table .span13{float:none;width:1004px;margin-left:0}table .span14{float:none;width:1084px;margin-left:0}table .span15{float:none;width:1164px;margin-left:0}table .span16{float:none;width:1244px;margin-left:0}table .span17{float:none;width:1324px;margin-left:0}table .span18{float:none;width:1404px;margin-left:0}table .span19{float:none;width:1484px;margin-left:0}table .span20{float:none;width:1564px;margin-left:0}table .span21{float:none;width:1644px;margin-left:0}table .span22{float:none;width:1724px;margin-left:0}table .span23{float:none;width:1804px;margin-left:0}table .span24{float:none;width:1884px;margin-left:0}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;*margin-right:.3em;line-height:14px;vertical-align:text-top;background-image:url("../img/glyphicons-halflings.png");background-position:14px 14px;background-repeat:no-repeat}[class^="icon-"]:last-child,[class*=" icon-"]:last-child{*margin-left:0}.icon-white{background-image:url("../img/glyphicons-halflings-white.png")}.icon-glass{background-position:0 0}.icon-music{background-position:-24px 0}.icon-search{background-position:-48px 0}.icon-envelope{background-position:-72px 0}.icon-heart{background-position:-96px 0}.icon-star{background-position:-120px 0}.icon-star-empty{background-position:-144px 0}.icon-user{background-position:-168px 0}.icon-film{background-position:-192px 0}.icon-th-large{background-position:-216px 0}.icon-th{background-position:-240px 0}.icon-th-list{background-position:-264px 0}.icon-ok{background-position:-288px 0}.icon-remove{background-position:-312px 0}.icon-zoom-in{background-position:-336px 0}.icon-zoom-out{background-position:-360px 0}.icon-off{background-position:-384px 0}.icon-signal{background-position:-408px 0}.icon-cog{background-position:-432px 0}.icon-trash{background-position:-456px 0}.icon-home{background-position:0 -24px}.icon-file{background-position:-24px -24px}.icon-time{background-position:-48px -24px}.icon-road{background-position:-72px -24px}.icon-download-alt{background-position:-96px -24px}.icon-download{background-position:-120px -24px}.icon-upload{background-position:-144px -24px}.icon-inbox{background-position:-168px -24px}.icon-play-circle{background-position:-192px -24px}.icon-repeat{background-position:-216px -24px}.icon-refresh{background-position:-240px -24px}.icon-list-alt{background-position:-264px -24px}.icon-lock{background-position:-287px -24px}.icon-flag{background-position:-312px -24px}.icon-headphones{background-position:-336px -24px}.icon-volume-off{background-position:-360px -24px}.icon-volume-down{background-position:-384px -24px}.icon-volume-up{background-position:-408px -24px}.icon-qrcode{background-position:-432px -24px}.icon-barcode{background-position:-456px -24px}.icon-tag{background-position:0 -48px}.icon-tags{background-position:-25px -48px}.icon-book{background-position:-48px -48px}.icon-bookmark{background-position:-72px -48px}.icon-print{background-position:-96px -48px}.icon-camera{background-position:-120px -48px}.icon-font{background-position:-144px -48px}.icon-bold{background-position:-167px -48px}.icon-italic{background-position:-192px -48px}.icon-text-height{background-position:-216px -48px}.icon-text-width{background-position:-240px -48px}.icon-align-left{background-position:-264px -48px}.icon-align-center{background-position:-288px -48px}.icon-align-right{background-position:-312px -48px}.icon-align-justify{background-position:-336px -48px}.icon-list{background-position:-360px -48px}.icon-indent-left{background-position:-384px -48px}.icon-indent-right{background-position:-408px -48px}.icon-facetime-video{background-position:-432px -48px}.icon-picture{background-position:-456px -48px}.icon-pencil{background-position:0 -72px}.icon-map-marker{background-position:-24px -72px}.icon-adjust{background-position:-48px -72px}.icon-tint{background-position:-72px -72px}.icon-edit{background-position:-96px -72px}.icon-share{background-position:-120px -72px}.icon-check{background-position:-144px -72px}.icon-move{background-position:-168px -72px}.icon-step-backward{background-position:-192px -72px}.icon-fast-backward{background-position:-216px -72px}.icon-backward{background-position:-240px -72px}.icon-play{background-position:-264px -72px}.icon-pause{background-position:-288px -72px}.icon-stop{background-position:-312px -72px}.icon-forward{background-position:-336px -72px}.icon-fast-forward{background-position:-360px -72px}.icon-step-forward{background-position:-384px -72px}.icon-eject{background-position:-408px -72px}.icon-chevron-left{background-position:-432px -72px}.icon-chevron-right{background-position:-456px -72px}.icon-plus-sign{background-position:0 -96px}.icon-minus-sign{background-position:-24px -96px}.icon-remove-sign{background-position:-48px -96px}.icon-ok-sign{background-position:-72px -96px}.icon-question-sign{background-position:-96px -96px}.icon-info-sign{background-position:-120px -96px}.icon-screenshot{background-position:-144px -96px}.icon-remove-circle{background-position:-168px -96px}.icon-ok-circle{background-position:-192px -96px}.icon-ban-circle{background-position:-216px -96px}.icon-arrow-left{background-position:-240px -96px}.icon-arrow-right{background-position:-264px -96px}.icon-arrow-up{background-position:-289px -96px}.icon-arrow-down{background-position:-312px -96px}.icon-share-alt{background-position:-336px -96px}.icon-resize-full{background-position:-360px -96px}.icon-resize-small{background-position:-384px -96px}.icon-plus{background-position:-408px -96px}.icon-minus{background-position:-433px -96px}.icon-asterisk{background-position:-456px -96px}.icon-exclamation-sign{background-position:0 -120px}.icon-gift{background-position:-24px -120px}.icon-leaf{background-position:-48px -120px}.icon-fire{background-position:-72px -120px}.icon-eye-open{background-position:-96px -120px}.icon-eye-close{background-position:-120px -120px}.icon-warning-sign{background-position:-144px -120px}.icon-plane{background-position:-168px -120px}.icon-calendar{background-position:-192px -120px}.icon-random{background-position:-216px -120px}.icon-comment{background-position:-240px -120px}.icon-magnet{background-position:-264px -120px}.icon-chevron-up{background-position:-288px -120px}.icon-chevron-down{background-position:-313px -119px}.icon-retweet{background-position:-336px -120px}.icon-shopping-cart{background-position:-360px -120px}.icon-folder-close{background-position:-384px -120px}.icon-folder-open{background-position:-408px -120px}.icon-resize-vertical{background-position:-432px -119px}.icon-resize-horizontal{background-position:-456px -118px}.icon-hdd{background-position:0 -144px}.icon-bullhorn{background-position:-24px -144px}.icon-bell{background-position:-48px -144px}.icon-certificate{background-position:-72px -144px}.icon-thumbs-up{background-position:-96px -144px}.icon-thumbs-down{background-position:-120px -144px}.icon-hand-right{background-position:-144px -144px}.icon-hand-left{background-position:-168px -144px}.icon-hand-up{background-position:-192px -144px}.icon-hand-down{background-position:-216px -144px}.icon-circle-arrow-right{background-position:-240px -144px}.icon-circle-arrow-left{background-position:-264px -144px}.icon-circle-arrow-up{background-position:-288px -144px}.icon-circle-arrow-down{background-position:-312px -144px}.icon-globe{background-position:-336px -144px}.icon-wrench{background-position:-360px -144px}.icon-tasks{background-position:-384px -144px}.icon-filter{background-position:-408px -144px}.icon-briefcase{background-position:-432px -144px}.icon-fullscreen{background-position:-456px -144px}.dropup,.dropdown{position:relative}.dropdown-toggle{*margin-bottom:-3px}.dropdown-toggle:active,.open .dropdown-toggle{outline:0}.caret{display:inline-block;width:0;height:0;vertical-align:top;border-top:4px solid #000;border-right:4px solid transparent;border-left:4px solid transparent;content:"";opacity:.3;filter:alpha(opacity=30)}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown:hover .caret,.open .caret{opacity:1;filter:alpha(opacity=100)}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:4px 0;margin:1px 0 0;list-style:none;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);*border-right-width:2px;*border-bottom-width:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{*width:100%;height:1px;margin:8px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:block;padding:3px 15px;clear:both;font-weight:normal;line-height:18px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#08c}.open{*z-index:1000}.open>.dropdown-menu{display:block}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #000;content:"\2191"}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}.typeahead{margin-top:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #eee;border:1px solid rgba(0,0,0,0.05);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);box-shadow:inset 0 1px 1px rgba(0,0,0,0.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,0.15)}.well-large{padding:24px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.well-small{padding:9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.fade{opacity:0;-webkit-transition:opacity .15s linear;-moz-transition:opacity .15s linear;-ms-transition:opacity .15s linear;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;-webkit-transition:height .35s ease;-moz-transition:height .35s ease;-ms-transition:height .35s ease;-o-transition:height .35s ease;transition:height .35s ease}.collapse.in{height:auto}.close{float:right;font-size:20px;font-weight:bold;line-height:18px;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover{color:#000;text-decoration:none;cursor:pointer;opacity:.4;filter:alpha(opacity=40)}button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none}.btn{display:inline-block;*display:inline;padding:4px 10px 4px;margin-bottom:0;*margin-left:.3em;font-size:13px;line-height:18px;*line-height:20px;color:#333;text-align:center;text-shadow:0 1px 1px rgba(255,255,255,0.75);vertical-align:middle;cursor:pointer;background-color:#f5f5f5;*background-color:#e6e6e6;background-image:-ms-linear-gradient(top,#fff,#e6e6e6);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e6e6e6));background-image:-webkit-linear-gradient(top,#fff,#e6e6e6);background-image:-o-linear-gradient(top,#fff,#e6e6e6);background-image:linear-gradient(top,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #ccc;*border:0;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-bottom-color:#b3b3b3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffff',endColorstr='#e6e6e6',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);*zoom:1;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn:hover,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{background-color:#e6e6e6;*background-color:#d9d9d9}.btn:active,.btn.active{background-color:#ccc \9}.btn:first-child{*margin-left:0}.btn:hover{color:#333;text-decoration:none;background-color:#e6e6e6;*background-color:#d9d9d9;background-position:0 -15px;-webkit-transition:background-position .1s linear;-moz-transition:background-position .1s linear;-ms-transition:background-position .1s linear;-o-transition:background-position .1s linear;transition:background-position .1s linear}.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn.active,.btn:active{background-color:#e6e6e6;background-color:#d9d9d9 \9;background-image:none;outline:0;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn.disabled,.btn[disabled]{cursor:default;background-color:#e6e6e6;background-image:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-large{padding:9px 14px;font-size:15px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:1px}.btn-small{padding:5px 9px;font-size:11px;line-height:16px}.btn-small [class^="icon-"]{margin-top:-1px}.btn-mini{padding:2px 6px;font-size:11px;line-height:14px}.btn-primary,.btn-primary:hover,.btn-warning,.btn-warning:hover,.btn-danger,.btn-danger:hover,.btn-success,.btn-success:hover,.btn-info,.btn-info:hover,.btn-inverse,.btn-inverse:hover{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.btn-primary.active,.btn-warning.active,.btn-danger.active,.btn-success.active,.btn-info.active,.btn-inverse.active{color:rgba(255,255,255,0.75)}.btn{border-color:#ccc;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25)}.btn-primary{background-color:#0074cc;*background-color:#05c;background-image:-ms-linear-gradient(top,#08c,#05c);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#05c));background-image:-webkit-linear-gradient(top,#08c,#05c);background-image:-o-linear-gradient(top,#08c,#05c);background-image:-moz-linear-gradient(top,#08c,#05c);background-image:linear-gradient(top,#08c,#05c);background-repeat:repeat-x;border-color:#05c #05c #003580;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#0088cc',endColorstr='#0055cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{background-color:#05c;*background-color:#004ab3}.btn-primary:active,.btn-primary.active{background-color:#004099 \9}.btn-warning{background-color:#faa732;*background-color:#f89406;background-image:-ms-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(top,#fbb450,#f89406);background-repeat:repeat-x;border-color:#f89406 #f89406 #ad6704;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fbb450',endColorstr='#f89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{background-color:#da4f49;*background-color:#bd362f;background-image:-ms-linear-gradient(top,#ee5f5b,#bd362f);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#bd362f));background-image:-webkit-linear-gradient(top,#ee5f5b,#bd362f);background-image:-o-linear-gradient(top,#ee5f5b,#bd362f);background-image:-moz-linear-gradient(top,#ee5f5b,#bd362f);background-image:linear-gradient(top,#ee5f5b,#bd362f);background-repeat:repeat-x;border-color:#bd362f #bd362f #802420;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ee5f5b',endColorstr='#bd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{background-color:#5bb75b;*background-color:#51a351;background-image:-ms-linear-gradient(top,#62c462,#51a351);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#51a351));background-image:-webkit-linear-gradient(top,#62c462,#51a351);background-image:-o-linear-gradient(top,#62c462,#51a351);background-image:-moz-linear-gradient(top,#62c462,#51a351);background-image:linear-gradient(top,#62c462,#51a351);background-repeat:repeat-x;border-color:#51a351 #51a351 #387038;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#62c462',endColorstr='#51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{background-color:#49afcd;*background-color:#2f96b4;background-image:-ms-linear-gradient(top,#5bc0de,#2f96b4);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#2f96b4));background-image:-webkit-linear-gradient(top,#5bc0de,#2f96b4);background-image:-o-linear-gradient(top,#5bc0de,#2f96b4);background-image:-moz-linear-gradient(top,#5bc0de,#2f96b4);background-image:linear-gradient(top,#5bc0de,#2f96b4);background-repeat:repeat-x;border-color:#2f96b4 #2f96b4 #1f6377;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#5bc0de',endColorstr='#2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{background-color:#414141;*background-color:#222;background-image:-ms-linear-gradient(top,#555,#222);background-image:-webkit-gradient(linear,0 0,0 100%,from(#555),to(#222));background-image:-webkit-linear-gradient(top,#555,#222);background-image:-o-linear-gradient(top,#555,#222);background-image:-moz-linear-gradient(top,#555,#222);background-image:linear-gradient(top,#555,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#555555',endColorstr='#222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:2px;*padding-bottom:2px}button.btn::-moz-focus-inner,input[type="submit"].btn::-moz-focus-inner{padding:0;border:0}button.btn.btn-large,input[type="submit"].btn.btn-large{*padding-top:7px;*padding-bottom:7px}button.btn.btn-small,input[type="submit"].btn.btn-small{*padding-top:3px;*padding-bottom:3px}button.btn.btn-mini,input[type="submit"].btn.btn-mini{*padding-top:1px;*padding-bottom:1px}.btn-group{position:relative;*margin-left:.3em;*zoom:1}.btn-group:before,.btn-group:after{display:table;content:""}.btn-group:after{clear:both}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:9px;margin-bottom:9px}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-group>.btn{position:relative;float:left;margin-left:-1px;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn:first-child{margin-left:0;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.btn-group>.btn:last-child,.btn-group>.dropdown-toggle{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.btn-group>.btn.large:first-child{margin-left:0;-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.btn-group>.btn.large:last-child,.btn-group>.large.dropdown-toggle{-webkit-border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.btn-group>.btn:hover,.btn-group>.btn:focus,.btn-group>.btn:active,.btn-group>.btn.active{z-index:2}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.dropdown-toggle{*padding-top:4px;padding-right:8px;*padding-bottom:4px;padding-left:8px;-webkit-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn-group>.btn-mini.dropdown-toggle{padding-right:5px;padding-left:5px}.btn-group>.btn-small.dropdown-toggle{*padding-top:4px;*padding-bottom:4px}.btn-group>.btn-large.dropdown-toggle{padding-right:12px;padding-left:12px}.btn-group.open .dropdown-toggle{background-image:none;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn-group.open .btn.dropdown-toggle{background-color:#e6e6e6}.btn-group.open .btn-primary.dropdown-toggle{background-color:#05c}.btn-group.open .btn-warning.dropdown-toggle{background-color:#f89406}.btn-group.open .btn-danger.dropdown-toggle{background-color:#bd362f}.btn-group.open .btn-success.dropdown-toggle{background-color:#51a351}.btn-group.open .btn-info.dropdown-toggle{background-color:#2f96b4}.btn-group.open .btn-inverse.dropdown-toggle{background-color:#222}.btn .caret{margin-top:7px;margin-left:0}.btn:hover .caret,.open.btn-group .caret{opacity:1;filter:alpha(opacity=100)}.btn-mini .caret{margin-top:5px}.btn-small .caret{margin-top:6px}.btn-large .caret{margin-top:6px;border-top-width:5px;border-right-width:5px;border-left-width:5px}.dropup .btn-large .caret{border-top:0;border-bottom:5px solid #000}.btn-primary .caret,.btn-warning .caret,.btn-danger .caret,.btn-info .caret,.btn-success .caret,.btn-inverse .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:.75;filter:alpha(opacity=75)}.alert{padding:8px 35px 8px 14px;margin-bottom:18px;color:#c09853;text-shadow:0 1px 0 rgba(255,255,255,0.5);background-color:#fcf8e3;border:1px solid #fbeed5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.alert-heading{color:inherit}.alert .close{position:relative;top:-2px;right:-21px;line-height:18px}.alert-success{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.alert-danger,.alert-error{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.alert-info{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.alert-block{padding-top:14px;padding-bottom:14px}.alert-block>p,.alert-block>ul{margin-bottom:0}.alert-block p+p{margin-top:5px}.nav{margin-bottom:18px;margin-left:0;list-style:none}.nav>li>a{display:block}.nav>li>a:hover{text-decoration:none;background-color:#eee}.nav>.pull-right{float:right}.nav .nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:18px;color:#999;text-shadow:0 1px 0 rgba(255,255,255,0.5);text-transform:uppercase}.nav li+.nav-header{margin-top:9px}.nav-list{padding-right:15px;padding-left:15px;margin-bottom:0}.nav-list>li>a,.nav-list .nav-header{margin-right:-15px;margin-left:-15px;text-shadow:0 1px 0 rgba(255,255,255,0.5)}.nav-list>li>a{padding:3px 15px}.nav-list>.active>a,.nav-list>.active>a:hover{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.2);background-color:#08c}.nav-list [class^="icon-"]{margin-right:2px}.nav-list .divider{*width:100%;height:1px;margin:8px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.nav-tabs,.nav-pills{*zoom:1}.nav-tabs:before,.nav-pills:before,.nav-tabs:after,.nav-pills:after{display:table;content:""}.nav-tabs:after,.nav-pills:after{clear:both}.nav-tabs>li,.nav-pills>li{float:left}.nav-tabs>li>a,.nav-pills>li>a{padding-right:12px;padding-left:12px;margin-right:2px;line-height:14px}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{margin-bottom:-1px}.nav-tabs>li>a{padding-top:8px;padding-bottom:8px;line-height:18px;border:1px solid transparent;-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover{border-color:#eee #eee #ddd}.nav-tabs>.active>a,.nav-tabs>.active>a:hover{color:#555;cursor:default;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent}.nav-pills>li>a{padding-top:8px;padding-bottom:8px;margin-top:2px;margin-bottom:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.nav-pills>.active>a,.nav-pills>.active>a:hover{color:#fff;background-color:#08c}.nav-stacked>li{float:none}.nav-stacked>li>a{margin-right:0}.nav-tabs.nav-stacked{border-bottom:0}.nav-tabs.nav-stacked>li>a{border:1px solid #ddd;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.nav-tabs.nav-stacked>li:first-child>a{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.nav-tabs.nav-stacked>li>a:hover{z-index:2;border-color:#ddd}.nav-pills.nav-stacked>li>a{margin-bottom:3px}.nav-pills.nav-stacked>li:last-child>a{margin-bottom:1px}.nav-tabs .dropdown-menu{-webkit-border-radius:0 0 5px 5px;-moz-border-radius:0 0 5px 5px;border-radius:0 0 5px 5px}.nav-pills .dropdown-menu{-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.nav-tabs .dropdown-toggle .caret,.nav-pills .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav-tabs .dropdown-toggle:hover .caret,.nav-pills .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .active .dropdown-toggle .caret,.nav-pills .active .dropdown-toggle .caret{border-top-color:#333;border-bottom-color:#333}.nav>.dropdown.active>a:hover{color:#000;cursor:pointer}.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover{color:#fff;background-color:#999;border-color:#999}.nav li.dropdown.open .caret,.nav li.dropdown.open.active .caret,.nav li.dropdown.open a:hover .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:1;filter:alpha(opacity=100)}.tabs-stacked .open>a:hover{border-color:#999}.tabbable{*zoom:1}.tabbable:before,.tabbable:after{display:table;content:""}.tabbable:after{clear:both}.tab-content{overflow:auto}.tabs-below>.nav-tabs,.tabs-right>.nav-tabs,.tabs-left>.nav-tabs{border-bottom:0}.tab-content>.tab-pane,.pill-content>.pill-pane{display:none}.tab-content>.active,.pill-content>.active{display:block}.tabs-below>.nav-tabs{border-top:1px solid #ddd}.tabs-below>.nav-tabs>li{margin-top:-1px;margin-bottom:0}.tabs-below>.nav-tabs>li>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.tabs-below>.nav-tabs>li>a:hover{border-top-color:#ddd;border-bottom-color:transparent}.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover{border-color:transparent #ddd #ddd #ddd}.tabs-left>.nav-tabs>li,.tabs-right>.nav-tabs>li{float:none}.tabs-left>.nav-tabs>li>a,.tabs-right>.nav-tabs>li>a{min-width:74px;margin-right:0;margin-bottom:3px}.tabs-left>.nav-tabs{float:left;margin-right:19px;border-right:1px solid #ddd}.tabs-left>.nav-tabs>li>a{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.tabs-left>.nav-tabs>li>a:hover{border-color:#eee #ddd #eee #eee}.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover{border-color:#ddd transparent #ddd #ddd;*border-right-color:#fff}.tabs-right>.nav-tabs{float:right;margin-left:19px;border-left:1px solid #ddd}.tabs-right>.nav-tabs>li>a{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.tabs-right>.nav-tabs>li>a:hover{border-color:#eee #eee #eee #ddd}.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover{border-color:#ddd #ddd #ddd transparent;*border-left-color:#fff}.navbar{*position:relative;*z-index:2;margin-bottom:18px;overflow:visible}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#2c2c2c;background-image:-moz-linear-gradient(top,#333,#222);background-image:-ms-linear-gradient(top,#333,#222);background-image:-webkit-gradient(linear,0 0,0 100%,from(#333),to(#222));background-image:-webkit-linear-gradient(top,#333,#222);background-image:-o-linear-gradient(top,#333,#222);background-image:linear-gradient(top,#333,#222);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#333333',endColorstr='#222222',GradientType=0);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.25),inset 0 -1px 0 rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.25),inset 0 -1px 0 rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.25),inset 0 -1px 0 rgba(0,0,0,0.1)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar{color:#999}.navbar .brand:hover{text-decoration:none}.navbar .brand{display:block;float:left;padding:8px 20px 12px;margin-left:-20px;font-size:20px;font-weight:200;line-height:1;color:#999}.navbar .navbar-text{margin-bottom:0;line-height:40px}.navbar .navbar-link{color:#999}.navbar .navbar-link:hover{color:#fff}.navbar .btn,.navbar .btn-group{margin-top:5px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;content:""}.navbar-form:after{clear:both}.navbar-form input,.navbar-form select,.navbar-form .radio,.navbar-form .checkbox{margin-top:5px}.navbar-form input,.navbar-form select{display:inline-block;margin-bottom:0}.navbar-form input[type="image"],.navbar-form input[type="checkbox"],.navbar-form input[type="radio"]{margin-top:3px}.navbar-form .input-append,.navbar-form .input-prepend{margin-top:6px;white-space:nowrap}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0}.navbar-search{position:relative;float:left;margin-top:6px;margin-bottom:0}.navbar-search .search-query{padding:4px 9px;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;color:#fff;background-color:#626262;border:1px solid #151515;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-webkit-transition:none;-moz-transition:none;-ms-transition:none;-o-transition:none;transition:none}.navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-search .search-query:focus,.navbar-search .search-query.focused{padding:5px 10px;color:#333;text-shadow:0 1px 0 #fff;background-color:#fff;border:0;outline:0;-webkit-box-shadow:0 0 3px rgba(0,0,0,0.15);-moz-box-shadow:0 0 3px rgba(0,0,0,0.15);box-shadow:0 0 3px rgba(0,0,0,0.15)}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-bottom:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding-right:0;padding-left:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-bottom{bottom:0}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{display:block;float:left}.navbar .nav>li>a{float:none;padding:9px 10px 11px;line-height:19px;color:#999;text-decoration:none;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar .btn{display:inline-block;padding:4px 10px 4px;margin:5px 5px 6px;line-height:18px}.navbar .btn-group{padding:5px 5px 6px;margin:0}.navbar .nav>li>a:hover{color:#fff;text-decoration:none;background-color:transparent}.navbar .nav .active>a,.navbar .nav .active>a:hover{color:#fff;text-decoration:none;background-color:#222}.navbar .divider-vertical{width:1px;height:40px;margin:0 9px;overflow:hidden;background-color:#222;border-right:1px solid #333}.navbar .nav.pull-right{margin-right:0;margin-left:10px}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;background-color:#2c2c2c;*background-color:#222;background-image:-ms-linear-gradient(top,#333,#222);background-image:-webkit-gradient(linear,0 0,0 100%,from(#333),to(#222));background-image:-webkit-linear-gradient(top,#333,#222);background-image:-o-linear-gradient(top,#333,#222);background-image:linear-gradient(top,#333,#222);background-image:-moz-linear-gradient(top,#333,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#333333',endColorstr='#222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075)}.navbar .btn-navbar:hover,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{background-color:#222;*background-color:#151515}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#080808 \9}.navbar .btn-navbar .icon-bar{display:block;width:18px;height:2px;background-color:#f5f5f5;-webkit-border-radius:1px;-moz-border-radius:1px;border-radius:1px;-webkit-box-shadow:0 1px 0 rgba(0,0,0,0.25);-moz-box-shadow:0 1px 0 rgba(0,0,0,0.25);box-shadow:0 1px 0 rgba(0,0,0,0.25)}.btn-navbar .icon-bar+.icon-bar{margin-top:3px}.navbar .dropdown-menu:before{position:absolute;top:-7px;left:9px;display:inline-block;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-left:7px solid transparent;border-bottom-color:rgba(0,0,0,0.2);content:''}.navbar .dropdown-menu:after{position:absolute;top:-6px;left:10px;display:inline-block;border-right:6px solid transparent;border-bottom:6px solid #fff;border-left:6px solid transparent;content:''}.navbar-fixed-bottom .dropdown-menu:before{top:auto;bottom:-7px;border-top:7px solid #ccc;border-bottom:0;border-top-color:rgba(0,0,0,0.2)}.navbar-fixed-bottom .dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown .dropdown-toggle .caret,.navbar .nav li.dropdown.open .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar .nav li.dropdown.active .caret{opacity:1;filter:alpha(opacity=100)}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{background-color:transparent}.navbar .nav li.dropdown.active>.dropdown-toggle:hover{color:#fff}.navbar .pull-right .dropdown-menu,.navbar .dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right .dropdown-menu:before,.navbar .dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right .dropdown-menu:after,.navbar .dropdown-menu.pull-right:after{right:13px;left:auto}.breadcrumb{padding:7px 14px;margin:0 0 18px;list-style:none;background-color:#fbfbfb;background-image:-moz-linear-gradient(top,#fff,#f5f5f5);background-image:-ms-linear-gradient(top,#fff,#f5f5f5);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#f5f5f5));background-image:-webkit-linear-gradient(top,#fff,#f5f5f5);background-image:-o-linear-gradient(top,#fff,#f5f5f5);background-image:linear-gradient(top,#fff,#f5f5f5);background-repeat:repeat-x;border:1px solid #ddd;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffff',endColorstr='#f5f5f5',GradientType=0);-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#999}.breadcrumb .active a{color:#333}.pagination{height:36px;margin:18px 0}.pagination ul{display:inline-block;*display:inline;margin-bottom:0;margin-left:0;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;*zoom:1;-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:0 1px 2px rgba(0,0,0,0.05);box-shadow:0 1px 2px rgba(0,0,0,0.05)}.pagination li{display:inline}.pagination a{float:left;padding:0 14px;line-height:34px;text-decoration:none;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a{background-color:#f5f5f5}.pagination .active a{color:#999;cursor:default}.pagination .disabled span,.pagination .disabled a,.pagination .disabled a:hover{color:#999;cursor:default;background-color:transparent}.pagination li:first-child a{border-left-width:1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.pagination li:last-child a{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.pagination-centered{text-align:center}.pagination-right{text-align:right}.pager{margin-bottom:18px;margin-left:0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;content:""}.pager:after{clear:both}.pager li{display:inline}.pager a{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.pager a:hover{text-decoration:none;background-color:#f5f5f5}.pager .next a{float:right}.pager .previous a{float:left}.pager .disabled a,.pager .disabled a:hover{color:#999;cursor:default;background-color:#fff}.modal-open .dropdown-menu{z-index:2050}.modal-open .dropdown.open{*z-index:2050}.modal-open .popover{z-index:2060}.modal-open .tooltip{z-index:2070}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000}.modal-backdrop.fade{opacity:0}.modal-backdrop,.modal-backdrop.fade.in{opacity:.8;filter:alpha(opacity=80)}.modal{position:fixed;top:50%;left:50%;z-index:1050;width:560px;margin:-250px 0 0 -280px;overflow:auto;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,0.3);*border:1px solid #999;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3);-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box}.modal.fade{top:-25%;-webkit-transition:opacity .3s linear,top .3s ease-out;-moz-transition:opacity .3s linear,top .3s ease-out;-ms-transition:opacity .3s linear,top .3s ease-out;-o-transition:opacity .3s linear,top .3s ease-out;transition:opacity .3s linear,top .3s ease-out}.modal.fade.in{top:50%}.modal-header{padding:9px 15px;border-bottom:1px solid #eee}.modal-header .close{margin-top:2px}.modal-body{max-height:400px;padding:15px;overflow-y:auto}.modal-form{margin-bottom:0}.modal-footer{padding:14px 15px 15px;margin-bottom:0;text-align:right;background-color:#f5f5f5;border-top:1px solid #ddd;-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;*zoom:1;-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.modal-footer:before,.modal-footer:after{display:table;content:""}.modal-footer:after{clear:both}.modal-footer .btn+.btn{margin-bottom:0;margin-left:5px}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.tooltip{position:absolute;z-index:1020;display:block;padding:5px;font-size:11px;opacity:0;filter:alpha(opacity=0);visibility:visible}.tooltip.in{opacity:.8;filter:alpha(opacity=80)}.tooltip.top{margin-top:-2px}.tooltip.right{margin-left:2px}.tooltip.bottom{margin-top:2px}.tooltip.left{margin-left:-2px}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top:5px solid #000;border-right:5px solid transparent;border-left:5px solid transparent}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-top:5px solid transparent;border-bottom:5px solid transparent;border-left:5px solid #000}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-right:5px solid transparent;border-bottom:5px solid #000;border-left:5px solid transparent}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-top:5px solid transparent;border-right:5px solid #000;border-bottom:5px solid transparent}.tooltip-inner{max-width:200px;padding:3px 8px;color:#fff;text-align:center;text-decoration:none;background-color:#000;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.tooltip-arrow{position:absolute;width:0;height:0}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;padding:5px}.popover.top{margin-top:-5px}.popover.right{margin-left:5px}.popover.bottom{margin-top:5px}.popover.left{margin-left:-5px}.popover.top .arrow{bottom:0;left:50%;margin-left:-5px;border-top:5px solid #000;border-right:5px solid transparent;border-left:5px solid transparent}.popover.right .arrow{top:50%;left:0;margin-top:-5px;border-top:5px solid transparent;border-right:5px solid #000;border-bottom:5px solid transparent}.popover.bottom .arrow{top:0;left:50%;margin-left:-5px;border-right:5px solid transparent;border-bottom:5px solid #000;border-left:5px solid transparent}.popover.left .arrow{top:50%;right:0;margin-top:-5px;border-top:5px solid transparent;border-bottom:5px solid transparent;border-left:5px solid #000}.popover .arrow{position:absolute;width:0;height:0}.popover-inner{width:280px;padding:3px;overflow:hidden;background:#000;background:rgba(0,0,0,0.8);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3)}.popover-title{padding:9px 15px;line-height:1;background-color:#f5f5f5;border-bottom:1px solid #eee;-webkit-border-radius:3px 3px 0 0;-moz-border-radius:3px 3px 0 0;border-radius:3px 3px 0 0}.popover-content{padding:14px;background-color:#fff;-webkit-border-radius:0 0 3px 3px;-moz-border-radius:0 0 3px 3px;border-radius:0 0 3px 3px;-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:18px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:1;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:0 1px 1px rgba(0,0,0,0.075);box-shadow:0 1px 1px rgba(0,0,0,0.075)}a.thumbnail:hover{border-color:#08c;-webkit-box-shadow:0 1px 4px rgba(0,105,214,0.25);-moz-box-shadow:0 1px 4px rgba(0,105,214,0.25);box-shadow:0 1px 4px rgba(0,105,214,0.25)}.thumbnail>img{display:block;max-width:100%;margin-right:auto;margin-left:auto}.thumbnail .caption{padding:9px}.label,.badge{font-size:10.998px;font-weight:bold;line-height:14px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);white-space:nowrap;vertical-align:baseline;background-color:#999}.label{padding:1px 4px 2px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.badge{padding:1px 9px 2px;-webkit-border-radius:9px;-moz-border-radius:9px;border-radius:9px}a.label:hover,a.badge:hover{color:#fff;text-decoration:none;cursor:pointer}.label-important,.badge-important{background-color:#b94a48}.label-important[href],.badge-important[href]{background-color:#953b39}.label-warning,.badge-warning{background-color:#f89406}.label-warning[href],.badge-warning[href]{background-color:#c67605}.label-success,.badge-success{background-color:#468847}.label-success[href],.badge-success[href]{background-color:#356635}.label-info,.badge-info{background-color:#3a87ad}.label-info[href],.badge-info[href]{background-color:#2d6987}.label-inverse,.badge-inverse{background-color:#333}.label-inverse[href],.badge-inverse[href]{background-color:#1a1a1a}@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-moz-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-ms-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-o-keyframes progress-bar-stripes{from{background-position:0 0}to{background-position:40px 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}.progress{height:18px;margin-bottom:18px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-ms-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f5f5f5),to(#f9f9f9));background-image:-webkit-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-o-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:linear-gradient(top,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#f5f5f5',endColorstr='#f9f9f9',GradientType=0);-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1)}.progress .bar{width:0;height:18px;font-size:12px;color:#fff;text-align:center;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e90d2;background-image:-moz-linear-gradient(top,#149bdf,#0480be);background-image:-webkit-gradient(linear,0 0,0 100%,from(#149bdf),to(#0480be));background-image:-webkit-linear-gradient(top,#149bdf,#0480be);background-image:-o-linear-gradient(top,#149bdf,#0480be);background-image:linear-gradient(top,#149bdf,#0480be);background-image:-ms-linear-gradient(top,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#149bdf',endColorstr='#0480be',GradientType=0);-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-webkit-box-sizing:border-box;-moz-box-sizing:border-box;-ms-box-sizing:border-box;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-ms-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress-striped .bar{background-color:#149bdf;background-image:-o-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-webkit-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-ms-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);-webkit-background-size:40px 40px;-moz-background-size:40px 40px;-o-background-size:40px 40px;background-size:40px 40px}.progress.active .bar{-webkit-animation:progress-bar-stripes 2s linear infinite;-moz-animation:progress-bar-stripes 2s linear infinite;-ms-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite}.progress-danger .bar{background-color:#dd514c;background-image:-moz-linear-gradient(top,#ee5f5b,#c43c35);background-image:-ms-linear-gradient(top,#ee5f5b,#c43c35);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#c43c35));background-image:-webkit-linear-gradient(top,#ee5f5b,#c43c35);background-image:-o-linear-gradient(top,#ee5f5b,#c43c35);background-image:linear-gradient(top,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ee5f5b',endColorstr='#c43c35',GradientType=0)}.progress-danger.progress-striped .bar{background-color:#ee5f5b;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-ms-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-success .bar{background-color:#5eb95e;background-image:-moz-linear-gradient(top,#62c462,#57a957);background-image:-ms-linear-gradient(top,#62c462,#57a957);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#57a957));background-image:-webkit-linear-gradient(top,#62c462,#57a957);background-image:-o-linear-gradient(top,#62c462,#57a957);background-image:linear-gradient(top,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#62c462',endColorstr='#57a957',GradientType=0)}.progress-success.progress-striped .bar{background-color:#62c462;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-ms-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-info .bar{background-color:#4bb1cf;background-image:-moz-linear-gradient(top,#5bc0de,#339bb9);background-image:-ms-linear-gradient(top,#5bc0de,#339bb9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#339bb9));background-image:-webkit-linear-gradient(top,#5bc0de,#339bb9);background-image:-o-linear-gradient(top,#5bc0de,#339bb9);background-image:linear-gradient(top,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#5bc0de',endColorstr='#339bb9',GradientType=0)}.progress-info.progress-striped .bar{background-color:#5bc0de;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-ms-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-warning .bar{background-color:#faa732;background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:-ms-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(top,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fbb450',endColorstr='#f89406',GradientType=0)}.progress-warning.progress-striped .bar{background-color:#fbb450;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-ms-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(-45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.accordion{margin-bottom:18px}.accordion-group{margin-bottom:2px;border:1px solid #e5e5e5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.accordion-heading{border-bottom:0}.accordion-heading .accordion-toggle{display:block;padding:8px 15px}.accordion-toggle{cursor:pointer}.accordion-inner{padding:9px 15px;border-top:1px solid #e5e5e5}.carousel{position:relative;margin-bottom:18px;line-height:1}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel .item{position:relative;display:none;-webkit-transition:.6s ease-in-out left;-moz-transition:.6s ease-in-out left;-ms-transition:.6s ease-in-out left;-o-transition:.6s ease-in-out left;transition:.6s ease-in-out left}.carousel .item>img{display:block;line-height:1}.carousel .active,.carousel .next,.carousel .prev{display:block}.carousel .active{left:0}.carousel .next,.carousel .prev{position:absolute;top:0;width:100%}.carousel .next{left:100%}.carousel .prev{left:-100%}.carousel .next.left,.carousel .prev.right{left:0}.carousel .active.left{left:-100%}.carousel .active.right{left:100%}.carousel-control{position:absolute;top:40%;left:15px;width:40px;height:40px;margin-top:-20px;font-size:60px;font-weight:100;line-height:30px;color:#fff;text-align:center;background:#222;border:3px solid #fff;-webkit-border-radius:23px;-moz-border-radius:23px;border-radius:23px;opacity:.5;filter:alpha(opacity=50)}.carousel-control.right{right:15px;left:auto}.carousel-control:hover{color:#fff;text-decoration:none;opacity:.9;filter:alpha(opacity=90)}.carousel-caption{position:absolute;right:0;bottom:0;left:0;padding:10px 15px 5px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{color:#fff}.hero-unit{padding:60px;margin-bottom:30px;background-color:#eee;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.hero-unit h1{margin-bottom:0;font-size:60px;line-height:1;letter-spacing:-1px;color:inherit}.hero-unit p{font-size:18px;font-weight:200;line-height:27px;color:inherit}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden} diff --git a/core/src/main/resources/spark/ui/static/sorttable.js b/core/src/main/resources/spark/ui/static/sorttable.js new file mode 100644 index 0000000000..7abb9011cc --- /dev/null +++ b/core/src/main/resources/spark/ui/static/sorttable.js @@ -0,0 +1,495 @@ +/* + SortTable + version 2 + 7th April 2007 + Stuart Langridge, http://www.kryogenix.org/code/browser/sorttable/ + + Instructions: + Download this file + Add to your HTML + Add class="sortable" to any table you'd like to make sortable + Click on the headers to sort + + Thanks to many, many people for contributions and suggestions. + Licenced as X11: http://www.kryogenix.org/code/browser/licence.html + This basically means: do what you want with it. +*/ + + +var stIsIE = /*@cc_on!@*/false; + +sorttable = { + init: function() { + // quit if this function has already been called + if (arguments.callee.done) return; + // flag this function so we don't do the same thing twice + arguments.callee.done = true; + // kill the timer + if (_timer) clearInterval(_timer); + + if (!document.createElement || !document.getElementsByTagName) return; + + sorttable.DATE_RE = /^(\d\d?)[\/\.-](\d\d?)[\/\.-]((\d\d)?\d\d)$/; + + forEach(document.getElementsByTagName('table'), function(table) { + if (table.className.search(/\bsortable\b/) != -1) { + sorttable.makeSortable(table); + } + }); + + }, + + makeSortable: function(table) { + if (table.getElementsByTagName('thead').length == 0) { + // table doesn't have a tHead. Since it should have, create one and + // put the first table row in it. + the = document.createElement('thead'); + the.appendChild(table.rows[0]); + table.insertBefore(the,table.firstChild); + } + // Safari doesn't support table.tHead, sigh + if (table.tHead == null) table.tHead = table.getElementsByTagName('thead')[0]; + + if (table.tHead.rows.length != 1) return; // can't cope with two header rows + + // Sorttable v1 put rows with a class of "sortbottom" at the bottom (as + // "total" rows, for example). This is B&R, since what you're supposed + // to do is put them in a tfoot. So, if there are sortbottom rows, + // for backwards compatibility, move them to tfoot (creating it if needed). + sortbottomrows = []; + for (var i=0; i5' : ' ▴'; + this.appendChild(sortrevind); + return; + } + if (this.className.search(/\bsorttable_sorted_reverse\b/) != -1) { + // if we're already sorted by this column in reverse, just + // re-reverse the table, which is quicker + sorttable.reverse(this.sorttable_tbody); + this.className = this.className.replace('sorttable_sorted_reverse', + 'sorttable_sorted'); + this.removeChild(document.getElementById('sorttable_sortrevind')); + sortfwdind = document.createElement('span'); + sortfwdind.id = "sorttable_sortfwdind"; + sortfwdind.innerHTML = stIsIE ? ' 6' : ' ▾'; + this.appendChild(sortfwdind); + return; + } + + // remove sorttable_sorted classes + theadrow = this.parentNode; + forEach(theadrow.childNodes, function(cell) { + if (cell.nodeType == 1) { // an element + cell.className = cell.className.replace('sorttable_sorted_reverse',''); + cell.className = cell.className.replace('sorttable_sorted',''); + } + }); + sortfwdind = document.getElementById('sorttable_sortfwdind'); + if (sortfwdind) { sortfwdind.parentNode.removeChild(sortfwdind); } + sortrevind = document.getElementById('sorttable_sortrevind'); + if (sortrevind) { sortrevind.parentNode.removeChild(sortrevind); } + + this.className += ' sorttable_sorted'; + sortfwdind = document.createElement('span'); + sortfwdind.id = "sorttable_sortfwdind"; + sortfwdind.innerHTML = stIsIE ? ' 6' : ' ▾'; + this.appendChild(sortfwdind); + + // build an array to sort. This is a Schwartzian transform thing, + // i.e., we "decorate" each row with the actual sort key, + // sort based on the sort keys, and then put the rows back in order + // which is a lot faster because you only do getInnerText once per row + row_array = []; + col = this.sorttable_columnindex; + rows = this.sorttable_tbody.rows; + for (var j=0; j 12) { + // definitely dd/mm + return sorttable.sort_ddmm; + } else if (second > 12) { + return sorttable.sort_mmdd; + } else { + // looks like a date, but we can't tell which, so assume + // that it's dd/mm (English imperialism!) and keep looking + sortfn = sorttable.sort_ddmm; + } + } + } + } + return sortfn; + }, + + getInnerText: function(node) { + // gets the text we want to use for sorting for a cell. + // strips leading and trailing whitespace. + // this is *not* a generic getInnerText function; it's special to sorttable. + // for example, you can override the cell text with a customkey attribute. + // it also gets .value for fields. + + if (!node) return ""; + + hasInputs = (typeof node.getElementsByTagName == 'function') && + node.getElementsByTagName('input').length; + + if (node.getAttribute("sorttable_customkey") != null) { + return node.getAttribute("sorttable_customkey"); + } + else if (typeof node.textContent != 'undefined' && !hasInputs) { + return node.textContent.replace(/^\s+|\s+$/g, ''); + } + else if (typeof node.innerText != 'undefined' && !hasInputs) { + return node.innerText.replace(/^\s+|\s+$/g, ''); + } + else if (typeof node.text != 'undefined' && !hasInputs) { + return node.text.replace(/^\s+|\s+$/g, ''); + } + else { + switch (node.nodeType) { + case 3: + if (node.nodeName.toLowerCase() == 'input') { + return node.value.replace(/^\s+|\s+$/g, ''); + } + case 4: + return node.nodeValue.replace(/^\s+|\s+$/g, ''); + break; + case 1: + case 11: + var innerText = ''; + for (var i = 0; i < node.childNodes.length; i++) { + innerText += sorttable.getInnerText(node.childNodes[i]); + } + return innerText.replace(/^\s+|\s+$/g, ''); + break; + default: + return ''; + } + } + }, + + reverse: function(tbody) { + // reverse the rows in a tbody + newrows = []; + for (var i=0; i=0; i--) { + tbody.appendChild(newrows[i]); + } + delete newrows; + }, + + /* sort functions + each sort function takes two parameters, a and b + you are comparing a[0] and b[0] */ + sort_numeric: function(a,b) { + aa = parseFloat(a[0].replace(/[^0-9.-]/g,'')); + if (isNaN(aa)) aa = 0; + bb = parseFloat(b[0].replace(/[^0-9.-]/g,'')); + if (isNaN(bb)) bb = 0; + return aa-bb; + }, + sort_alpha: function(a,b) { + if (a[0]==b[0]) return 0; + if (a[0] 0 ) { + var q = list[i]; list[i] = list[i+1]; list[i+1] = q; + swap = true; + } + } // for + t--; + + if (!swap) break; + + for(var i = t; i > b; --i) { + if ( comp_func(list[i], list[i-1]) < 0 ) { + var q = list[i]; list[i] = list[i-1]; list[i-1] = q; + swap = true; + } + } // for + b++; + + } // while(swap) + } +} + +/* ****************************************************************** + Supporting functions: bundled here to avoid depending on a library + ****************************************************************** */ + +// Dean Edwards/Matthias Miller/John Resig + +/* for Mozilla/Opera9 */ +if (document.addEventListener) { + document.addEventListener("DOMContentLoaded", sorttable.init, false); +} + +/* for Internet Explorer */ +/*@cc_on @*/ +/*@if (@_win32) + document.write(" to your HTML - Add class="sortable" to any table you'd like to make sortable - Click on the headers to sort - - Thanks to many, many people for contributions and suggestions. - Licenced as X11: http://www.kryogenix.org/code/browser/licence.html - This basically means: do what you want with it. -*/ - - -var stIsIE = /*@cc_on!@*/false; - -sorttable = { - init: function() { - // quit if this function has already been called - if (arguments.callee.done) return; - // flag this function so we don't do the same thing twice - arguments.callee.done = true; - // kill the timer - if (_timer) clearInterval(_timer); - - if (!document.createElement || !document.getElementsByTagName) return; - - sorttable.DATE_RE = /^(\d\d?)[\/\.-](\d\d?)[\/\.-]((\d\d)?\d\d)$/; - - forEach(document.getElementsByTagName('table'), function(table) { - if (table.className.search(/\bsortable\b/) != -1) { - sorttable.makeSortable(table); - } - }); - - }, - - makeSortable: function(table) { - if (table.getElementsByTagName('thead').length == 0) { - // table doesn't have a tHead. Since it should have, create one and - // put the first table row in it. - the = document.createElement('thead'); - the.appendChild(table.rows[0]); - table.insertBefore(the,table.firstChild); - } - // Safari doesn't support table.tHead, sigh - if (table.tHead == null) table.tHead = table.getElementsByTagName('thead')[0]; - - if (table.tHead.rows.length != 1) return; // can't cope with two header rows - - // Sorttable v1 put rows with a class of "sortbottom" at the bottom (as - // "total" rows, for example). This is B&R, since what you're supposed - // to do is put them in a tfoot. So, if there are sortbottom rows, - // for backwards compatibility, move them to tfoot (creating it if needed). - sortbottomrows = []; - for (var i=0; i5' : ' ▴'; - this.appendChild(sortrevind); - return; - } - if (this.className.search(/\bsorttable_sorted_reverse\b/) != -1) { - // if we're already sorted by this column in reverse, just - // re-reverse the table, which is quicker - sorttable.reverse(this.sorttable_tbody); - this.className = this.className.replace('sorttable_sorted_reverse', - 'sorttable_sorted'); - this.removeChild(document.getElementById('sorttable_sortrevind')); - sortfwdind = document.createElement('span'); - sortfwdind.id = "sorttable_sortfwdind"; - sortfwdind.innerHTML = stIsIE ? ' 6' : ' ▾'; - this.appendChild(sortfwdind); - return; - } - - // remove sorttable_sorted classes - theadrow = this.parentNode; - forEach(theadrow.childNodes, function(cell) { - if (cell.nodeType == 1) { // an element - cell.className = cell.className.replace('sorttable_sorted_reverse',''); - cell.className = cell.className.replace('sorttable_sorted',''); - } - }); - sortfwdind = document.getElementById('sorttable_sortfwdind'); - if (sortfwdind) { sortfwdind.parentNode.removeChild(sortfwdind); } - sortrevind = document.getElementById('sorttable_sortrevind'); - if (sortrevind) { sortrevind.parentNode.removeChild(sortrevind); } - - this.className += ' sorttable_sorted'; - sortfwdind = document.createElement('span'); - sortfwdind.id = "sorttable_sortfwdind"; - sortfwdind.innerHTML = stIsIE ? ' 6' : ' ▾'; - this.appendChild(sortfwdind); - - // build an array to sort. This is a Schwartzian transform thing, - // i.e., we "decorate" each row with the actual sort key, - // sort based on the sort keys, and then put the rows back in order - // which is a lot faster because you only do getInnerText once per row - row_array = []; - col = this.sorttable_columnindex; - rows = this.sorttable_tbody.rows; - for (var j=0; j 12) { - // definitely dd/mm - return sorttable.sort_ddmm; - } else if (second > 12) { - return sorttable.sort_mmdd; - } else { - // looks like a date, but we can't tell which, so assume - // that it's dd/mm (English imperialism!) and keep looking - sortfn = sorttable.sort_ddmm; - } - } - } - } - return sortfn; - }, - - getInnerText: function(node) { - // gets the text we want to use for sorting for a cell. - // strips leading and trailing whitespace. - // this is *not* a generic getInnerText function; it's special to sorttable. - // for example, you can override the cell text with a customkey attribute. - // it also gets .value for fields. - - if (!node) return ""; - - hasInputs = (typeof node.getElementsByTagName == 'function') && - node.getElementsByTagName('input').length; - - if (node.getAttribute("sorttable_customkey") != null) { - return node.getAttribute("sorttable_customkey"); - } - else if (typeof node.textContent != 'undefined' && !hasInputs) { - return node.textContent.replace(/^\s+|\s+$/g, ''); - } - else if (typeof node.innerText != 'undefined' && !hasInputs) { - return node.innerText.replace(/^\s+|\s+$/g, ''); - } - else if (typeof node.text != 'undefined' && !hasInputs) { - return node.text.replace(/^\s+|\s+$/g, ''); - } - else { - switch (node.nodeType) { - case 3: - if (node.nodeName.toLowerCase() == 'input') { - return node.value.replace(/^\s+|\s+$/g, ''); - } - case 4: - return node.nodeValue.replace(/^\s+|\s+$/g, ''); - break; - case 1: - case 11: - var innerText = ''; - for (var i = 0; i < node.childNodes.length; i++) { - innerText += sorttable.getInnerText(node.childNodes[i]); - } - return innerText.replace(/^\s+|\s+$/g, ''); - break; - default: - return ''; - } - } - }, - - reverse: function(tbody) { - // reverse the rows in a tbody - newrows = []; - for (var i=0; i=0; i--) { - tbody.appendChild(newrows[i]); - } - delete newrows; - }, - - /* sort functions - each sort function takes two parameters, a and b - you are comparing a[0] and b[0] */ - sort_numeric: function(a,b) { - aa = parseFloat(a[0].replace(/[^0-9.-]/g,'')); - if (isNaN(aa)) aa = 0; - bb = parseFloat(b[0].replace(/[^0-9.-]/g,'')); - if (isNaN(bb)) bb = 0; - return aa-bb; - }, - sort_alpha: function(a,b) { - if (a[0]==b[0]) return 0; - if (a[0] 0 ) { - var q = list[i]; list[i] = list[i+1]; list[i+1] = q; - swap = true; - } - } // for - t--; - - if (!swap) break; - - for(var i = t; i > b; --i) { - if ( comp_func(list[i], list[i-1]) < 0 ) { - var q = list[i]; list[i] = list[i-1]; list[i-1] = q; - swap = true; - } - } // for - b++; - - } // while(swap) - } -} - -/* ****************************************************************** - Supporting functions: bundled here to avoid depending on a library - ****************************************************************** */ - -// Dean Edwards/Matthias Miller/John Resig - -/* for Mozilla/Opera9 */ -if (document.addEventListener) { - document.addEventListener("DOMContentLoaded", sorttable.init, false); -} - -/* for Internet Explorer */ -/*@cc_on @*/ -/*@if (@_win32) - document.write(" + {title} + + + +
    + +
    +
    + +
    + +
    +
    +

    + {title} +

    +
    +
    +
    + {content}
    -
    -
      -
    • Master: {sc.master}
    • -
    • Application: {sc.appName}
    • -
    • Executors: {sc.getExecutorStorageStatus.size}
    • -
    -
    -
    -
    ; - sparkPage(newContent ++ content, title) + + } - /** Returns a page containing the supplied content and the spark css, js, and logo. */ - def sparkPage(content: => Seq[Node], title: String): Seq[Node] = { + /** Returns a page with the spark css/js and a simple format. Used for scheduler UI. */ + def basicSparkPage(content: => Seq[Node], title: String): Seq[Node] = { diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 9b34cdd27f..f5ed08744c 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -11,6 +11,7 @@ import scala.xml.{NodeSeq, Node} import spark.scheduler.Stage import spark.ui.UIUtils._ +import spark.ui.Page._ import spark.storage.StorageLevel /** Page showing list of all ongoing and recently finished stages */ @@ -33,7 +34,7 @@ private[spark] class IndexPage(parent: JobProgressUI) {

    Completed Stages

    ++ completedStageTable ++

    Failed Stages

    ++ failedStageTable - headerSparkPage(content, parent.sc, "Spark Stages") + headerSparkPage(content, parent.sc, "Spark Stages", Jobs) } def getElapsedTime(submitted: Option[Long], completed: Long): String = { diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 518f48cb81..c9294a7261 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -7,6 +7,7 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import spark.ui.UIUtils._ +import spark.ui.Page._ import spark.util.Distribution import spark.Utils import spark.scheduler.cluster.TaskInfo @@ -26,7 +27,7 @@ private[spark] class StagePage(parent: JobProgressUI) {

    Summary Metrics

    No tasks have finished yet

    Tasks

    No tasks have finished yet
    - return headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId)) + return headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId), Jobs) } val tasks = listener.stageToTaskInfos(stageId) @@ -70,7 +71,7 @@ private[spark] class StagePage(parent: JobProgressUI) { val content =

    Summary Metrics

    ++ quantileTable ++

    Tasks

    ++ taskTable; - headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId)) + headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId), Jobs) } def quantileRow(data: Seq[String]): Seq[Node] = {data.map(d => {d})} diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index 772c669a1a..d284134391 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -7,6 +7,7 @@ import scala.xml.Node import spark.storage.{RDDInfo, StorageUtils} import spark.Utils import spark.ui.UIUtils._ +import spark.ui.Page._ /** Page showing list of RDD's currently stored in the cluster */ private[spark] class IndexPage(parent: BlockManagerUI) { @@ -42,7 +43,7 @@ private[spark] class IndexPage(parent: BlockManagerUI) {
    ++ {rddTable}; - headerSparkPage(content, parent.sc, "Spark Storage ") + headerSparkPage(content, parent.sc, "Spark Storage ", Storage) } def rddRow(rdd: RDDInfo): Seq[Node] = { diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index dd9fd2e4c5..65952f711a 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -8,6 +8,7 @@ import spark.storage.{StorageStatus, StorageUtils} import spark.ui.UIUtils._ import spark.Utils import spark.storage.BlockManagerMasterActor.BlockStatus +import spark.ui.Page._ /** Page showing storage details for a given RDD */ private[spark] class RDDPage(parent: BlockManagerUI) { @@ -70,7 +71,7 @@ private[spark] class RDDPage(parent: BlockManagerUI) { ; - headerSparkPage(content, parent.sc, "RDD Info: " + rddInfo.name) + headerSparkPage(content, parent.sc, "RDD Info: " + rddInfo.name, Jobs) } def blockRow(blk: (String, BlockStatus)): Seq[Node] = { -- cgit v1.2.3 From 62c2c6b8561a00b2103fe5fb80b42b9975a5b8db Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 27 Jun 2013 21:47:22 -0700 Subject: Forcing Jetty to run as daemon --- core/src/main/scala/spark/ui/JettyUtils.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 5e20f41501..8cc50bc5b0 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -14,6 +14,7 @@ import scala.xml.Node import spark.{SparkContext, Logging} import org.eclipse.jetty.util.log.Log +import org.eclipse.jetty.util.thread.QueuedThreadPool /** Utilities for launching a web server using Jetty's HTTP Server class */ private[spark] object JettyUtils extends Logging { @@ -93,9 +94,15 @@ private[spark] object JettyUtils extends Logging { @tailrec def connect(currentPort: Int): (Server, Int) = { val server = new Server(currentPort) + val pool = new QueuedThreadPool + pool.setDaemon(true) + server.setThreadPool(pool) server.setHandler(handlerList) + Try { server.start() } match { - case s: Success[_] => (server, server.getConnectors.head.getLocalPort) + case s: Success[_] => + sys.addShutdownHook(server.stop()) // Be kind, un-bind + (server, server.getConnectors.head.getLocalPort) case f: Failure[_] => server.stop() logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) @@ -103,6 +110,8 @@ private[spark] object JettyUtils extends Logging { connect((currentPort + 1) % 65536) } } + + connect(port) } } -- cgit v1.2.3 From c767e7437059aa35bca3e8bb93264b35853c7a8f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 27 Jun 2013 21:48:58 -0700 Subject: Removing incorrect test statement --- core/src/test/scala/spark/scheduler/JobLoggerSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala index 4000c4d520..699901f1a1 100644 --- a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala @@ -41,7 +41,6 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers val rootStage = new Stage(0, rootRdd, None, List(shuffleMapStage), jobID) joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStage, 4)) - joblogger.getEventQueue.size should be (1) joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getName) parentRdd.setName("MyRDD") joblogger.getRddNameTest(parentRdd) should be ("MyRDD") -- cgit v1.2.3 From c537e869f3a12e75fac862cc6fdcf27a3e1e01b4 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 27 Jun 2013 22:02:03 -0700 Subject: Missing logo file --- .../resources/spark/ui/static/spark-logo-77x50px-hd.png | Bin 0 -> 3536 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 core/src/main/resources/spark/ui/static/spark-logo-77x50px-hd.png diff --git a/core/src/main/resources/spark/ui/static/spark-logo-77x50px-hd.png b/core/src/main/resources/spark/ui/static/spark-logo-77x50px-hd.png new file mode 100644 index 0000000000..6c5f0993c4 Binary files /dev/null and b/core/src/main/resources/spark/ui/static/spark-logo-77x50px-hd.png differ -- cgit v1.2.3 From 4974b658edf2716ff3c6f2e6863cddb2a4ddf891 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 27 Jun 2013 22:16:40 -0700 Subject: Look at JAVA_HOME before PATH to determine Java executable --- run | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/run b/run index 7c06a55062..805466ea2c 100755 --- a/run +++ b/run @@ -67,14 +67,15 @@ if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then fi fi else - if [ `command -v java` ]; then - RUNNER="java" + if [ -n "${JAVA_HOME}" ]; then + RUNNER="${JAVA_HOME}/bin/java" else - if [ -z "$JAVA_HOME" ]; then + if [ `command -v java` ]; then + RUNNER="java" + else echo "JAVA_HOME is not set" >&2 exit 1 fi - RUNNER="${JAVA_HOME}/bin/java" fi if [ -z "$SCALA_LIBRARY_PATH" ]; then if [ -z "$SCALA_HOME" ]; then -- cgit v1.2.3 From 249f0e54ba6c358b3d786b1ba42e5d2ea22ed9db Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 28 Jun 2013 13:25:26 -0700 Subject: Minor changes from Matei's review --- core/src/main/scala/spark/ui/JettyUtils.scala | 2 +- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 21 +++++++++------------ .../main/scala/spark/ui/jobs/JobProgressUI.scala | 2 +- 3 files changed, 11 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 8cc50bc5b0..9e4b884379 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -160,8 +160,8 @@ private[spark] object UIUtils { {jobs}
      -
    • Master: {sc.master}
    • Application: {sc.appName}
    • +
    • Master: {sc.master}
    • Executors: {sc.getExecutorStorageStatus.size}
    diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index f5ed08744c..27c017d7ee 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -1,7 +1,5 @@ package spark.ui.jobs -import akka.util.Duration - import java.util.Date import javax.servlet.http.HttpServletRequest @@ -26,9 +24,9 @@ private[spark] class IndexPage(parent: JobProgressUI) { val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq - val activeStageTable: NodeSeq = listingTable(stageHeaders, stageRow(), activeStages) - val completedStageTable = listingTable(stageHeaders, stageRow(), completedStages) - val failedStageTable: NodeSeq = listingTable(stageHeaders, stageRow(false), failedStages) + val activeStageTable: NodeSeq = listingTable(stageHeaders, stageRow, activeStages) + val completedStageTable = listingTable(stageHeaders, stageRow, completedStages) + val failedStageTable: NodeSeq = listingTable(stageHeaders, stageRow, failedStages) val content =

    Active Stages

    ++ activeStageTable ++

    Completed Stages

    ++ completedStageTable ++ @@ -44,7 +42,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { } } - def makeSlider(completed: Int, total: Int): Seq[Node] = { + def makeProgressBar(completed: Int, total: Int): Seq[Node] = { val width=130 val height=15 val completeWidth = (completed.toDouble / total) * width @@ -53,12 +51,12 @@ private[spark] class IndexPage(parent: JobProgressUI) { + fill="rgb(51,51,51)" stroke="black" stroke-width="1" /> } - def stageRow(showLink: Boolean = true)(s: Stage): Seq[Node] = { + def stageRow(s: Stage): Seq[Node] = { val submissionTime = s.submissionTime match { case Some(t) => dateFmt.format(new Date(t)) case None => "Unknown" @@ -74,13 +72,12 @@ private[spark] class IndexPage(parent: JobProgressUI) { val totalTasks = s.numPartitions - {if (showLink) {{s.id}} - else {{s.id}}} - {s.origin} + {s.id} + {s.origin} {submissionTime} {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))} - {makeSlider(completedTasks, totalTasks)} + {makeProgressBar(completedTasks, totalTasks)} {completedTasks} / {totalTasks} {listener.stageToTasksFailed.getOrElse(s.id, 0) match { case f if f > 0 => "(%s failed)".format(f) diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index a18bf0f81e..f584d1e187 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -23,7 +23,7 @@ import spark.Utils private[spark] class JobProgressUI(val sc: SparkContext) { private var _listener: Option[JobProgressListener] = None def listener = _listener.get - val dateFmt = new SimpleDateFormat("EEE, MMM d yyyy HH:mm:ss") + val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") private val indexPage = new IndexPage(this) private val stagePage = new StagePage(this) -- cgit v1.2.3 From 8113c55df8f4b5f34140ddba5e58e132e3dc2d23 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Fri, 28 Jun 2013 13:46:21 -0700 Subject: [Feedback] Get rid of -m, set MASTER from SPARK_MASTER_IP/PORT automagically --- spark-shell | 25 +++++++++++++------------ 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/spark-shell b/spark-shell index ea67a3e6b8..a8e72143fb 100755 --- a/spark-shell +++ b/spark-shell @@ -1,24 +1,14 @@ #!/bin/bash --posix # # Shell script for starting the Spark Shell REPL +# Note that it will set MASTER to spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT} +# if those two env vars are set in spark-env.sh but MASTER is not. # Options: -# -m Set MASTER to spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT # -c Set the number of cores for REPL to use # FWDIR="`dirname $0`" for o in "$@"; do - if [ "$1" = "-m" -o "$1" = "--master" ]; then - shift - if [ -e "$FWDIR/conf/spark-env.sh" ]; then - . "$FWDIR/conf/spark-env.sh" - fi - if [ -z "$MASTER" ]; then - MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" - fi - export MASTER - fi - if [ "$1" = "-c" -o "$1" = "--cores" ]; then shift if [ -n "$1" ]; then @@ -28,6 +18,17 @@ for o in "$@"; do fi done +# Set MASTER from spark-env if possible +if [ -z "$MASTER" ]; then + if [ -e "$FWDIR/conf/spark-env.sh" ]; then + . "$FWDIR/conf/spark-env.sh" + fi + if [[ "x" != "x$SPARK_MASTER_IP" && "y" != "y$SPARK_MASTER_PORT" ]]; then + MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" + export MASTER + fi +fi + # Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in # binary distribution of Spark where Scala is not installed exit_status=127 -- cgit v1.2.3 From 473961d82e70f4cdb9e3d12b0dfcb4b70e6121ab Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 29 Jun 2013 08:38:04 -0700 Subject: Styling for progress bar --- core/src/main/resources/spark/ui/static/webui.css | 8 +++++++ core/src/main/scala/spark/ui/jobs/IndexPage.scala | 28 +++++++++++++++++++---- 2 files changed, 31 insertions(+), 5 deletions(-) diff --git a/core/src/main/resources/spark/ui/static/webui.css b/core/src/main/resources/spark/ui/static/webui.css index c1ad4dbb17..f7537bb766 100644 --- a/core/src/main/resources/spark/ui/static/webui.css +++ b/core/src/main/resources/spark/ui/static/webui.css @@ -39,3 +39,11 @@ padding-top: 10px; padding-bottom: 10px; } + +.progress-cell { + width: 134px; + border-right: 0; + padding: 0; + padding-top: 7px; + padding-left: 4px; +} diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 27c017d7ee..cd04542faa 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -24,9 +24,27 @@ private[spark] class IndexPage(parent: JobProgressUI) { val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq - val activeStageTable: NodeSeq = listingTable(stageHeaders, stageRow, activeStages) - val completedStageTable = listingTable(stageHeaders, stageRow, completedStages) - val failedStageTable: NodeSeq = listingTable(stageHeaders, stageRow, failedStages) + /** Special table which merges two header cells. */ + def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { + + + + + + + + + + + + {rows.map(r => makeRow(r))} + +
    Stage IdOriginSubmittedDurationTasks: Complete/TotalShuffle ActivityStored RDD
    + } + + val activeStageTable: NodeSeq = stageTable(stageRow, activeStages) + val completedStageTable = stageTable(stageRow, completedStages) + val failedStageTable: NodeSeq = stageTable(stageRow, failedStages) val content =

    Active Stages

    ++ activeStageTable ++

    Completed Stages

    ++ completedStageTable ++ @@ -77,8 +95,8 @@ private[spark] class IndexPage(parent: JobProgressUI) { {submissionTime} {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))} - {makeProgressBar(completedTasks, totalTasks)} - {completedTasks} / {totalTasks} + {makeProgressBar(completedTasks, totalTasks)} + {completedTasks} / {totalTasks} {listener.stageToTasksFailed.getOrElse(s.id, 0) match { case f if f > 0 => "(%s failed)".format(f) case _ => -- cgit v1.2.3 From e721ff7e5a2d738750c40e95d6ba53898eaa7051 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 29 Jun 2013 11:26:30 -0700 Subject: Allowing details for failed stages --- .../main/scala/spark/scheduler/DAGScheduler.scala | 6 +- .../scheduler/cluster/ClusterTaskSetManager.scala | 1 + .../scheduler/local/LocalTaskSetManager.scala | 1 + .../main/scala/spark/ui/jobs/JobProgressUI.scala | 30 ++++---- core/src/main/scala/spark/ui/jobs/StagePage.scala | 87 +++++++++++++--------- 5 files changed, 74 insertions(+), 51 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 82d419453b..d9ddc41aa2 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -618,8 +618,11 @@ class DAGScheduler( handleExecutorLost(bmAddress.executorId, Some(task.generation)) } + case ExceptionFailure(className, description, stackTrace) => + // Do nothing here, left up to the TaskScheduler to decide how to handle user failures + case other => - // Non-fetch failure -- probably a bug in user code; abort all jobs depending on this stage + // Unrecognized failure - abort all jobs depending on this stage abortStage(idToStage(task.stageId), task + " failed: " + other) } } @@ -667,6 +670,7 @@ class DAGScheduler( */ private def abortStage(failedStage: Stage, reason: String) { val dependentStages = resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage)).toSeq + failedStage.completionTime = Some(System.currentTimeMillis()) for (resultStage <- dependentStages) { val job = resultStageToJob(resultStage) val error = new SparkException("Job failed: " + reason) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index d72b0bfc9f..6965cde5da 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -571,6 +571,7 @@ private[spark] class ClusterTaskSetManager( return case ef: ExceptionFailure => + sched.listener.taskEnded(tasks(index), ef, null, null, info, null) val key = ef.description val now = System.currentTimeMillis val (printFull, dupCount) = { diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index 70b69bb26f..499116f653 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -152,6 +152,7 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas info.markFailed() decreaseRunningTasks(1) val reason: ExceptionFailure = ser.deserialize[ExceptionFailure](serializedData, getClass.getClassLoader) + sched.listener.taskEnded(task, reason, null, null, info, null) if (!finished(index)) { copiesRunning(index) -= 1 numFailures(index) += 1 diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index f584d1e187..aafa414055 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -12,12 +12,11 @@ import scala.Seq import scala.collection.mutable.{HashSet, ListBuffer, HashMap, ArrayBuffer} import spark.ui.JettyUtils._ -import spark.SparkContext +import spark.{ExceptionFailure, SparkContext, Success, Utils} import spark.scheduler._ import spark.scheduler.cluster.TaskInfo import spark.executor.TaskMetrics -import spark.Success -import spark.Utils +import collection.mutable /** Web UI showing progress status of all jobs in the given SparkContext. */ private[spark] class JobProgressUI(val sc: SparkContext) { @@ -51,7 +50,8 @@ private[spark] class JobProgressListener extends SparkListener { val stageToTasksComplete = HashMap[Int, Int]() val stageToTasksFailed = HashMap[Int, Int]() - val stageToTaskInfos = HashMap[Int, ArrayBuffer[(TaskInfo, TaskMetrics)]]() + val stageToTaskInfos = + HashMap[Int, ArrayBuffer[(TaskInfo, TaskMetrics, Option[ExceptionFailure])]]() override def onJobStart(jobStart: SparkListenerJobStart) {} @@ -67,8 +67,6 @@ private[spark] class JobProgressListener extends SparkListener { if (stages.size > RETAINED_STAGES) { val toRemove = RETAINED_STAGES / 10 stages.takeRight(toRemove).foreach( s => { - stageToTasksComplete.remove(s.id) - stageToTasksFailed.remove(s.id) stageToTaskInfos.remove(s.id) }) stages.trimEnd(toRemove) @@ -80,14 +78,18 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId - taskEnd.reason match { - case Success => - stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 - case _ => - stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 - } - val taskList = stageToTaskInfos.getOrElse(sid, ArrayBuffer[(TaskInfo, TaskMetrics)]()) - taskList += ((taskEnd.taskInfo, taskEnd.taskMetrics)) + val failureInfo: Option[ExceptionFailure] = + taskEnd.reason match { + case e: ExceptionFailure => + stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 + Some(e) + case _ => + stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 + None + } + val taskList = stageToTaskInfos.getOrElse( + sid, ArrayBuffer[(TaskInfo, TaskMetrics, Option[ExceptionFailure])]()) + taskList += ((taskEnd.taskInfo, taskEnd.taskMetrics, failureInfo)) stageToTaskInfos(sid) = taskList } diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index c9294a7261..ed96fc2994 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -9,7 +9,7 @@ import scala.xml.Node import spark.ui.UIUtils._ import spark.ui.Page._ import spark.util.Distribution -import spark.Utils +import spark.{ExceptionFailure, Utils} import spark.scheduler.cluster.TaskInfo import spark.executor.TaskMetrics @@ -38,56 +38,71 @@ private[spark] class StagePage(parent: JobProgressUI) { val taskHeaders: Seq[String] = Seq("Task ID", "Duration", "Locality Level", "Worker", "Launch Time") ++ {if (shuffleRead) Seq("Shuffle Read") else Nil} ++ - {if (shuffleWrite) Seq("Shuffle Write") else Nil} + {if (shuffleWrite) Seq("Shuffle Write") else Nil} ++ + Seq("Details") val taskTable = listingTable(taskHeaders, taskRow, tasks) - val serviceTimes = tasks.map{case (info, metrics) => metrics.executorRunTime.toDouble} - val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles().map( - ms => parent.formatDuration(ms.toLong)) - - def getQuantileCols(data: Seq[Double]) = - Distribution(data).get.getQuantiles().map(d => Utils.memoryBytesToString(d.toLong)) - - val shuffleReadSizes = tasks.map { - case(info, metrics) => - metrics.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble - } - val shuffleReadQuantiles = "Shuffle Read (Remote)" +: getQuantileCols(shuffleReadSizes) - - val shuffleWriteSizes = tasks.map { - case(info, metrics) => - metrics.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble - } - val shuffleWriteQuantiles = "Shuffle Write" +: getQuantileCols(shuffleWriteSizes) - - val listings: Seq[Seq[String]] = Seq(serviceQuantiles, - if (shuffleRead) shuffleReadQuantiles else Nil, - if (shuffleWrite) shuffleWriteQuantiles else Nil) - - val quantileHeaders = Seq("Metric", "Min", "25%", "50%", "75%", "Max") - val quantileTable = listingTable(quantileHeaders, quantileRow, listings) + // Excludes tasks which failed and have incomplete metrics + val validTasks = tasks.filter(t => Option(t._2).isDefined) + + val summaryTable: Option[Seq[Node]] = + if (validTasks.size == 0) { + None + } + else { + val serviceTimes = validTasks.map{case (info, metrics, exception) => + metrics.executorRunTime.toDouble} + val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles().map( + ms => parent.formatDuration(ms.toLong)) + + def getQuantileCols(data: Seq[Double]) = + Distribution(data).get.getQuantiles().map(d => Utils.memoryBytesToString(d.toLong)) + + val shuffleReadSizes = validTasks.map { + case(info, metrics, exception) => + metrics.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble + } + val shuffleReadQuantiles = "Shuffle Read (Remote)" +: getQuantileCols(shuffleReadSizes) + + val shuffleWriteSizes = validTasks.map { + case(info, metrics, exception) => + metrics.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble + } + val shuffleWriteQuantiles = "Shuffle Write" +: getQuantileCols(shuffleWriteSizes) + + val listings: Seq[Seq[String]] = Seq(serviceQuantiles, + if (shuffleRead) shuffleReadQuantiles else Nil, + if (shuffleWrite) shuffleWriteQuantiles else Nil) + + val quantileHeaders = Seq("Metric", "Min", "25%", "50%", "75%", "Max") + def quantileRow(data: Seq[String]): Seq[Node] = {data.map(d => {d})} + Some(listingTable(quantileHeaders, quantileRow, listings)) + } val content = -

    Summary Metrics

    ++ quantileTable ++

    Tasks

    ++ taskTable; +

    Summary Metrics

    ++ summaryTable.getOrElse(Nil) ++

    Tasks

    ++ taskTable; headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId), Jobs) } - def quantileRow(data: Seq[String]): Seq[Node] = {data.map(d => {d})} - def taskRow(taskData: (TaskInfo, TaskMetrics)): Seq[Node] = { - val (info, metrics) = taskData + def taskRow(taskData: (TaskInfo, TaskMetrics, Option[ExceptionFailure])): Seq[Node] = { + def fmtStackTrace(trace: Seq[StackTraceElement]): Seq[Node] = + trace.map(e => {e.toString}) + val (info, metrics, exception) = taskData {info.taskId} - {parent.formatDuration(metrics.executorRunTime)} + {Option(metrics).map{m => parent.formatDuration(m.executorRunTime)}.getOrElse("")} {info.taskLocality} {info.hostPort} {dateFmt.format(new Date(info.launchTime))} - {metrics.shuffleReadMetrics.map{m => - {Utils.memoryBytesToString(m.remoteBytesRead)}}.getOrElse("") } - {metrics.shuffleWriteMetrics.map{m => - {Utils.memoryBytesToString(m.shuffleBytesWritten)}}.getOrElse("") } + {Option(metrics).flatMap{m => m.shuffleReadMetrics}.map{s => + {Utils.memoryBytesToString(s.remoteBytesRead)}}.getOrElse("")} + {Option(metrics).flatMap{m => m.shuffleWriteMetrics}.map{s => + {Utils.memoryBytesToString(s.shuffleBytesWritten)}}.getOrElse("")} + {exception.map(e => + {e.className}
    {fmtStackTrace(e.stackTrace)}
    ).getOrElse("")} } } -- cgit v1.2.3 From 4358acfe07e991090fbe009aafe3f5110fbf0c40 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 29 Jun 2013 15:25:06 -0700 Subject: Initialize Twitter4J OAuth from system properties instead of prompting --- .../scala/spark/streaming/StreamingContext.scala | 4 ++- .../streaming/api/java/JavaStreamingContext.scala | 23 +++++++--------- .../streaming/dstream/TwitterInputDStream.scala | 32 ++++++---------------- .../test/java/spark/streaming/JavaAPISuite.java | 2 +- 4 files changed, 23 insertions(+), 38 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index e61438fe3a..36b841af8f 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -381,7 +381,9 @@ class StreamingContext private ( /** * Create a input stream that returns tweets received from Twitter. - * @param twitterAuth Twitter4J authentication + * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth + * authorization; this uses the system properties twitter4j.oauth.consumerKey, + * .consumerSecret, .accessToken and .accessTokenSecret. * @param filters Set of filter strings to get only those tweets that match them * @param storageLevel Storage level to use for storing the received objects */ diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index c4a223b419..ed7b789d98 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -307,7 +307,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Create a input stream that returns tweets received from Twitter. - * @param twitterAuth Twitter4J Authorization + * @param twitterAuth Twitter4J Authorization object * @param filters Set of filter strings to get only those tweets that match them * @param storageLevel Storage level to use for storing the received objects */ @@ -320,10 +320,9 @@ class JavaStreamingContext(val ssc: StreamingContext) { } /** - * Create a input stream that returns tweets received from Twitter using - * java.util.Preferences to store OAuth token. OAuth key and secret should - * be provided using system properties twitter4j.oauth.consumerKey and - * twitter4j.oauth.consumerSecret + * Create a input stream that returns tweets received from Twitter using Twitter4J's default + * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, + * .consumerSecret, .accessToken and .accessTokenSecret to be set. * @param filters Set of filter strings to get only those tweets that match them * @param storageLevel Storage level to use for storing the received objects */ @@ -347,10 +346,9 @@ class JavaStreamingContext(val ssc: StreamingContext) { } /** - * Create a input stream that returns tweets received from Twitter using - * java.util.Preferences to store OAuth token. OAuth key and secret should - * be provided using system properties twitter4j.oauth.consumerKey and - * twitter4j.oauth.consumerSecret + * Create a input stream that returns tweets received from Twitter using Twitter4J's default + * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, + * .consumerSecret, .accessToken and .accessTokenSecret to be set. * @param filters Set of filter strings to get only those tweets that match them */ def twitterStream( @@ -370,10 +368,9 @@ class JavaStreamingContext(val ssc: StreamingContext) { } /** - * Create a input stream that returns tweets received from Twitter using - * java.util.Preferences to store OAuth token. OAuth key and secret should - * be provided using system properties twitter4j.oauth.consumerKey and - * twitter4j.oauth.consumerSecret + * Create a input stream that returns tweets received from Twitter using Twitter4J's default + * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, + * .consumerSecret, .accessToken and .accessTokenSecret to be set. */ def twitterStream(): JavaDStream[Status] = { ssc.twitterStream() diff --git a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala index e0c654d385..ff7a58be45 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala @@ -4,21 +4,21 @@ import spark._ import spark.streaming._ import storage.StorageLevel import twitter4j._ -import twitter4j.auth.BasicAuthorization import twitter4j.auth.Authorization import java.util.prefs.Preferences +import twitter4j.conf.ConfigurationBuilder import twitter4j.conf.PropertyConfiguration import twitter4j.auth.OAuthAuthorization import twitter4j.auth.AccessToken /* A stream of Twitter statuses, potentially filtered by one or more keywords. * -* @constructor create a new Twitter stream using the supplied username and password to authenticate. +* @constructor create a new Twitter stream using the supplied Twitter4J authentication credentials. * An optional set of string filters can be used to restrict the set of tweets. The Twitter API is * such that this may return a sampled subset of all tweets during each interval. * -* Includes a simple implementation of OAuth using consumer key and secret provided using system -* properties twitter4j.oauth.consumerKey and twitter4j.oauth.consumerSecret +* If no Authorization object is provided, initializes OAuth authorization using the system +* properties twitter4j.oauth.consumerKey, .consumerSecret, .accessToken and .accessTokenSecret. */ private[streaming] class TwitterInputDStream( @@ -28,28 +28,14 @@ class TwitterInputDStream( storageLevel: StorageLevel ) extends NetworkInputDStream[Status](ssc_) { - lazy val createOAuthAuthorization: Authorization = { - val userRoot = Preferences.userRoot(); - val token = Option(userRoot.get(PropertyConfiguration.OAUTH_ACCESS_TOKEN, null)) - val tokenSecret = Option(userRoot.get(PropertyConfiguration.OAUTH_ACCESS_TOKEN_SECRET, null)) - val oAuth = new OAuthAuthorization(new PropertyConfiguration(System.getProperties())) - if (token.isEmpty || tokenSecret.isEmpty) { - val requestToken = oAuth.getOAuthRequestToken() - println("Authorize application using URL: "+requestToken.getAuthorizationURL()) - println("Enter PIN: ") - val pin = Console.readLine - val accessToken = if (pin.length() > 0) oAuth.getOAuthAccessToken(requestToken, pin) else oAuth.getOAuthAccessToken() - userRoot.put(PropertyConfiguration.OAUTH_ACCESS_TOKEN, accessToken.getToken()) - userRoot.put(PropertyConfiguration.OAUTH_ACCESS_TOKEN_SECRET, accessToken.getTokenSecret()) - userRoot.flush() - } else { - oAuth.setOAuthAccessToken(new AccessToken(token.get, tokenSecret.get)); - } - oAuth + private def createOAuthAuthorization(): Authorization = { + new OAuthAuthorization(new ConfigurationBuilder().build()) } + + private val authorization = twitterAuth.getOrElse(createOAuthAuthorization()) override def getReceiver(): NetworkReceiver[Status] = { - new TwitterReceiver(if (twitterAuth.isEmpty) createOAuthAuthorization else twitterAuth.get, filters, storageLevel) + new TwitterReceiver(authorization, filters, storageLevel) } } diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index e5fdbe1b7a..4cf10582a9 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -1267,7 +1267,7 @@ public class JavaAPISuite implements Serializable { @Test public void testTwitterStream() { String[] filters = new String[] { "good", "bad", "ugly" }; - JavaDStream test = ssc.twitterStream("username", "password", filters, StorageLevel.MEMORY_ONLY()); + JavaDStream test = ssc.twitterStream(filters, StorageLevel.MEMORY_ONLY()); } @Test -- cgit v1.2.3 From 5cfcd3c336cc13e9fd448ae122216e4b583b77b4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 29 Jun 2013 15:37:27 -0700 Subject: Remove Twitter4J specific repo since it's in Maven central --- pom.xml | 11 ----------- project/SparkBuild.scala | 3 +-- 2 files changed, 1 insertion(+), 13 deletions(-) diff --git a/pom.xml b/pom.xml index 3bcb2a3f34..7a31be98b2 100644 --- a/pom.xml +++ b/pom.xml @@ -109,17 +109,6 @@ false - - twitter4j-repo - Twitter4J Repository - http://twitter4j.org/maven2/ - - true - - - false - - diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 07572201de..5e4692162e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -138,8 +138,7 @@ object SparkBuild extends Build { resolvers ++= Seq( "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", "Spray Repository" at "http://repo.spray.cc/", - "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/", - "Twitter4J Repository" at "http://twitter4j.org/maven2/" + "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/" ), libraryDependencies ++= Seq( -- cgit v1.2.3 From 03d0b858c807339b4221bedffa29ac76eef5352e Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 30 Jun 2013 15:38:58 -0700 Subject: Made use of spark.executor.memory setting consistent and documented it Conflicts: core/src/main/scala/spark/SparkContext.scala --- core/src/main/scala/spark/SparkContext.scala | 24 +++++++++++------ .../spark/scheduler/cluster/SchedulerBackend.scala | 11 ++------ docs/configuration.md | 31 ++++++++++++++-------- docs/ec2-scripts.md | 5 ++-- docs/tuning.md | 6 ++--- 5 files changed, 43 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 70a9d7698c..366afb2a2a 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -115,13 +115,14 @@ class SparkContext( // Environment variables to pass to our executors private[spark] val executorEnvs = HashMap[String, String]() // Note: SPARK_MEM is included for Mesos, but overwritten for standalone mode in ExecutorRunner - for (key <- Seq("SPARK_MEM", "SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", - "SPARK_TESTING")) { + for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING")) { val value = System.getenv(key) if (value != null) { executorEnvs(key) = value } } + // Since memory can be set with a system property too, use that + executorEnvs("SPARK_MEM") = SparkContext.executorMemoryRequested + "m" if (environment != null) { executorEnvs ++= environment } @@ -156,14 +157,12 @@ class SparkContext( scheduler case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => - // Check to make sure SPARK_MEM <= memoryPerSlave. Otherwise Spark will just hang. + // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang. val memoryPerSlaveInt = memoryPerSlave.toInt - val sparkMemEnv = System.getenv("SPARK_MEM") - val sparkMemEnvInt = if (sparkMemEnv != null) Utils.memoryStringToMb(sparkMemEnv) else 512 - if (sparkMemEnvInt > memoryPerSlaveInt) { + if (SparkContext.executorMemoryRequested > memoryPerSlaveInt) { throw new SparkException( - "Slave memory (%d MB) cannot be smaller than SPARK_MEM (%d MB)".format( - memoryPerSlaveInt, sparkMemEnvInt)) + "Asked to launch cluster with %d MB RAM / worker but requested %d MB/worker".format( + memoryPerSlaveInt, SparkContext.executorMemoryRequested)) } val scheduler = new ClusterScheduler(this) @@ -881,6 +880,15 @@ object SparkContext { /** Find the JAR that contains the class of a particular object */ def jarOfObject(obj: AnyRef): Seq[String] = jarOfClass(obj.getClass) + + /** Get the amount of memory per executor requested through system properties or SPARK_MEM */ + private[spark] val executorMemoryRequested = { + // TODO: Might need to add some extra memory for the non-heap parts of the JVM + Option(System.getProperty("spark.executor.memory")) + .orElse(Option(System.getenv("SPARK_MEM"))) + .map(Utils.memoryStringToMb) + .getOrElse(512) + } } diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala index 9ac875de3a..8844057a5c 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala @@ -1,6 +1,6 @@ package spark.scheduler.cluster -import spark.Utils +import spark.{SparkContext, Utils} /** * A backend interface for cluster scheduling systems that allows plugging in different ones under @@ -14,14 +14,7 @@ private[spark] trait SchedulerBackend { def defaultParallelism(): Int // Memory used by each executor (in megabytes) - protected val executorMemory = { - // TODO: Might need to add some extra memory for the non-heap parts of the JVM - Option(System.getProperty("spark.executor.memory")) - .orElse(Option(System.getenv("SPARK_MEM"))) - .map(Utils.memoryStringToMb) - .getOrElse(512) - } - + protected val executorMemory: Int = SparkContext.executorMemoryRequested // TODO: Probably want to add a killTask too } diff --git a/docs/configuration.md b/docs/configuration.md index 2de512f896..ae61769e31 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -25,23 +25,25 @@ Inside `spark-env.sh`, you *must* set at least the following two variables: * `SCALA_HOME`, to point to your Scala installation. * `MESOS_NATIVE_LIBRARY`, if you are [running on a Mesos cluster](running-on-mesos.html). -In addition, there are four other variables that control execution. These can be set *either in `spark-env.sh` -or in each job's driver program*, because they will automatically be propagated to workers from the driver. -For a multi-user environment, we recommend setting the in the driver program instead of `spark-env.sh`, so -that different user jobs can use different amounts of memory, JVM options, etc. +In addition, there are four other variables that control execution. These should be set *in the environment that +launches the job's driver program* instead of `spark-env.sh`, because they will be automatically propagated to +workers. Setting these per-job instead of in `spark-env.sh` ensures that different jobs can have different settings +for these variables. -* `SPARK_MEM`, to set the amount of memory used per node (this should be in the same format as the - JVM's -Xmx option, e.g. `300m` or `1g`) * `SPARK_JAVA_OPTS`, to add JVM options. This includes any system properties that you'd like to pass with `-D`. * `SPARK_CLASSPATH`, to add elements to Spark's classpath. * `SPARK_LIBRARY_PATH`, to add search directories for native libraries. +* `SPARK_MEM`, to set the amount of memory used per node. This should be in the same format as the + JVM's -Xmx option, e.g. `300m` or `1g`. Note that this option will soon be deprecated in favor of + the `spark.executor.memory` system property, so we recommend using that in new code. -Note that if you do set these in `spark-env.sh`, they will override the values set by user programs, which -is undesirable; you can choose to have `spark-env.sh` set them only if the user program hasn't, as follows: +Beware that if you do set these variables in `spark-env.sh`, they will override the values set by user programs, +which is undesirable; if you prefer, you can choose to have `spark-env.sh` set them only if the user program +hasn't, as follows: {% highlight bash %} -if [ -z "$SPARK_MEM" ] ; then - SPARK_MEM="1g" +if [ -z "$SPARK_JAVA_OPTS" ] ; then + SPARK_JAVA_OPTS="-verbose:gc" fi {% endhighlight %} @@ -55,10 +57,17 @@ val sc = new SparkContext(...) {% endhighlight %} Most of the configurable system properties control internal settings that have reasonable default values. However, -there are at least four properties that you will commonly want to control: +there are at least five properties that you will commonly want to control: + + + + + diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md index dc57035eba..eab8a0ff20 100644 --- a/docs/ec2-scripts.md +++ b/docs/ec2-scripts.md @@ -106,9 +106,8 @@ permissions on your private key file, you can run `launch` with the # Configuration You can edit `/root/spark/conf/spark-env.sh` on each machine to set Spark configuration options, such -as JVM options and, most crucially, the amount of memory to use per machine (`SPARK_MEM`). -This file needs to be copied to **every machine** to reflect the change. The easiest way to do this -is to use a script we provide called `copy-dir`. First edit your `spark-env.sh` file on the master, +as JVM options. This file needs to be copied to **every machine** to reflect the change. The easiest way to +do this is to use a script we provide called `copy-dir`. First edit your `spark-env.sh` file on the master, then run `~/spark-ec2/copy-dir /root/spark/conf` to RSYNC it to all the workers. The [configuration guide](configuration.html) describes the available configuration options. diff --git a/docs/tuning.md b/docs/tuning.md index 32c7ab86e9..5ffca54481 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -157,9 +157,9 @@ their work directories), *not* on your driver program. **Cache Size Tuning** -One important configuration parameter for GC is the amount of memory that should be used for -caching RDDs. By default, Spark uses 66% of the configured memory (`SPARK_MEM`) to cache RDDs. This means that - 33% of memory is available for any objects created during task execution. +One important configuration parameter for GC is the amount of memory that should be used for caching RDDs. +By default, Spark uses 66% of the configured executor memory (`spark.executor.memory` or `SPARK_MEM`) to +cache RDDs. This means that 33% of memory is available for any objects created during task execution. In case your tasks slow down and you find that your JVM is garbage-collecting frequently or running out of memory, lowering this value will help reduce the memory consumption. To change this to say 50%, you can call -- cgit v1.2.3 From 5bbd0eec84867937713ceb8438f25a943765a084 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 30 Jun 2013 17:00:26 -0700 Subject: Update docs on SCALA_LIBRARY_PATH --- conf/spark-env.sh.template | 18 ++++++------------ docs/configuration.md | 4 +++- 2 files changed, 9 insertions(+), 13 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 37565ca827..b8936314ec 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -3,8 +3,10 @@ # This file contains environment variables required to run Spark. Copy it as # spark-env.sh and edit that to configure Spark for your site. At a minimum, # the following two variables should be set: -# - MESOS_NATIVE_LIBRARY, to point to your Mesos native library (libmesos.so) -# - SCALA_HOME, to point to your Scala installation +# - SCALA_HOME, to point to your Scala installation, or SCALA_LIBRARY_PATH to +# point to the directory for Scala library JARs (if you install Scala as a +# Debian or RPM package, these are in a separate path, often /usr/share/java) +# - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos # # If using the standalone deploy mode, you can also set variables for it: # - SPARK_MASTER_IP, to bind the master to a different IP address @@ -12,14 +14,6 @@ # - SPARK_WORKER_CORES, to set the number of cores to use on this machine # - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g) # - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT -# - SPARK_WORKER_INSTANCES, to set the number of worker instances/processes to be spawned on every slave machine -# -# Finally, Spark also relies on the following variables, but these can be set -# on just the *master* (i.e. in your driver program), and will automatically -# be propagated to workers: -# - SPARK_MEM, to change the amount of memory used per node (this should -# be in the same format as the JVM's -Xmx option, e.g. 300m or 1g) -# - SPARK_CLASSPATH, to add elements to Spark's classpath -# - SPARK_JAVA_OPTS, to add JVM options -# - SPARK_LIBRARY_PATH, to add extra search paths for native libraries. +# - SPARK_WORKER_INSTANCES, to set the number of worker instances/processes +# to be spawned on every slave machine diff --git a/docs/configuration.md b/docs/configuration.md index ae61769e31..3266db7af1 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -22,7 +22,9 @@ the copy executable. Inside `spark-env.sh`, you *must* set at least the following two variables: -* `SCALA_HOME`, to point to your Scala installation. +* `SCALA_HOME`, to point to your Scala installation, or `SCALA_LIBRARY_PATH` to point to the directory for Scala + library JARs (if you install Scala as a Debian or RPM package, there is no `SCALA_HOME`, but these libraries + are in a separate path, typically /usr/share/java; look for `scala-library.jar`). * `MESOS_NATIVE_LIBRARY`, if you are [running on a Mesos cluster](running-on-mesos.html). In addition, there are four other variables that control execution. These should be set *in the environment that -- cgit v1.2.3 From 39ae073b5cd0dcfe4a00d9f205c88bad9df37870 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 30 Jun 2013 17:11:14 -0700 Subject: Increase SLF4j version in Maven too --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 7a31be98b2..48e623fa1c 100644 --- a/pom.xml +++ b/pom.xml @@ -56,7 +56,7 @@ 2.0.3 1.0-M2.1 1.1.1 - 1.6.1 + 1.7.2 4.1.2 1.2.17 -- cgit v1.2.3 From 3296d132b6ce042843de6e7384800e089b49e5fa Mon Sep 17 00:00:00 2001 From: root Date: Mon, 1 Jul 2013 02:45:00 +0000 Subject: Fix performance bug with new Python code not using buffered streams --- core/src/main/scala/spark/SparkEnv.scala | 3 +- .../main/scala/spark/api/python/PythonRDD.scala | 33 +++++++++++----------- 2 files changed, 19 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 7ccde2e818..ec59b4f48f 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -59,7 +59,8 @@ class SparkEnv ( def createPythonWorker(pythonExec: String, envVars: Map[String, String]): java.net.Socket = { synchronized { - pythonWorkers.getOrElseUpdate((pythonExec, envVars), new PythonWorkerFactory(pythonExec, envVars)).create() + val key = (pythonExec, envVars) + pythonWorkers.getOrElseUpdate(key, new PythonWorkerFactory(pythonExec, envVars)).create() } } diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 63140cf37f..3f283afa62 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -45,37 +45,38 @@ private[spark] class PythonRDD[T: ClassManifest]( new Thread("stdin writer for " + pythonExec) { override def run() { SparkEnv.set(env) - val out = new PrintWriter(worker.getOutputStream) - val dOut = new DataOutputStream(worker.getOutputStream) + val stream = new BufferedOutputStream(worker.getOutputStream) + val dataOut = new DataOutputStream(stream) + val printOut = new PrintWriter(stream) // Partition index - dOut.writeInt(split.index) + dataOut.writeInt(split.index) // sparkFilesDir - PythonRDD.writeAsPickle(SparkFiles.getRootDirectory, dOut) + PythonRDD.writeAsPickle(SparkFiles.getRootDirectory, dataOut) // Broadcast variables - dOut.writeInt(broadcastVars.length) + dataOut.writeInt(broadcastVars.length) for (broadcast <- broadcastVars) { - dOut.writeLong(broadcast.id) - dOut.writeInt(broadcast.value.length) - dOut.write(broadcast.value) - dOut.flush() + dataOut.writeLong(broadcast.id) + dataOut.writeInt(broadcast.value.length) + dataOut.write(broadcast.value) } + dataOut.flush() // Serialized user code for (elem <- command) { - out.println(elem) + printOut.println(elem) } - out.flush() + printOut.flush() // Data values for (elem <- parent.iterator(split, context)) { - PythonRDD.writeAsPickle(elem, dOut) + PythonRDD.writeAsPickle(elem, dataOut) } - dOut.flush() - out.flush() + dataOut.flush() + printOut.flush() worker.shutdownOutput() } }.start() // Return an iterator that read lines from the process's stdout - val stream = new DataInputStream(worker.getInputStream) + val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream)) return new Iterator[Array[Byte]] { def next(): Array[Byte] = { val obj = _nextObj @@ -288,7 +289,7 @@ class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) // This happens on the master, where we pass the updates to Python through a socket val socket = new Socket(serverHost, serverPort) val in = socket.getInputStream - val out = new DataOutputStream(socket.getOutputStream) + val out = new DataOutputStream(new BufferedOutputStream(socket.getOutputStream)) out.writeInt(val2.size) for (array <- val2) { out.writeInt(array.length) -- cgit v1.2.3 From ec31e68d5df259e6df001529235d8c906ff02a6f Mon Sep 17 00:00:00 2001 From: root Date: Mon, 1 Jul 2013 06:20:14 +0000 Subject: Fixed PySpark perf regression by not using socket.makefile(), and improved debuggability by letting "print" statements show up in the executor's stderr Conflicts: core/src/main/scala/spark/api/python/PythonRDD.scala --- .../main/scala/spark/api/python/PythonRDD.scala | 10 ++++-- .../spark/api/python/PythonWorkerFactory.scala | 20 ++++++++++- python/pyspark/daemon.py | 42 ++++++++++++---------- 3 files changed, 50 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 3f283afa62..31d8ea89d4 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -22,6 +22,8 @@ private[spark] class PythonRDD[T: ClassManifest]( accumulator: Accumulator[JList[Array[Byte]]]) extends RDD[Array[Byte]](parent) { + val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + // Similar to Runtime.exec(), if we are given a single string, split it into words // using a standard StringTokenizer (i.e. by spaces) def this(parent: RDD[T], command: String, envVars: JMap[String, String], @@ -45,7 +47,7 @@ private[spark] class PythonRDD[T: ClassManifest]( new Thread("stdin writer for " + pythonExec) { override def run() { SparkEnv.set(env) - val stream = new BufferedOutputStream(worker.getOutputStream) + val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) val dataOut = new DataOutputStream(stream) val printOut = new PrintWriter(stream) // Partition index @@ -76,7 +78,7 @@ private[spark] class PythonRDD[T: ClassManifest]( }.start() // Return an iterator that read lines from the process's stdout - val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream)) + val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream, bufferSize)) return new Iterator[Array[Byte]] { def next(): Array[Byte] = { val obj = _nextObj @@ -276,6 +278,8 @@ class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) extends AccumulatorParam[JList[Array[Byte]]] { Utils.checkHost(serverHost, "Expected hostname") + + val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList @@ -289,7 +293,7 @@ class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) // This happens on the master, where we pass the updates to Python through a socket val socket = new Socket(serverHost, serverPort) val in = socket.getInputStream - val out = new DataOutputStream(new BufferedOutputStream(socket.getOutputStream)) + val out = new DataOutputStream(new BufferedOutputStream(socket.getOutputStream, bufferSize)) out.writeInt(val2.size) for (array <- val2) { out.writeInt(array.length) diff --git a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala index 8844411d73..85d1dfeac8 100644 --- a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala @@ -51,7 +51,6 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String val workerEnv = pb.environment() workerEnv.putAll(envVars) daemon = pb.start() - daemonPort = new DataInputStream(daemon.getInputStream).readInt() // Redirect the stderr to ours new Thread("stderr reader for " + pythonExec) { @@ -69,6 +68,25 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } } }.start() + + val in = new DataInputStream(daemon.getInputStream) + daemonPort = in.readInt() + + // Redirect further stdout output to our stderr + new Thread("stdout reader for " + pythonExec) { + override def run() { + scala.util.control.Exception.ignoring(classOf[IOException]) { + // FIXME HACK: We copy the stream on the level of bytes to + // attempt to dodge encoding problems. + var buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + System.err.write(buf, 0, len) + len = in.read(buf) + } + } + } + }.start() } catch { case e => { stopDaemon() diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 78a2da1e18..78c9457b84 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -1,10 +1,13 @@ import os +import signal +import socket import sys +import traceback import multiprocessing from ctypes import c_bool from errno import EINTR, ECHILD -from socket import socket, AF_INET, SOCK_STREAM, SOMAXCONN -from signal import signal, SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN +from socket import AF_INET, SOCK_STREAM, SOMAXCONN +from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN from pyspark.worker import main as worker_main from pyspark.serializers import write_int @@ -33,11 +36,12 @@ def compute_real_exit_code(exit_code): def worker(listen_sock): # Redirect stdout to stderr os.dup2(2, 1) + sys.stdout = sys.stderr # The sys.stdout object is different from file descriptor 1 # Manager sends SIGHUP to request termination of workers in the pool def handle_sighup(*args): assert should_exit() - signal(SIGHUP, handle_sighup) + signal.signal(SIGHUP, handle_sighup) # Cleanup zombie children def handle_sigchld(*args): @@ -51,7 +55,7 @@ def worker(listen_sock): handle_sigchld() elif err.errno != ECHILD: raise - signal(SIGCHLD, handle_sigchld) + signal.signal(SIGCHLD, handle_sigchld) # Handle clients while not should_exit(): @@ -70,19 +74,22 @@ def worker(listen_sock): # never receives SIGCHLD unless a worker crashes. if os.fork() == 0: # Leave the worker pool - signal(SIGHUP, SIG_DFL) + signal.signal(SIGHUP, SIG_DFL) listen_sock.close() - # Handle the client then exit - sockfile = sock.makefile() + # Read the socket using fdopen instead of socket.makefile() because the latter + # seems to be very slow; note that we need to dup() the file descriptor because + # otherwise writes also cause a seek that makes us miss data on the read side. + infile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) + outfile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) exit_code = 0 try: - worker_main(sockfile, sockfile) + worker_main(infile, outfile) except SystemExit as exc: - exit_code = exc.code + exit_code = exc.code finally: - sockfile.close() - sock.close() - os._exit(compute_real_exit_code(exit_code)) + outfile.flush() + sock.close() + os._exit(compute_real_exit_code(exit_code)) else: sock.close() @@ -92,7 +99,6 @@ def launch_worker(listen_sock): try: worker(listen_sock) except Exception as err: - import traceback traceback.print_exc() os._exit(1) else: @@ -105,7 +111,7 @@ def manager(): os.setpgid(0, 0) # Create a listening socket on the AF_INET loopback interface - listen_sock = socket(AF_INET, SOCK_STREAM) + listen_sock = socket.socket(AF_INET, SOCK_STREAM) listen_sock.bind(('127.0.0.1', 0)) listen_sock.listen(max(1024, 2 * POOLSIZE, SOMAXCONN)) listen_host, listen_port = listen_sock.getsockname() @@ -121,8 +127,8 @@ def manager(): exit_flag.value = True # Gracefully exit on SIGTERM, don't die on SIGHUP - signal(SIGTERM, lambda signum, frame: shutdown()) - signal(SIGHUP, SIG_IGN) + signal.signal(SIGTERM, lambda signum, frame: shutdown()) + signal.signal(SIGHUP, SIG_IGN) # Cleanup zombie children def handle_sigchld(*args): @@ -133,7 +139,7 @@ def manager(): except EnvironmentError as err: if err.errno not in (ECHILD, EINTR): raise - signal(SIGCHLD, handle_sigchld) + signal.signal(SIGCHLD, handle_sigchld) # Initialization complete sys.stdout.close() @@ -148,7 +154,7 @@ def manager(): shutdown() raise finally: - signal(SIGTERM, SIG_DFL) + signal.signal(SIGTERM, SIG_DFL) exit_flag.value = True # Send SIGHUP to notify workers of shutdown os.kill(0, SIGHUP) -- cgit v1.2.3 From 7cd490ef5ba28df31f5e061eff83c855731dfca4 Mon Sep 17 00:00:00 2001 From: root Date: Mon, 1 Jul 2013 06:25:17 +0000 Subject: Clarify that PySpark is not supported on Windows --- docs/python-programming-guide.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 7f1e7cf93d..e8aaac74d0 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -17,10 +17,9 @@ There are a few key differences between the Python and Scala APIs: * Python is dynamically typed, so RDDs can hold objects of different types. * PySpark does not currently support the following Spark features: - Special functions on RDDs of doubles, such as `mean` and `stdev` - - `lookup` + - `lookup`, `sample` and `sort` - `persist` at storage levels other than `MEMORY_ONLY` - - `sample` - - `sort` + - Execution on Windows -- this is slated for a future release In PySpark, RDDs support the same methods as their Scala counterparts but take Python functions and return Python collection types. Short functions can be passed to RDD methods using Python's [`lambda`](http://www.diveintopython.net/power_of_introspection/lambda_functions.html) syntax: -- cgit v1.2.3 From 5de326db7d5235dbf259a4c8b388cb24c40fd2ec Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 1 Jul 2013 09:19:45 -0700 Subject: Print exception message --- core/src/main/scala/spark/scheduler/SparkListener.scala | 1 + core/src/main/scala/spark/ui/jobs/StagePage.scala | 6 +++++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index bac984b5c9..8de3aa91a4 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -68,6 +68,7 @@ class StatsReportListener extends SparkListener with Logging { showBytesDistribution("task result size:", (_, metric) => Some(metric.resultSize)) //runtime breakdown + val runtimePcts = stageCompleted.stageInfo.taskInfos.map{ case (info, metrics) => RuntimePercentage(info.duration, metrics) } diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index ed96fc2994..c6f87fc652 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -102,7 +102,11 @@ private[spark] class StagePage(parent: JobProgressUI) { {Option(metrics).flatMap{m => m.shuffleWriteMetrics}.map{s => }.getOrElse("")} + + {e.className} ({e.description})
    + {fmtStackTrace(e.stackTrace)} +
    ).getOrElse("")} + } } -- cgit v1.2.3 From 735c951a09f71512cf7be834802d1e88567ca9ae Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 1 Jul 2013 09:33:22 -0700 Subject: Adding test script --- .../main/scala/spark/ui/UIWorkloadGenerator.scala | 61 ++++++++++++++++++++++ 1 file changed, 61 insertions(+) create mode 100644 core/src/main/scala/spark/ui/UIWorkloadGenerator.scala diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala new file mode 100644 index 0000000000..24cfe36aaa --- /dev/null +++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala @@ -0,0 +1,61 @@ +package spark.ui + +import scala.util.Random + +import spark.SparkContext +import spark.SparkContext._ + +/** + * Continuously generates jobs that expose various features of the WebUI (internal testing tool). + * + * Usage: ./run spark.ui.UIWorkloadGenerator [master] + */ +private[spark] object UIWorkloadGenerator { + val NUM_PARTITIONS = 100 + val INTER_JOB_WAIT_MS = 500 + + def main(args: Array[String]) { + val master = args(0) + val appName = "Spark UI Tester" + val sc = new SparkContext(master, appName) + + // NOTE: Right now there is no easy way for us to show spark.job.annotation for a given phase, + // but we pass it here anyways since it will be useful once we do. + def setName(s: String) = { + sc.addLocalProperties("spark.job.annotation", s) + } + val baseData = sc.makeRDD(1 to NUM_PARTITIONS * 10, NUM_PARTITIONS) + def nextFloat() = (new Random()).nextFloat() + + val jobs = Seq[(String, () => Long)]( + ("Count", baseData.count), + ("Cache and Count", baseData.map(x => x).cache.count), + ("Single Shuffle", baseData.map(x => (x % 10, x)).reduceByKey(_ + _).count), + ("Entirely failed phase", baseData.map(x => throw new Exception).count), + ("Partially failed phase", { + baseData.map{x => + val probFailure = (4.0 / NUM_PARTITIONS) + if (nextFloat() < probFailure) { + throw new Exception("This is a task failure") + } + 1 + }.count + }), + ("Job with delays", baseData.map(x => Thread.sleep(1000)).count) + ) + + while (true) { + for ((desc, job) <- jobs) { + try { + setName(desc) + job() + println("Job funished: " + desc) + } catch { + case e: Exception => + println("Job Failed: " + desc) + } + Thread.sleep(INTER_JOB_WAIT_MS) + } + } + } +} -- cgit v1.2.3 From 8688689387eb6e2ac34add5e460dc54fe32b6def Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 1 Jul 2013 13:40:12 -0700 Subject: Various formatting changes --- core/src/main/scala/spark/Utils.scala | 8 ++++---- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 6 ++---- core/src/test/scala/spark/ui/UISuite.scala | 16 ++++++++-------- 3 files changed, 14 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 6966ee9ee9..af926b1e36 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -495,13 +495,13 @@ private object Utils extends Logging { ms match { case t if t < second => - "%dms".format(t) + "%d ms".format(t) case t if t < minute => - "%d.%03ds".format(t / second, t % second) + "%.1f s".format(t.toFloat / second) case t if t < hour => - "%d:%02d".format(t / minute, (t % minute) / second) + "%.1f m".format(t.toFloat / minute) case t => - "%d:%02d:%02d".format(t / hour, t % hour / minute, (t % hour) % minute / second) + "%.2f h".format(t.toFloat / hour) } } diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index cd04542faa..1e675ab2cb 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -18,8 +18,6 @@ private[spark] class IndexPage(parent: JobProgressUI) { val dateFmt = parent.dateFmt def render(request: HttpServletRequest): Seq[Node] = { - val stageHeaders = Seq("Stage ID", "Origin", "Submitted", "Duration", "Progress", - "Tasks: Complete/Total", "Shuffle Activity", "Stored RDD") val activeStages = listener.activeStages.toSeq val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq @@ -67,9 +65,9 @@ private[spark] class IndexPage(parent: JobProgressUI) { + fill="white" stroke="rgb(51,51,51)" stroke-width="1" /> + fill="rgb(0,136,204)" stroke="black" stroke-width="1" /> } diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index 127ab5ebc2..aa191f628b 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -34,13 +34,13 @@ class UISuite extends FunSuite { val hour = minute * 60 def str = Utils.msDurationToString(_) - assert(str(123) === "123ms") - assert(str(second) === "1.000s") - assert(str(second + 452) === "1.452s") - assert(str(hour) === "1:00:00") - assert(str(minute) === "1:00") - assert(str(minute + 4 * second + 34) === "1:04") - assert(str(10 * hour + minute + 4 * second) === "10:01:04") - assert(str(10 * hour + 59 * minute + 59 * second + 999) === "10:59:59") + assert(str(123) === "123 ms") + assert(str(second) === "1.0 s") + assert(str(second + 462) === "1.5 s") + assert(str(hour) === "1.00 h") + assert(str(minute) === "1.0 m") + assert(str(minute + 4 * second + 34) === "1.1 m") + assert(str(10 * hour + minute + 4 * second) === "10.02 h") + assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11.00 h") } } -- cgit v1.2.3 From 30b90342410a196d99d25113fbc1e37ce2a8200f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 1 Jul 2013 13:48:01 -0700 Subject: Fixing bug where logs aren't shown --- core/src/main/scala/spark/deploy/worker/Worker.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index b6a26245fc..6ae1cef940 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -45,7 +45,7 @@ private[spark] class Worker( val envVar = System.getenv("SPARK_PUBLIC_DNS") if (envVar != null) envVar else host } - val webUi = new WorkerWebUI(self, workDir, Some(webUiPort)) + var webUi: WorkerWebUI = null var coresUsed = 0 var memoryUsed = 0 @@ -77,6 +77,7 @@ private[spark] class Worker( sparkHome = new File(Option(System.getenv("SPARK_HOME")).getOrElse(".")) logInfo("Spark home: " + sparkHome) createWorkDir() + webUi = new WorkerWebUI(self, workDir, Some(webUiPort)) webUi.start() connectToMaster() } -- cgit v1.2.3 From 1025d7d1efb20e3683a604259c92eeef400cc3fa Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 1 Jul 2013 14:40:53 -0700 Subject: Package refactoring --- core/src/main/scala/spark/ui/JettyUtils.scala | 116 +------------------------- core/src/main/scala/spark/ui/Page.scala | 3 + core/src/main/scala/spark/ui/UIUtils.scala | 113 +++++++++++++++++++++++++ 3 files changed, 118 insertions(+), 114 deletions(-) create mode 100644 core/src/main/scala/spark/ui/Page.scala create mode 100644 core/src/main/scala/spark/ui/UIUtils.scala diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 9e4b884379..57f85bafd2 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -8,13 +8,12 @@ import net.liftweb.json.{JValue, pretty, render} import org.eclipse.jetty.server.{Server, Request, Handler} import org.eclipse.jetty.server.handler.{ResourceHandler, HandlerList, ContextHandler, AbstractHandler} +import org.eclipse.jetty.util.thread.QueuedThreadPool import scala.util.{Try, Success, Failure} import scala.xml.Node -import spark.{SparkContext, Logging} -import org.eclipse.jetty.util.log.Log -import org.eclipse.jetty.util.thread.QueuedThreadPool +import spark.Logging /** Utilities for launching a web server using Jetty's HTTP Server class */ private[spark] object JettyUtils extends Logging { @@ -111,117 +110,6 @@ private[spark] object JettyUtils extends Logging { } } - connect(port) } } - -object Page extends Enumeration { val Storage, Jobs = Value } - -/** Utility functions for generating XML pages with spark content. */ -private[spark] object UIUtils { - import Page._ - - /** Returns a spark page with correctly formatted headers */ - def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String, page: Page.Value) - : Seq[Node] = { - val storage = page match { - case Storage =>
  • Storage
  • - case _ =>
  • Storage
  • - } - val jobs = page match { - case Jobs =>
  • Jobs
  • - case _ =>
  • Jobs
  • - } - - - - - - - - - {title} - - - -
    - -
    -
    - -
    -
    - -
    -
    -

    - {title} -

    -
    -
    -
    - {content} -
    - - - } - - /** Returns a page with the spark css/js and a simple format. Used for scheduler UI. */ - def basicSparkPage(content: => Seq[Node], title: String): Seq[Node] = { - - - - - - - {title} - - - -
    -
    -
    - -
    -
    -

    - {title} -

    -
    -
    - {content} -
    - - - } - - /** Returns an HTML table constructed by generating a row for each object in a sequence. */ - def listingTable[T](headers: Seq[String], makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { -
    Property NameDefaultMeaning
    spark.executor.memory512m + Amount of memory to use per executor process, in the same format as JVM memory strings (e.g. `512m`, `2g`). +
    spark.serializer spark.JavaSerializer{Utils.memoryBytesToString(s.shuffleBytesWritten)}{exception.map(e => - {e.className}
    {fmtStackTrace(e.stackTrace)}
    ).getOrElse("")}
    - {headers.map(h => )} - - {rows.map(r => makeRow(r))} - -
    {h}
    - } -} diff --git a/core/src/main/scala/spark/ui/Page.scala b/core/src/main/scala/spark/ui/Page.scala new file mode 100644 index 0000000000..c853b44b76 --- /dev/null +++ b/core/src/main/scala/spark/ui/Page.scala @@ -0,0 +1,3 @@ +package spark.ui + +private[spark] object Page extends Enumeration { val Storage, Jobs = Value } \ No newline at end of file diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala new file mode 100644 index 0000000000..7b79290d1b --- /dev/null +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -0,0 +1,113 @@ +package spark.ui + +import scala.xml.Node + +import spark.SparkContext + +/** Utility functions for generating XML pages with spark content. */ +private[spark] object UIUtils { + import Page._ + + /** Returns a spark page with correctly formatted headers */ + def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String, page: Page.Value) + : Seq[Node] = { + val storage = page match { + case Storage =>
  • Storage
  • + case _ =>
  • Storage
  • + } + val jobs = page match { + case Jobs =>
  • Jobs
  • + case _ =>
  • Jobs
  • + } + + + + + + + + + {title} + + + +
    + +
    +
    + +
    +
    + +
    +
    +

    + {title} +

    +
    +
    +
    + {content} +
    + + + } + + /** Returns a page with the spark css/js and a simple format. Used for scheduler UI. */ + def basicSparkPage(content: => Seq[Node], title: String): Seq[Node] = { + + + + + + + {title} + + + +
    +
    +
    + +
    +
    +

    + {title} +

    +
    +
    + {content} +
    + + + } + + /** Returns an HTML table constructed by generating a row for each object in a sequence. */ + def listingTable[T](headers: Seq[String], makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { + + {headers.map(h => )} + + {rows.map(r => makeRow(r))} + +
    {h}
    + } +} \ No newline at end of file -- cgit v1.2.3 From 9a42d04efa1ff620306ec8864c9f95186fc1a199 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 1 Jul 2013 14:43:13 -0700 Subject: Throw exception for missing resource --- core/src/main/scala/spark/ui/JettyUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 57f85bafd2..bc6f9c10d5 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -69,7 +69,7 @@ private[spark] object JettyUtils extends Logging { case Some(res) => staticHandler.setResourceBase(res.toString) case None => - logError("Could not find resource path for Web UI: " + resourceBase) + throw new Exception("Could not find resource path for Web UI: " + resourceBase) } staticHandler } -- cgit v1.2.3 From 6fdbc68f2c5b220d1618d5a78d46aa0f844cae45 Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Mon, 1 Jul 2013 16:05:55 -0700 Subject: Fixing missed hbase dependency in examples hadoop2-yarn profile --- examples/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/examples/pom.xml b/examples/pom.xml index 3e5271ec2f..78ec58729b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -193,6 +193,11 @@ hadoop-yarn-common provided
    + + org.apache.hbase + hbase + 0.94.6 + -- cgit v1.2.3 From 8ca1cc1786a376b227ba69872efbf8e26952b80d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 2 Jul 2013 15:56:34 -0700 Subject: Adding truncation for log files --- core/src/main/scala/spark/Utils.scala | 14 ++++++++++ .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 16 ++++++----- core/src/test/scala/spark/ui/UISuite.scala | 31 ++++++++++++++++++++++ 3 files changed, 54 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index af926b1e36..c1616de641 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -603,6 +603,20 @@ private object Utils extends Logging { portBound } + /** Return a string containing the last `n` bytes of a file. */ + def lastNBytes(path: String, n: Int): String = { + val file = new File(path) + val length = file.length() + val buff = new Array[Byte](math.min(n, length.toInt)) + val skip = math.max(0, length - n) + val stream = new FileInputStream(file) + + stream.skip(skip) + stream.read(buff) + stream.close() + Source.fromBytes(buff).mkString + } + /** * Clone an object using a Spark serializer. */ diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 2e6279566c..16564d5619 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -3,14 +3,12 @@ package spark.deploy.worker.ui import akka.actor.ActorRef import akka.util.{Duration, Timeout} -import java.io.File +import java.io.{FileInputStream, File} import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} -import scala.io.Source - import spark.{Utils, Logging} import spark.ui.JettyUtils import spark.ui.JettyUtils._ @@ -56,11 +54,15 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") + + val maxBytes = 1024 * 1024 // Guard against OOM + val defaultBytes = 100 * 1024 + val numBytes = Option(request.getParameter("numBytes")) + .flatMap(s => Some(s.toInt)).getOrElse(defaultBytes) + val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val source = Source.fromFile(path) - val lines = source.mkString - source.close() - lines + val pre = "==== Last %s bytes of %s/%s/%s ====\n".format(numBytes, appId, executorId, logType) + pre + Utils.lastNBytes(path, math.min(numBytes, maxBytes)) } def stop() { diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index aa191f628b..e4bb3abc33 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -5,6 +5,10 @@ import org.eclipse.jetty.server.Server import java.net.ServerSocket import scala.util.{Failure, Success, Try} import spark.Utils +import com.google.common.io.Files +import org.apache.commons.io.FileUtils +import java.io.{FileOutputStream, File} +import com.google.common.base.Charsets class UISuite extends FunSuite { test("jetty port increases under contention") { @@ -43,4 +47,31 @@ class UISuite extends FunSuite { assert(str(10 * hour + minute + 4 * second) === "10.02 h") assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11.00 h") } + + test("reading last n bytes of a file") { + val tmpDir = Files.createTempDir() + + // File smaller than limit + val f1Path = tmpDir + "/f1" + val f1 = new FileOutputStream(f1Path) + f1.write("a\nb\nc\nd".getBytes(Charsets.UTF_8)) + f1.close() + assert(Utils.lastNBytes(f1Path, 1024) === "a\nb\nc\nd") + + // File larger than limit + val f2Path = tmpDir + "/f2" + val f2 = new FileOutputStream(f2Path) + f2.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) + f2.close() + assert(Utils.lastNBytes(f2Path, 8) === "5\n6\n7\n8\n") + + // Request limit too + val f3Path = tmpDir + "/f2" + val f3 = new FileOutputStream(f3Path) + f3.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) + f3.close() + assert(Utils.lastNBytes(f3Path, 8) === "5\n6\n7\n8\n") + + FileUtils.deleteDirectory(tmpDir) + } } -- cgit v1.2.3 From 39e2325675e40c2231e26a7e693035f54039b1d5 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 2 Jul 2013 16:28:40 -0700 Subject: Removing dead code --- core/src/main/scala/spark/Utils.scala | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index c1616de641..c5faf79ac3 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -589,19 +589,6 @@ private object Utils extends Logging { "%s at %s:%s".format(callSiteInfo.lastSparkMethod, callSiteInfo.firstUserFile, callSiteInfo.firstUserLine) } - /** - * Try to find a free port to bind to on the local host. This should ideally never be needed, - * except that, unfortunately, some of the networking libraries we currently rely on (e.g. Spray) - * don't let users bind to port 0 and then figure out which free port they actually bound to. - * We work around this by binding a ServerSocket and immediately unbinding it. This is *not* - * necessarily guaranteed to work, but it's the best we can do. - */ - def findFreePort(): Int = { - val socket = new ServerSocket(0) - val portBound = socket.getLocalPort - socket.close() - portBound - } /** Return a string containing the last `n` bytes of a file. */ def lastNBytes(path: String, n: Int): String = { -- cgit v1.2.3 From 923cf929003c67963e273fcdcd5b01baf68df8b5 Mon Sep 17 00:00:00 2001 From: "Y.CORP.YAHOO.COM\\tgraves" Date: Tue, 2 Jul 2013 21:18:59 -0500 Subject: Rework from pull request. Removed --user option from Spark on Yarn Client, made the user of JAVA_HOME environment variable conditional on if its set, and created addCredentials in each of the SparkHadoopUtil classes to only add the credentials when the profile is hadoop2-yarn. --- core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala | 4 ++++ .../hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala | 7 +++++++ core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala | 14 +++++++++----- .../scala/spark/deploy/yarn/ClientArguments.scala | 8 +------- .../scala/spark/deploy/yarn/WorkerRunnable.scala | 12 +++++++----- core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala | 4 ++++ core/src/main/scala/spark/PairRDDFunctions.scala | 4 ---- core/src/main/scala/spark/SparkContext.scala | 8 -------- core/src/main/scala/spark/rdd/HadoopRDD.scala | 2 ++ core/src/main/scala/spark/scheduler/InputFormatInfo.scala | 11 +++-------- .../main/scala/spark/streaming/PairDStreamFunctions.scala | 4 ---- 11 files changed, 37 insertions(+), 41 deletions(-) diff --git a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala index a0fb4fe25d..f1c86de4cc 100644 --- a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala @@ -1,5 +1,6 @@ package spark.deploy import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapred.JobConf /** @@ -20,4 +21,7 @@ object SparkHadoopUtil { // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems def newConfiguration(): Configuration = new Configuration() + + // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster + def addCredentials(conf: JobConf) {} } diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala index b96c047e10..301a57fffa 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala @@ -1,6 +1,7 @@ package spark.deploy import collection.mutable.HashMap +import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.conf.Configuration @@ -49,4 +50,10 @@ object SparkHadoopUtil { // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems // Always create a new config, dont reuse yarnConf. def newConfiguration(): Configuration = new YarnConfiguration(new Configuration()) + + // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster + def addCredentials(conf: JobConf) { + val jobCreds = conf.getCredentials(); + jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) + } } diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala index f20cc31c7c..514c17f241 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala @@ -45,7 +45,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl appContext.setQueue(args.amQueue) appContext.setAMContainerSpec(amContainer) - appContext.setUser(args.amUser) + appContext.setUser(UserGroupInformation.getCurrentUser().getShortUserName()) submitApp(appContext) @@ -141,9 +141,6 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl val log4jConfLocalRes = localResources.getOrElse("log4j.properties", null) val env = new HashMap[String, String]() - Apps.addToEnvironment(env, Environment.USER.name, args.amUser) - // set this so that UGI set to correct user in unsecure mode - Apps.addToEnvironment(env, "HADOOP_USER_NAME", args.amUser) // If log4j present, ensure ours overrides all others if (log4jConfLocalRes != null) Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./") @@ -171,6 +168,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl env("SPARK_YARN_LOG4J_SIZE") = log4jConfLocalRes.getSize().toString() } + // Add each SPARK-* key to the environment System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } return env @@ -224,7 +222,13 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl } // Command for the ApplicationMaster - val commands = List[String](Environment.JAVA_HOME.$() + "/bin/java " + + var javaCommand = "java"; + val javaHome = System.getenv("JAVA_HOME") + if (javaHome != null && !javaHome.isEmpty()) { + javaCommand = Environment.JAVA_HOME.$() + "/bin/java" + } + + val commands = List[String](javaCommand + " -server " + JAVA_OPTS + " spark.deploy.yarn.ApplicationMaster" + diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala index 24110558e7..07e7edea36 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala @@ -13,7 +13,6 @@ class ClientArguments(val args: Array[String]) { var workerMemory = 1024 var workerCores = 1 var numWorkers = 2 - var amUser = System.getProperty("user.name") var amQueue = System.getProperty("QUEUE", "default") var amMemory: Int = 512 // TODO @@ -58,10 +57,6 @@ class ClientArguments(val args: Array[String]) { workerCores = value args = tail - case ("--user") :: value :: tail => - amUser = value - args = tail - case ("--queue") :: value :: tail => amQueue = value args = tail @@ -96,8 +91,7 @@ class ClientArguments(val args: Array[String]) { " --worker-cores NUM Number of cores for the workers (Default: 1). This is unsused right now.\n" + " --master-memory MEM Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" + " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" + - " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')\n" + - " --user USERNAME Run the ApplicationMaster (and slaves) as a different user\n" + " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')" ) System.exit(exitCode) } diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala index e22d256a84..cc6f3344a1 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala @@ -85,7 +85,13 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S credentials.writeTokenStorageToStream(dob) ctx.setContainerTokens(ByteBuffer.wrap(dob.getData())) - val commands = List[String](Environment.JAVA_HOME.$() + "/bin/java " + + var javaCommand = "java"; + val javaHome = System.getenv("JAVA_HOME") + if (javaHome != null && !javaHome.isEmpty()) { + javaCommand = Environment.JAVA_HOME.$() + "/bin/java" + } + + val commands = List[String](javaCommand + " -server " + // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in an inconsistent state. @@ -152,10 +158,6 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S def prepareEnvironment: HashMap[String, String] = { val env = new HashMap[String, String]() - // should we add this ? - Apps.addToEnvironment(env, Environment.USER.name, Utils.getUserNameFromEnvironment()) - // set this so that UGI set to correct user in unsecure mode - Apps.addToEnvironment(env, "HADOOP_USER_NAME", Utils.getUserNameFromEnvironment()) // If log4j present, ensure ours overrides all others if (System.getenv("SPARK_YARN_LOG4J_PATH") != null) { diff --git a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala index a0fb4fe25d..f1c86de4cc 100644 --- a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala @@ -1,5 +1,6 @@ package spark.deploy import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapred.JobConf /** @@ -20,4 +21,7 @@ object SparkHadoopUtil { // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems def newConfiguration(): Configuration = new Configuration() + + // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster + def addCredentials(conf: JobConf) {} } diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index b397601184..c9d698fc09 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -614,10 +614,6 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( outputFormatClass: Class[_ <: OutputFormat[_, _]], conf: JobConf = new JobConf(self.context.hadoopConfiguration), codec: Option[Class[_ <: CompressionCodec]] = None) { - // make sure to propogate any credentials from the current user to the jobConf - // for Hadoop security - val jobCreds = conf.getCredentials(); - jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) conf.setOutputKeyClass(keyClass) conf.setOutputValueClass(valueClass) // conf.setOutputFormat(outputFormatClass) // Doesn't work in Scala 2.9 due to what may be a generics bug diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index c3a56938b5..6c37203707 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -295,10 +295,6 @@ class SparkContext( valueClass: Class[V], minSplits: Int = defaultMinSplits ): RDD[(K, V)] = { - // make sure to propogate any credentials from the current user to the jobConf - // for Hadoop security - val jobCreds = conf.getCredentials(); - jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minSplits) } @@ -311,10 +307,6 @@ class SparkContext( minSplits: Int = defaultMinSplits ) : RDD[(K, V)] = { val conf = new JobConf(hadoopConfiguration) - // make sure to propogate any credentials from the current user to the jobConf - // for Hadoop security - val jobCreds = conf.getCredentials(); - jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) FileInputFormat.setInputPaths(conf, path) new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minSplits) } diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index cbf5512e24..07c103503c 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -15,6 +15,7 @@ import org.apache.hadoop.mapred.RecordReader import org.apache.hadoop.mapred.Reporter import org.apache.hadoop.util.ReflectionUtils +import spark.deploy.SparkHadoopUtil import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext} import spark.util.NextIterator import org.apache.hadoop.conf.Configurable @@ -50,6 +51,7 @@ class HadoopRDD[K, V]( private val confBroadcast = sc.broadcast(new SerializableWritable(conf)) override def getPartitions: Array[Partition] = { + SparkHadoopUtil.addCredentials(conf); val inputFormat = createInputFormat(conf) if (inputFormat.isInstanceOf[Configurable]) { inputFormat.asInstanceOf[Configurable].setConf(conf) diff --git a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala index 30a56d7135..17d0ea4f80 100644 --- a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala @@ -9,6 +9,7 @@ import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.conf.Configuration import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.collection.JavaConversions._ +import spark.deploy.SparkHadoopUtil /** @@ -71,10 +72,7 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl // This method does not expect failures, since validate has already passed ... private def prefLocsFromMapreduceInputFormat(): Set[SplitInfo] = { val conf = new JobConf(configuration) - // make sure to propogate any credentials from the current user to the jobConf - // for Hadoop security - val jobCreds = conf.getCredentials(); - jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) + SparkHadoopUtil.addCredentials(conf); FileInputFormat.setInputPaths(conf, path) val instance: org.apache.hadoop.mapreduce.InputFormat[_, _] = @@ -94,10 +92,7 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl // This method does not expect failures, since validate has already passed ... private def prefLocsFromMapredInputFormat(): Set[SplitInfo] = { val jobConf = new JobConf(configuration) - // make sure to propogate any credentials from the current user to the jobConf - // for Hadoop security - val jobCreds = jobConf.getCredentials(); - jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) + SparkHadoopUtil.addCredentials(jobConf); FileInputFormat.setInputPaths(jobConf, path) val instance: org.apache.hadoop.mapred.InputFormat[_, _] = diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala index 20ee1d3c5d..8d0a83d439 100644 --- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala @@ -472,10 +472,6 @@ extends Serializable { outputFormatClass: Class[_ <: OutputFormat[_, _]], conf: JobConf = new JobConf ) { - // make sure to propogate any credentials from the current user to the jobConf - // for Hadoop security - val jobCreds = conf.getCredentials(); - jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) val saveFunc = (rdd: RDD[(K, V)], time: Time) => { val file = rddToFileName(prefix, suffix, time) rdd.saveAsHadoopFile(file, keyClass, valueClass, outputFormatClass, conf) -- cgit v1.2.3 From 04567a1771ec02e1efcc3ce078291a975fbb4e68 Mon Sep 17 00:00:00 2001 From: Mingfei Date: Wed, 3 Jul 2013 17:43:37 +0800 Subject: update guava version from 11.0.1 to 14.0.1 --- project/SparkBuild.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5e4692162e..f65e398d95 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -142,7 +142,8 @@ object SparkBuild extends Build { ), libraryDependencies ++= Seq( - "com.google.guava" % "guava" % "11.0.1", + "com.google.guava" % "guava" % "14.0.1", + "com.google.code.findbugs" % "jsr305" % "1.3.+", "log4j" % "log4j" % "1.2.16", "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, -- cgit v1.2.3 From 0f06d6217d0da67633d9ede73df67b94133fd3d5 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 4 Jul 2013 01:05:39 -0700 Subject: s/ActorSystemImpl/ExtendedActorSystem/ as ActorSystemImpl results in a warning --- core/src/main/scala/spark/util/AkkaUtils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index e93cc3b485..0cff0d5b01 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -1,6 +1,6 @@ package spark.util -import akka.actor.{ActorRef, Props, ActorSystemImpl, ActorSystem} +import akka.actor.{ActorRef, Props, ExtendedActorSystem, ActorSystem} import com.typesafe.config.ConfigFactory import akka.util.duration._ import akka.pattern.ask @@ -56,7 +56,7 @@ private[spark] object AkkaUtils { // Figure out the port number we bound to, in case port was passed as 0. This is a bit of a // hack because Akka doesn't let you figure out the port through the public API yet. - val provider = actorSystem.asInstanceOf[ActorSystemImpl].provider + val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider val boundPort = provider.asInstanceOf[RemoteActorRefProvider].transport.address.port.get return (actorSystem, boundPort) } -- cgit v1.2.3 From 6ccfb73ca92a72d5cca6a504d9ee332a16d9293d Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Thu, 4 Jul 2013 19:19:44 +0800 Subject: Add fair scheduler config template file --- conf/fairscheduler.xml.template | 15 +++++++++++++++ core/src/test/resources/fairscheduler.xml | 1 + 2 files changed, 16 insertions(+) create mode 100644 conf/fairscheduler.xml.template diff --git a/conf/fairscheduler.xml.template b/conf/fairscheduler.xml.template new file mode 100644 index 0000000000..04a6b418dc --- /dev/null +++ b/conf/fairscheduler.xml.template @@ -0,0 +1,15 @@ + + + + 2 + 1 + FAIR + + + 3 + 2 + FIFO + + + + diff --git a/core/src/test/resources/fairscheduler.xml b/core/src/test/resources/fairscheduler.xml index 5a688b0ebb..6e573b1883 100644 --- a/core/src/test/resources/fairscheduler.xml +++ b/core/src/test/resources/fairscheduler.xml @@ -1,3 +1,4 @@ + 2 -- cgit v1.2.3 From c0c3155c3c0e37ba31cbeeccb21654bdf5ec0cfc Mon Sep 17 00:00:00 2001 From: Lian Cheng Date: Fri, 5 Jul 2013 00:54:10 +0800 Subject: Bug fix: SPARK-789 https://spark-project.atlassian.net/browse/SPARK-789 --- core/src/main/scala/spark/deploy/master/Master.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 3e965e82ac..87f304b6cd 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -278,7 +278,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act exec.state = ExecutorState.KILLED } app.markFinished(state) - app.driver ! ApplicationRemoved(state.toString) + if (state != ApplicationState.FINISHED) { + app.driver ! ApplicationRemoved(state.toString) + } schedule() } } -- cgit v1.2.3 From 7687ed529217161ea681939870f6e1e042139e65 Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Thu, 4 Jul 2013 13:48:33 -0700 Subject: Use standard ASF published avro module instead of a proprietory built one --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 48e623fa1c..5b4c1d48f7 100644 --- a/pom.xml +++ b/pom.xml @@ -620,12 +620,12 @@ org.apache.avro avro - 1.7.1.cloudera.2 + 1.7.1 org.apache.avro avro-ipc - 1.7.1.cloudera.2 + 1.7.1 -- cgit v1.2.3 From e4ff544a8d43a23ad2110556102bb83e34ae71b4 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 5 Jul 2013 10:34:45 +0800 Subject: Clean StageToInfos periodically when spark.cleaner.ttl is enabled --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index f7d60be5db..51b10ed045 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -312,7 +312,7 @@ class DAGScheduler( handleExecutorLost(execId) case completion: CompletionEvent => - sparkListeners.foreach(_.onTaskEnd(SparkListenerTaskEnd(completion.task, + sparkListeners.foreach(_.onTaskEnd(SparkListenerTaskEnd(completion.task, completion.reason, completion.taskInfo, completion.taskMetrics))) handleTaskCompletion(completion) @@ -651,7 +651,7 @@ class DAGScheduler( "(generation " + currentGeneration + ")") } } - + private def handleExecutorGained(execId: String, hostPort: String) { // remove from failedGeneration(execId) ? if (failedGeneration.contains(execId)) { @@ -747,6 +747,10 @@ class DAGScheduler( sizeBefore = pendingTasks.size pendingTasks.clearOldValues(cleanupTime) logInfo("pendingTasks " + sizeBefore + " --> " + pendingTasks.size) + + sizeBefore = stageToInfos.size + stageToInfos.clearOldValues(cleanupTime) + logInfo("stageToInfos " + sizeBefore + " --> " + stageToInfos.size) } def stop() { -- cgit v1.2.3 From 729e463f649332c5480d2d175d42d4ba0dd3cb74 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 11:13:41 -0700 Subject: Import RidgeRegression example Conflicts: run --- ml/src/main/scala/spark/ml/RidgeRegression.scala | 110 +++++++++++++++++++++ .../scala/spark/ml/RidgeRegressionGenerator.scala | 70 +++++++++++++ project/SparkBuild.scala | 12 ++- 3 files changed, 191 insertions(+), 1 deletion(-) create mode 100644 ml/src/main/scala/spark/ml/RidgeRegression.scala create mode 100644 ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala new file mode 100644 index 0000000000..7896873d44 --- /dev/null +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -0,0 +1,110 @@ +package spark.ml + +import spark._ +import spark.SparkContext._ + +import org.apache.commons.math3.distribution.NormalDistribution +import org.jblas.DoubleMatrix +import org.jblas.Solve + +/** + * Ridge Regression from Joseph Gonzalez's implementation in MLBase + */ + +class RidgeRegressionModel( + val wOpt: DoubleMatrix, + val lambdaOpt: Double, + val lambdas: List[(Double, Double, DoubleMatrix)]) { + + def predict(test_data: spark.RDD[Array[Double]]) = { + test_data.map(x => new DoubleMatrix(1, x.length, x:_*).mmul(this.wOpt)) + } +} + +object RidgeRegression extends Logging { + + def train(data: spark.RDD[(Double, Array[Double])], + lambdaLow: Double = 0.0, + lambdaHigh: Double = 10000.0) = { + + data.cache() + val nfeatures = data.take(1)(0)._2.length + val nexamples = data.count + + // Compute XtX - Size of XtX is nfeatures by nfeatures + val XtX = data.map { + case (y, features) => + val x = new DoubleMatrix(1, features.length, features:_*) + x.transpose().mmul(x) + }.reduce(_.add(_)) + + // Compute Xt*y - Size of Xty is nfeatures by 1 + val Xty = data.map { + case (y, features) => + new DoubleMatrix(features.length, 1, features:_*).mul(y) + }.reduce(_.add(_)) + + // Define a function to compute the leave one out cross validation error + // for a single example + def crossValidate(lambda: Double) = { + // Compute the MLE ridge regression parameter value + + // Ridge Regression parameter = inv(XtX + \lambda*I) * Xty + val XtXlambda = DoubleMatrix.eye(nfeatures).muli(lambda).addi(XtX) + val w = Solve.solveSymmetric(XtXlambda, Xty) + + val invXtX = Solve.solveSymmetric(XtXlambda, + DoubleMatrix.eye(nfeatures)) + + // compute the leave one out cross validation score + val cvError = data.map { + case (y, features) => + val x = new DoubleMatrix(features.length, 1, features:_*) + val yhat = w.transpose().mmul(x).get(0) + val H_ii = x.transpose().mmul(invXtX).mmul(x).get(0) + val residual = (y - yhat) / (1.0 - H_ii) + residual * residual + }.reduce(_ + _) + (lambda, cvError, w) + } + + // Binary search for the best assignment to lambda. + def binSearch(low: Double, high: Double): List[(Double, Double, DoubleMatrix)] = { + val mid = (high - low) / 2 + low + val lowValue = crossValidate((mid - low) / 2 + low) + val highValue = crossValidate((high - mid) / 2 + mid) + val (newLow, newHigh) = if (lowValue._2 < highValue._2) { + (low, mid + (high-low)/4) + } else { + (mid - (high-low)/4, high) + } + if (newHigh - newLow > 1.0E-7) { + lowValue :: highValue :: binSearch(newLow, newHigh) + } else { + List(lowValue, highValue) + } + } + + // Actually compute the best lambda + val lambdas = binSearch(lambdaLow, lambdaHigh).sortBy(_._1) + + // Find the best parameter set + val (lambdaOpt, cverror, wOpt) = lambdas.reduce((a, b) => if (a._2 < b._2) a else b) + + logInfo("RidgeRegression: optimal lambda " + lambdaOpt) + + // Return the model which contains the solution + new RidgeRegressionModel(wOpt, lambdaOpt, lambdas) + } + + def main(args: Array[String]) { + if (args.length != 2) { + println("Usage: RidgeRegression ") + System.exit(1) + } + val sc = new SparkContext(args(0), "RidgeRegression") + val data = RidgeRegressionGenerator.loadData(sc, args(1)) + val model = train(data, 0, 100) + sc.stop() + } +} diff --git a/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala b/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala new file mode 100644 index 0000000000..22a1e4613b --- /dev/null +++ b/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala @@ -0,0 +1,70 @@ +package spark.ml + +import spark._ +import spark.SparkContext._ + +import org.apache.commons.math3.distribution.NormalDistribution +import org.jblas.DoubleMatrix + +object RidgeRegressionGenerator { + + // Helper methods to load and save data used for RidgeRegression + // Data format: + // , ... + // where , are feature values in Double and + // is the corresponding label as Double + def loadData(sc: SparkContext, dir: String) = { + val data = sc.textFile(dir).map{ line => + val parts = line.split(",") + val label = parts(0).toDouble + val features = parts(1).trim().split(" ").map(_.toDouble) + (label, features) + } + data + } + + def saveData(data: RDD[(Double, Array[Double])], dir: String) { + val dataStr = data.map(x => x._1 + "," + x._2.mkString(" ")) + dataStr.saveAsTextFile(dir) + } + + def main(args: Array[String]) { + if (args.length != 2) { + println("Usage: RidgeRegressionGenerator ") + System.exit(1) + } + org.jblas.util.Random.seed(42) + val sc = new SparkContext(args(0), "RidgeRegressionGenerator") + + val nexamples = 1000 + val nfeatures = 100 + val eps = 10 + val parts = 2 + + // 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 = 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 NormalDistribution(0, eps) + rnd.reseedRandomGenerator(42 + p) + + val normalValues = (0 until examplesInPartition).map(_ => rnd.sample()) + val yObs = new DoubleMatrix(examplesInPartition, 1, normalValues:_*).addi(y) + + (0 until examplesInPartition).map(i => + (yObs.get(i, 0), X.getRow(i).toArray) + ) + } + + saveData(data, args(1)) + System.exit(0) + } +} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5e4692162e..731671c23b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -25,7 +25,7 @@ object SparkBuild extends Build { //val HADOOP_MAJOR_VERSION = "2" //val HADOOP_YARN = true - lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming) + lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming, ml) lazy val core = Project("core", file("core"), settings = coreSettings) @@ -37,6 +37,8 @@ object SparkBuild extends Build { lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn (core) + lazy val ml = Project("ml", file("ml"), settings = mlSettings) dependsOn (core) + // A configuration to set an alternative publishLocalConfiguration lazy val MavenCompile = config("m2r") extend(Compile) lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") @@ -219,6 +221,14 @@ object SparkBuild extends Build { def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") + def mlSettings = examplesSettings ++ Seq( + name := "spark-ml", + libraryDependencies ++= Seq( + "org.jblas" % "jblas" % "1.2.3", + "org.apache.commons" % "commons-math3" % "3.2" + ) + ) + def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", resolvers ++= Seq( -- cgit v1.2.3 From 6a9a9a364ce3b158c4162e401f90eb4d305104e8 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 4 Jun 2013 16:27:02 -0700 Subject: Minor clean up of the RidgeRegression code. I am not even sure why I did this :s. --- ml/src/main/scala/spark/ml/RidgeRegression.scala | 38 ++++++++++------------ .../scala/spark/ml/RidgeRegressionGenerator.scala | 38 ++++++++++++---------- project/SparkBuild.scala | 2 +- 3 files changed, 39 insertions(+), 39 deletions(-) diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala index 7896873d44..b8b632e111 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -1,9 +1,7 @@ package spark.ml -import spark._ -import spark.SparkContext._ +import spark.{Logging, RDD, SparkContext} -import org.apache.commons.math3.distribution.NormalDistribution import org.jblas.DoubleMatrix import org.jblas.Solve @@ -23,39 +21,36 @@ class RidgeRegressionModel( object RidgeRegression extends Logging { - def train(data: spark.RDD[(Double, Array[Double])], - lambdaLow: Double = 0.0, + def train(data: RDD[(Double, Array[Double])], + lambdaLow: Double = 0.0, lambdaHigh: Double = 10000.0) = { data.cache() - val nfeatures = data.take(1)(0)._2.length - val nexamples = data.count + val nfeatures: Int = data.take(1)(0)._2.length + val nexamples: Long = data.count() // Compute XtX - Size of XtX is nfeatures by nfeatures - val XtX = data.map { - case (y, features) => - val x = new DoubleMatrix(1, features.length, features:_*) - x.transpose().mmul(x) + val XtX: DoubleMatrix = data.map { case (y, features) => + val x = new DoubleMatrix(1, features.length, features:_*) + x.transpose().mmul(x) }.reduce(_.add(_)) // Compute Xt*y - Size of Xty is nfeatures by 1 - val Xty = data.map { - case (y, features) => - new DoubleMatrix(features.length, 1, features:_*).mul(y) + val Xty: DoubleMatrix = data.map { case (y, features) => + new DoubleMatrix(features.length, 1, features:_*).mul(y) }.reduce(_.add(_)) // Define a function to compute the leave one out cross validation error // for a single example - def crossValidate(lambda: Double) = { - // Compute the MLE ridge regression parameter value + def crossValidate(lambda: Double): (Double, Double, DoubleMatrix) = { + // Compute the MLE ridge regression parameter value // Ridge Regression parameter = inv(XtX + \lambda*I) * Xty val XtXlambda = DoubleMatrix.eye(nfeatures).muli(lambda).addi(XtX) val w = Solve.solveSymmetric(XtXlambda, Xty) - val invXtX = Solve.solveSymmetric(XtXlambda, - DoubleMatrix.eye(nfeatures)) - + val invXtX = Solve.solveSymmetric(XtXlambda, DoubleMatrix.eye(nfeatures)) + // compute the leave one out cross validation score val cvError = data.map { case (y, features) => @@ -74,11 +69,12 @@ object RidgeRegression extends Logging { val lowValue = crossValidate((mid - low) / 2 + low) val highValue = crossValidate((high - mid) / 2 + mid) val (newLow, newHigh) = if (lowValue._2 < highValue._2) { - (low, mid + (high-low)/4) + (low, mid + (high-low)/4) } else { (mid - (high-low)/4, high) } if (newHigh - newLow > 1.0E-7) { + // :: is list prepend in Scala. lowValue :: highValue :: binSearch(newLow, newHigh) } else { List(lowValue, highValue) @@ -88,7 +84,7 @@ object RidgeRegression extends Logging { // Actually compute the best lambda val lambdas = binSearch(lambdaLow, lambdaHigh).sortBy(_._1) - // Find the best parameter set + // Find the best parameter set by taking the lowest cverror. val (lambdaOpt, cverror, wOpt) = lambdas.reduce((a, b) => if (a._2 < b._2) a else b) logInfo("RidgeRegression: optimal lambda " + lambdaOpt) diff --git a/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala b/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala index 22a1e4613b..ff8640bb50 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala @@ -1,11 +1,11 @@ package spark.ml -import spark._ -import spark.SparkContext._ +import spark.{RDD, SparkContext} import org.apache.commons.math3.distribution.NormalDistribution import org.jblas.DoubleMatrix + object RidgeRegressionGenerator { // Helper methods to load and save data used for RidgeRegression @@ -23,30 +23,34 @@ object RidgeRegressionGenerator { data } - def saveData(data: RDD[(Double, Array[Double])], dir: String) { + private def saveData(data: RDD[(Double, Array[Double])], dir: String) { val dataStr = data.map(x => x._1 + "," + x._2.mkString(" ")) dataStr.saveAsTextFile(dir) } def main(args: Array[String]) { if (args.length != 2) { - println("Usage: RidgeRegressionGenerator ") + println("Usage: RidgeRegressionGenerator " + + " ") System.exit(1) } - org.jblas.util.Random.seed(42) - val sc = new SparkContext(args(0), "RidgeRegressionGenerator") - val nexamples = 1000 - val nfeatures = 100 + 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 parts = 2 + + 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 = sc.parallelize(0 until parts, parts).flatMap { p => + 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 @@ -56,15 +60,15 @@ object RidgeRegressionGenerator { val rnd = new NormalDistribution(0, eps) rnd.reseedRandomGenerator(42 + p) - val normalValues = (0 until examplesInPartition).map(_ => rnd.sample()) - val yObs = new DoubleMatrix(examplesInPartition, 1, normalValues:_*).addi(y) - - (0 until examplesInPartition).map(i => + val normalValues = Array.fill[Double](examplesInPartition)(rnd.sample()) + val yObs = new DoubleMatrix(normalValues).addi(y) + + Iterator.tabulate(examplesInPartition) { i => (yObs.get(i, 0), X.getRow(i).toArray) - ) + } } - saveData(data, args(1)) - System.exit(0) + saveData(data, outputPath) + sc.stop() } } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 731671c23b..aa877ad4a7 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -221,7 +221,7 @@ object SparkBuild extends Build { def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") - def mlSettings = examplesSettings ++ Seq( + def mlSettings = sharedSettings ++ Seq( name := "spark-ml", libraryDependencies ++= Seq( "org.jblas" % "jblas" % "1.2.3", -- cgit v1.2.3 From c070decb8e746bb295b3d95f49713d0d389f92b3 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 6 Jun 2013 11:04:33 -0700 Subject: Add methods to normalize the data before training Also update model after training based appropriately. --- ml/src/main/scala/spark/ml/RidgeRegression.scala | 124 ++++++++++++++++++++--- 1 file changed, 111 insertions(+), 13 deletions(-) diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala index b8b632e111..881316c0fc 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -1,6 +1,7 @@ package spark.ml import spark.{Logging, RDD, SparkContext} +import spark.SparkContext._ import org.jblas.DoubleMatrix import org.jblas.Solve @@ -11,20 +12,32 @@ import org.jblas.Solve class RidgeRegressionModel( val wOpt: DoubleMatrix, + val bOpt: Double, val lambdaOpt: Double, val lambdas: List[(Double, Double, DoubleMatrix)]) { def predict(test_data: spark.RDD[Array[Double]]) = { - test_data.map(x => new DoubleMatrix(1, x.length, x:_*).mmul(this.wOpt)) + test_data.map(x => (new DoubleMatrix(1, x.length, x:_*).mmul(this.wOpt)).get(0) + this.bOpt) } } +case class RidgeRegressionData( + val data: RDD[(Double, Array[Double])], + val normalizedData: RDD[(Double, Array[Double])], + val yMean: Double, + val xColMean: Array[Double], + val xColSd: Array[Double] +) + object RidgeRegression extends Logging { - def train(data: RDD[(Double, Array[Double])], + def train(inputData: RDD[(Double, Array[Double])], lambdaLow: Double = 0.0, lambdaHigh: Double = 10000.0) = { + val ridgeData = normalize(inputData) + val data = ridgeData.normalizedData + data.cache() val nfeatures: Int = data.take(1)(0)._2.length val nexamples: Long = data.count() @@ -51,19 +64,27 @@ object RidgeRegression extends Logging { val invXtX = Solve.solveSymmetric(XtXlambda, DoubleMatrix.eye(nfeatures)) - // compute the leave one out cross validation score - val cvError = data.map { + // compute the generalized cross validation score + // TODO: Is there a way to calculate this using one pass on the data ? + val H_ii_mean = data.map { case (y, features) => val x = new DoubleMatrix(features.length, 1, features:_*) - val yhat = w.transpose().mmul(x).get(0) val H_ii = x.transpose().mmul(invXtX).mmul(x).get(0) - val residual = (y - yhat) / (1.0 - H_ii) + H_ii + }.reduce(_ + _) / nexamples + + val gcv = data.map { + case (y, features) => + val x = new DoubleMatrix(features.length, 1, features:_*) + val yhat = w.transpose().mmul(x).get(0) + val residual = (y - yhat) / (1.0 - H_ii_mean) residual * residual - }.reduce(_ + _) - (lambda, cvError, w) + }.reduce(_ + _) / nexamples + + (lambda, gcv, w) } - // Binary search for the best assignment to lambda. + // Binary search for the best assignment to lambda. def binSearch(low: Double, high: Double): List[(Double, Double, DoubleMatrix)] = { val mid = (high - low) / 2 + low val lowValue = crossValidate((mid - low) / 2 + low) @@ -87,12 +108,89 @@ object RidgeRegression extends Logging { // Find the best parameter set by taking the lowest cverror. val (lambdaOpt, cverror, wOpt) = lambdas.reduce((a, b) => if (a._2 < b._2) a else b) - logInfo("RidgeRegression: optimal lambda " + lambdaOpt) + // Return the model which contains the solution + val trainModel = new RidgeRegressionModel(wOpt, 0.0, lambdaOpt, lambdas) + val normModel = normalizeModel(trainModel, ridgeData.xColSd, ridgeData.xColMean, ridgeData.yMean) + + logInfo("RidgeRegression: optimal lambda " + normModel.lambdaOpt) + logInfo("RidgeRegression: optimal weights " + normModel.wOpt) + logInfo("RidgeRegression: optimal intercept " + normModel.bOpt) + logInfo("RidgeRegression: optimal GCV " + cverror) + + normModel + } + + /** + * yMu = Mean[Y] + * xMuVec = Mean[X] + * xSigmaVec = StdDev[X] + * + * // Shift the data + * Xtrain = (X - xMuVec) / xSigmaVec + * Ytrain = Y - yMu + */ + def normalize(data: RDD[(Double, Array[Double])]) = { + data.cache() + + val nexamples: Long = data.count() + val nfeatures: Int = data.take(1)(0)._2.length + + // Calculate the mean for Y + val yMean: Double = data.map { case (y, features) => y }.reduce(_ + _) / nexamples - // Return the model which contains the solution - new RidgeRegressionModel(wOpt, lambdaOpt, lambdas) + // NOTE: We shuffle X by column here to compute column sum and sum of squares. + val xColSumSq: RDD[(Int, (Double, Double))] = data.flatMap { case(y, features) => + val nCols = features.length + // Traverse over every column and emit (col, value, value^2) + (0 until nCols).map(i => (i, (features(i), features(i)*features(i)))) + }.reduceByKey { case(x1, x2) => + (x1._1 + x2._1, x1._2 + x2._2) + } + val xColSumsMap = xColSumSq.collectAsMap() + + // Compute mean and unbiased variance using column sums + val xColMeans = (0 until nfeatures).map(x => xColSumsMap(x)._1 / nexamples).toArray + val xColSd = (0 until nfeatures).map {x => + val v = (xColSumsMap(x)._2 - (math.pow(xColSumsMap(x)._1, 2) / nexamples)) / (nexamples) + math.sqrt(v) + }.toArray + + // Shift the data + val normalizedData = data.map { case(y, features) => + val yNormalized = y - yMean + val featuresNormalized = (0 until nfeatures).map( + column => (features(column) - xColMeans(column)) / xColSd(column) + ).toArray + (yNormalized, featuresNormalized) + } + new RidgeRegressionData(data, normalizedData, yMean, xColMeans, xColSd) } + /** + * Augment and return then final model (derivation): + * y = w' ( (xPred - xMu) / xSigma ) + yMu + * y = w' xPred/sigma + (yMu - w' (xMu/ xSigmaVec) + * Note that the / operator is point wise divions + * + * model.w = w' / sigma // point wise division + * model.b = yMu - w' * (xMu / xSigmaVec) // scalar offset + * + * // Make predictions + * yPred = model.w' * xPred + model.b + */ + def normalizeModel(model: RidgeRegressionModel, + xColSd: Array[Double], xColMeans: Array[Double], + yMean: Double) = { + val colSdMat = new DoubleMatrix(xColSd.length, 1, xColSd:_*) + val colMeanMat = new DoubleMatrix(xColMeans.length, 1, xColMeans:_*) + + val wOpt = model.wOpt.div(colSdMat) + val bOpt = yMean - model.wOpt.transpose().mmul(colMeanMat.div(colSdMat)).get(0) + + new RidgeRegressionModel(wOpt, bOpt, model.lambdaOpt, model.lambdas) + } + + def main(args: Array[String]) { if (args.length != 2) { println("Usage: RidgeRegression ") @@ -100,7 +198,7 @@ object RidgeRegression extends Logging { } val sc = new SparkContext(args(0), "RidgeRegression") val data = RidgeRegressionGenerator.loadData(sc, args(1)) - val model = train(data, 0, 100) + val model = train(data, 0, 1000) sc.stop() } } -- cgit v1.2.3 From c8169c0a3331c8c89cd963f8f14ea69dbe3ef966 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 6 Jun 2013 11:05:11 -0700 Subject: Add LPSA data set. Data from http://www-stat.stanford.edu/~tibs/ElemStatLearn/datasets/prostate.data --- ml/data/ridge-data/lpsa.data | 67 ++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 67 insertions(+) create mode 100644 ml/data/ridge-data/lpsa.data diff --git a/ml/data/ridge-data/lpsa.data b/ml/data/ridge-data/lpsa.data new file mode 100644 index 0000000000..fdd16e36b4 --- /dev/null +++ b/ml/data/ridge-data/lpsa.data @@ -0,0 +1,67 @@ +-0.4307829,-1.63735562648104 -2.00621178480549 -1.86242597251066 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +-0.1625189,-1.98898046126935 -0.722008756122123 -0.787896192088153 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +-0.1625189,-1.57881887548545 -2.1887840293994 1.36116336875686 -1.02470580167082 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.155348103855541 +-0.1625189,-2.16691708463163 -0.807993896938655 -0.787896192088153 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +0.3715636,-0.507874475300631 -0.458834049396776 -0.250631301876899 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +0.7654678,-2.03612849966376 -0.933954647105133 -1.86242597251066 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +0.8544153,-0.557312518810673 -0.208756571683607 -0.787896192088153 0.990146852537193 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.2669476,-0.929360463147704 -0.0578991819441687 0.152317365781542 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.2669476,-2.28833047634983 -0.0706369432557794 -0.116315079324086 0.80409888772376 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.2669476,0.223498042876113 -1.41471935455355 -0.116315079324086 -1.02470580167082 -0.522940888712441 -0.29928234305568 0.342627053981254 0.199211097885341 +1.3480731,0.107785900236813 -1.47221551299731 0.420949810887169 -1.02470580167082 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.687186906466865 +1.446919,0.162180092313795 -1.32557369901905 0.286633588334355 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.4701758,-1.49795329918548 -0.263601072284232 0.823898478545609 0.788388310173035 -0.522940888712441 -0.29928234305568 0.342627053981254 0.199211097885341 +1.4929041,0.796247055396743 0.0476559407005752 0.286633588334355 -1.02470580167082 -0.522940888712441 0.394013435896129 -1.04215728919298 -0.864466507337306 +1.5581446,-1.62233848461465 -0.843294091975396 -3.07127197548598 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.5993876,-0.990720665490831 0.458513517212311 0.823898478545609 1.07379746308195 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.6389967,-0.171901281967138 -0.489197399065355 -0.65357996953534 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.6956156,-1.60758252338831 -0.590700340358265 -0.65357996953534 -0.619561070667254 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.7137979,0.366273918511144 -0.414014962912583 -0.116315079324086 0.232904453212813 -0.522940888712441 0.971228997418125 0.342627053981254 1.26288870310799 +1.8000583,-0.710307384579833 0.211731938156277 0.152317365781542 -1.02470580167082 -0.522940888712441 -0.442797990776478 0.342627053981254 1.61744790484887 +1.8484548,-0.262791728113881 -1.16708345615721 0.420949810887169 0.0846342590816532 -0.522940888712441 0.163172393491611 0.342627053981254 1.97200710658975 +1.8946169,0.899043117369237 -0.590700340358265 0.152317365781542 -1.02470580167082 -0.522940888712441 1.28643254437683 -1.04215728919298 -0.864466507337306 +1.9242487,-0.903451690500615 1.07659722048274 0.152317365781542 1.28380453408541 -0.522940888712441 -0.442797990776478 -1.04215728919298 -0.864466507337306 +2.008214,-0.0633337899773081 -1.38088970920094 0.958214701098423 0.80409888772376 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +2.0476928,-1.15393789990757 -0.961853075398404 -0.116315079324086 -1.02470580167082 -0.522940888712441 -0.442797990776478 -1.04215728919298 -0.864466507337306 +2.1575593,0.0620203721138446 0.0657973885499142 1.22684714620405 -0.468824786336838 -0.522940888712441 1.31421001659859 1.72741139715549 -0.332627704725983 +2.1916535,-0.75731027755674 -2.92717970468456 0.018001143228728 -1.02470580167082 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.332627704725983 +2.2137539,1.11226993252773 1.06484916245061 0.555266033439982 0.877691038550889 1.89254797819741 1.43890404648442 0.342627053981254 0.376490698755783 +2.2772673,-0.468768642850639 -1.43754788774533 -1.05652863719378 0.576050411655607 -0.522940888712441 0.0120483832567209 0.342627053981254 -0.687186906466865 +2.2975726,-0.618884859896728 -1.1366360750781 -0.519263746982526 -1.02470580167082 -0.522940888712441 -0.863171185425945 3.11219574032972 1.97200710658975 +2.3272777,-0.651431999123483 0.55329161145762 -0.250631301876899 1.11210019001038 -0.522940888712441 -0.179808625688859 -1.04215728919298 -0.864466507337306 +2.5217206,0.115499102435224 -0.512233676577595 0.286633588334355 1.13650173283446 -0.522940888712441 -0.179808625688859 0.342627053981254 -0.155348103855541 +2.5533438,0.266341329949937 -0.551137885443386 -0.384947524429713 0.354857790686005 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.332627704725983 +2.5687881,1.16902610257751 0.855491905752846 2.03274448152093 1.22628985326088 1.89254797819741 2.02833774827712 3.11219574032972 2.68112551007152 +2.6567569,-0.218972367124187 0.851192298581141 0.555266033439982 -1.02470580167082 -0.522940888712441 -0.863171185425945 0.342627053981254 0.908329501367106 +2.677591,0.263121415733908 1.4142681068416 0.018001143228728 1.35980653053822 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +2.7180005,-0.0704736333296423 1.52000996595417 0.286633588334355 1.39364261119802 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.332627704725983 +2.7942279,-0.751957286017338 0.316843561689933 -1.99674219506348 0.911736065044475 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +2.8063861,-0.685277652430997 1.28214038482516 0.823898478545609 0.232904453212813 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.155348103855541 +2.8124102,-0.244991501432929 0.51882005949686 -0.384947524429713 0.823246560137838 -0.522940888712441 -0.863171185425945 0.342627053981254 0.553770299626224 +2.8419982,-0.75731027755674 2.09041984898851 1.22684714620405 1.53428167116843 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +2.8535925,1.20962937075363 -0.242882661178889 1.09253092365124 -1.02470580167082 -0.522940888712441 1.24263233939889 3.11219574032972 2.50384590920108 +2.9204698,0.570886990493502 0.58243883987948 0.555266033439982 1.16006887775962 -0.522940888712441 1.07357183940747 0.342627053981254 1.61744790484887 +2.9626924,0.719758684343624 0.984970304132004 1.09253092365124 1.52137230773457 -0.522940888712441 -0.179808625688859 0.342627053981254 -0.509907305596424 +2.9626924,-1.52406140158064 1.81975700990333 0.689582255992796 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +2.9729753,-0.132431544081234 2.68769877553723 1.09253092365124 1.53428167116843 -0.522940888712441 -0.442797990776478 0.342627053981254 -0.687186906466865 +3.0130809,0.436161292804989 -0.0834447307428255 -0.519263746982526 -1.02470580167082 1.89254797819741 1.07357183940747 0.342627053981254 1.26288870310799 +3.0373539,-0.161195191984091 -0.671900359186746 1.7641120364153 1.13650173283446 -0.522940888712441 -0.863171185425945 0.342627053981254 0.0219314970149 +3.2752562,1.39927182372944 0.513852869452676 0.689582255992796 -1.02470580167082 1.89254797819741 1.49394503405693 0.342627053981254 -0.155348103855541 +3.3375474,1.51967002306341 -0.852203755696565 0.555266033439982 -0.104527297798983 1.89254797819741 1.85927724828569 0.342627053981254 0.908329501367106 +3.3928291,0.560725834706224 1.87867703391426 1.09253092365124 1.39364261119802 -0.522940888712441 0.486423065822545 0.342627053981254 1.26288870310799 +3.4355988,1.00765532502814 1.69426310090641 1.89842825896812 1.53428167116843 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.509907305596424 +3.4578927,1.10152996153577 -0.10927271844907 0.689582255992796 -1.02470580167082 1.89254797819741 1.97630171771485 0.342627053981254 1.61744790484887 +3.5160131,0.100001934217311 -1.30380956369388 0.286633588334355 0.316555063757567 -0.522940888712441 0.28786643052924 0.342627053981254 0.553770299626224 +3.5307626,0.987291634724086 -0.36279314978779 -0.922212414640967 0.232904453212813 -0.522940888712441 1.79270085261407 0.342627053981254 1.26288870310799 +3.5652984,1.07158528137575 0.606453149641961 1.7641120364153 -0.432854616994416 1.89254797819741 0.528504607720369 0.342627053981254 0.199211097885341 +3.5876769,0.180156323255198 0.188987436375017 -0.519263746982526 1.09956763075594 -0.522940888712441 0.708239632330506 0.342627053981254 0.199211097885341 +3.6309855,1.65687973755377 -0.256675483533719 0.018001143228728 -1.02470580167082 1.89254797819741 1.79270085261407 0.342627053981254 1.26288870310799 +3.6800909,0.5720085322365 0.239854450210939 -0.787896192088153 1.0605418233138 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +3.7123518,0.323806133438225 -0.606717660886078 -0.250631301876899 -1.02470580167082 1.89254797819741 0.342907418101747 0.342627053981254 0.199211097885341 +3.9843437,1.23668206715898 2.54220539083611 0.152317365781542 -1.02470580167082 1.89254797819741 1.89037692416194 0.342627053981254 1.26288870310799 +3.993603,0.180156323255198 0.154448192444669 1.62979581386249 0.576050411655607 1.89254797819741 0.708239632330506 0.342627053981254 1.79472750571931 +4.029806,1.60906277046565 1.10378605019827 0.555266033439982 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +4.1295508,1.0036214996026 0.113496885050331 -0.384947524429713 0.860016436332751 1.89254797819741 -0.863171185425945 0.342627053981254 -0.332627704725983 +4.3851468,1.25591974271076 0.577607033774471 0.555266033439982 -1.02470580167082 1.89254797819741 1.07357183940747 0.342627053981254 1.26288870310799 +4.6844434,2.09650591351268 0.625488598331018 -2.66832330782754 -1.02470580167082 1.89254797819741 1.67954222367555 0.342627053981254 0.553770299626224 +5.477509,1.30028987435881 0.338383613253713 0.555266033439982 1.00481276295349 1.89254797819741 1.24263233939889 0.342627053981254 1.97200710658975 -- cgit v1.2.3 From 4dc13bf5be713443f22ca9c27530fb1d3cf48993 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 7 Jun 2013 15:10:13 -0700 Subject: Revert back to closed form CV error --- ml/src/main/scala/spark/ml/RidgeRegression.scala | 17 +++++------------ 1 file changed, 5 insertions(+), 12 deletions(-) diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala index 881316c0fc..2288a284e2 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -65,23 +65,16 @@ object RidgeRegression extends Logging { val invXtX = Solve.solveSymmetric(XtXlambda, DoubleMatrix.eye(nfeatures)) // compute the generalized cross validation score - // TODO: Is there a way to calculate this using one pass on the data ? - val H_ii_mean = data.map { - case (y, features) => - val x = new DoubleMatrix(features.length, 1, features:_*) - val H_ii = x.transpose().mmul(invXtX).mmul(x).get(0) - H_ii - }.reduce(_ + _) / nexamples - - val gcv = data.map { + val cverror = data.map { case (y, features) => val x = new DoubleMatrix(features.length, 1, features:_*) val yhat = w.transpose().mmul(x).get(0) - val residual = (y - yhat) / (1.0 - H_ii_mean) + val H_ii = x.transpose().mmul(invXtX).mmul(x).get(0) + val residual = (y - yhat) / (1.0 - H_ii) residual * residual }.reduce(_ + _) / nexamples - (lambda, gcv, w) + (lambda, cverror, w) } // Binary search for the best assignment to lambda. @@ -115,7 +108,7 @@ object RidgeRegression extends Logging { logInfo("RidgeRegression: optimal lambda " + normModel.lambdaOpt) logInfo("RidgeRegression: optimal weights " + normModel.wOpt) logInfo("RidgeRegression: optimal intercept " + normModel.bOpt) - logInfo("RidgeRegression: optimal GCV " + cverror) + logInfo("RidgeRegression: cross-validation error " + cverror) normModel } -- cgit v1.2.3 From b9d9b6f98168ae211e36e94ee4c2d8defa525169 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sat, 8 Jun 2013 17:04:30 -0700 Subject: Add a unit test for Ridge Regression --- ml/src/test/resources/log4j.properties | 11 ++++++ .../test/scala/spark/ml/RidgeRegressionSuite.scala | 41 ++++++++++++++++++++++ 2 files changed, 52 insertions(+) create mode 100644 ml/src/test/resources/log4j.properties create mode 100644 ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala diff --git a/ml/src/test/resources/log4j.properties b/ml/src/test/resources/log4j.properties new file mode 100644 index 0000000000..390c92763c --- /dev/null +++ b/ml/src/test/resources/log4j.properties @@ -0,0 +1,11 @@ +# Set everything to be logged to the file core/target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=ml/target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN + diff --git a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala new file mode 100644 index 0000000000..cc46602edd --- /dev/null +++ b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala @@ -0,0 +1,41 @@ +package spark.ml + +import spark.SparkContext +import spark.SparkContext._ + +import org.apache.commons.math3.distribution.NormalDistribution +import org.scalatest.FunSuite + +class RidgeRegressionSuite extends FunSuite { + + // Test if we can correctly learn Y = 3 + X1 + X2 when + // X1 and X2 are collinear. + test("multi-collinear variables") { + val rnd = new NormalDistribution(0, 1) + rnd.reseedRandomGenerator(43) + val sc = new SparkContext("local", "test") + val x1 = Array.fill[Double](20)(rnd.sample()) + + // Pick a mean close to mean of x1 + val rnd1 = new NormalDistribution(0.1, 0.01) + rnd1.reseedRandomGenerator(42) + val x2 = Array.fill[Double](20)(rnd1.sample()) + + val xMat = (0 until 20).map(i => Array(x1(i), x2(i))).toArray + + val y = xMat.map(i => 3 + i(0) + i(1)) + val testData = (0 until 20).map(i => (y(i), xMat(i))).toArray + + val testRDD = sc.parallelize(testData, 2) + val model = RidgeRegression.train(testRDD, 0, 10) + + assert(model.bOpt >= 2.9 && model.bOpt <= 3.1) + assert(model.wOpt.length === 2) + assert(model.wOpt.get(0) >= 0.9 && model.wOpt.get(0) <= 1.1) + assert(model.wOpt.get(1) >= 0.9 && model.wOpt.get(1) <= 1.1) + + sc.stop() + // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown + System.clearProperty("spark.driver.port") + } +} -- cgit v1.2.3 From 282c8ed78843fc9a3555b4b6dbdb709d2632d360 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 10 Jun 2013 23:16:11 -0700 Subject: Add LogisticRegression using StochasticGradientDescent. Also refactor RidgeRegression and LogisticRegression to re-use code and update the test as well --- ml/src/main/scala/spark/ml/Gradient.scala | 24 ++++ ml/src/main/scala/spark/ml/GradientDescent.scala | 46 +++++++ .../main/scala/spark/ml/LogisticRegression.scala | 77 ++++++++++++ .../spark/ml/LogisticRegressionGenerator.scala | 40 ++++++ ml/src/main/scala/spark/ml/MLUtils.scala | 27 +++++ ml/src/main/scala/spark/ml/Regression.scala | 53 ++++++++ ml/src/main/scala/spark/ml/RidgeRegression.scala | 134 ++++++--------------- .../scala/spark/ml/RidgeRegressionGenerator.scala | 24 +--- ml/src/main/scala/spark/ml/Updater.scala | 16 +++ .../test/scala/spark/ml/RidgeRegressionSuite.scala | 8 +- 10 files changed, 327 insertions(+), 122 deletions(-) create mode 100644 ml/src/main/scala/spark/ml/Gradient.scala create mode 100644 ml/src/main/scala/spark/ml/GradientDescent.scala create mode 100644 ml/src/main/scala/spark/ml/LogisticRegression.scala create mode 100644 ml/src/main/scala/spark/ml/LogisticRegressionGenerator.scala create mode 100644 ml/src/main/scala/spark/ml/MLUtils.scala create mode 100644 ml/src/main/scala/spark/ml/Regression.scala create mode 100644 ml/src/main/scala/spark/ml/Updater.scala diff --git a/ml/src/main/scala/spark/ml/Gradient.scala b/ml/src/main/scala/spark/ml/Gradient.scala new file mode 100644 index 0000000000..a7e8327133 --- /dev/null +++ b/ml/src/main/scala/spark/ml/Gradient.scala @@ -0,0 +1,24 @@ +package spark.ml + +import org.jblas.DoubleMatrix + +abstract class Gradient extends Serializable { + def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): + (DoubleMatrix, Double) +} + +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) + } +} diff --git a/ml/src/main/scala/spark/ml/GradientDescent.scala b/ml/src/main/scala/spark/ml/GradientDescent.scala new file mode 100644 index 0000000000..80d4c44a13 --- /dev/null +++ b/ml/src/main/scala/spark/ml/GradientDescent.scala @@ -0,0 +1,46 @@ +package spark.ml + +import spark.{Logging, RDD, SparkContext} +import spark.SparkContext._ + +import org.jblas.DoubleMatrix + +import scala.collection.mutable.ArrayBuffer + + +object GradientDescent { + + def runMiniBatchSGD( + data: RDD[(Double, Array[Double])], + gradient: Gradient, + updater: Updater, + stepSize: Double, + numIters: Int, + miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { + + val lossHistory = new ArrayBuffer[Double] + + val nfeatures: Int = data.take(1)(0)._2.length + val nexamples: Long = data.count() + val miniBatchSize = nexamples * miniBatchFraction + + var weights = DoubleMatrix.ones(nfeatures) + var reg_val = 0.0 + + for (i <- 1 to numIters) { + val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42).map { + case (y, features) => + val featuresRow = new DoubleMatrix(features.length, 1, features:_*) + val (grad, loss) = gradient.compute(featuresRow, y, weights) + (grad, loss) + }.reduce((a, b) => (a._1.add(b._1), a._2 + b._2)) + + lossHistory.append(lossSum / miniBatchSize + reg_val) + val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i) + weights = update._1 + reg_val = update._2 + } + + (weights, lossHistory.toArray) + } +} diff --git a/ml/src/main/scala/spark/ml/LogisticRegression.scala b/ml/src/main/scala/spark/ml/LogisticRegression.scala new file mode 100644 index 0000000000..5647bf5a84 --- /dev/null +++ b/ml/src/main/scala/spark/ml/LogisticRegression.scala @@ -0,0 +1,77 @@ +package spark.ml + +import spark.{Logging, RDD, SparkContext} +import spark.SparkContext._ + +import org.jblas.DoubleMatrix + +/** + * Logistic Regression using Stochastic Gradient Descent. + */ +class LogisticRegressionModel( + weights: DoubleMatrix, + intercept: Double, + val losses: Array[Double]) extends RegressionModel(weights, intercept) { + + override def predict(test_data: spark.RDD[Array[Double]]) = { + test_data.map { x => + val margin = (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept + 1.0/(1.0 + math.exp(margin * -1)) + } + } +} + +class LogisticRegressionData(data: RDD[(Double, Array[Double])]) extends RegressionData(data) { + override def normalizeData() = { + // Shift only the features for LogisticRegression + data.map { case(y, features) => + val featuresNormalized = (0 until nfeatures).map( + column => (features(column) - xColMean(column)) / xColSd(column) + ).toArray + (y, featuresNormalized) + } + } + + override def scaleModel(m: RegressionModel) = { + val model = m.asInstanceOf[LogisticRegressionModel] + val colSdMat = new DoubleMatrix(xColSd.length, 1, xColSd:_*) + val colMeanMat = new DoubleMatrix(xColMean.length, 1, xColMean:_*) + + val weights = model.weights.div(colSdMat) + val intercept = -1.0 * model.weights.transpose().mmul(colMeanMat.div(colSdMat)).get(0) + + new LogisticRegressionModel(weights, intercept, model.losses) + } +} + +object LogisticRegression extends Logging { + val STEP_SIZE = 1.0 + val MINI_BATCH_FRACTION = 1.0 + + def train(input: RDD[(Double, Array[Double])], numIters: Int) = { + input.cache() + + val lrData = new LogisticRegressionData(input) + val data = lrData.normalizeData() + val (weights, losses) = GradientDescent.runMiniBatchSGD( + data, new LogisticGradient(), new SimpleUpdater(), STEP_SIZE, numIters, MINI_BATCH_FRACTION) + + val computedModel = new LogisticRegressionModel(weights, 0, losses) + val model = lrData.scaleModel(computedModel) + logInfo("Final model weights " + model.weights) + logInfo("Final model intercept " + model.intercept) + logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(",")) + model + } + + def main(args: Array[String]) { + if (args.length != 3) { + println("Usage: LogisticRegression ") + System.exit(1) + } + val sc = new SparkContext(args(0), "LogisticRegression") + val data = MLUtils.loadData(sc, args(1)) + val model = train(data, args(2).toInt) + sc.stop() + } +} diff --git a/ml/src/main/scala/spark/ml/LogisticRegressionGenerator.scala b/ml/src/main/scala/spark/ml/LogisticRegressionGenerator.scala new file mode 100644 index 0000000000..ff46cfa46d --- /dev/null +++ b/ml/src/main/scala/spark/ml/LogisticRegressionGenerator.scala @@ -0,0 +1,40 @@ +package spark.ml + +import spark.{RDD, SparkContext} + +import org.apache.commons.math3.distribution.NormalDistribution +import org.jblas.DoubleMatrix + +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 NormalDistribution(0, 1) + rnd.reseedRandomGenerator(42 + idx) + + val y = if (idx % 2 == 0) 0 else 1 + val x = Array.fill[Double](nfeatures) { + rnd.sample() + (y * eps) + } + (y, x) + } + + MLUtils.saveData(data, outputPath) + sc.stop() + } +} diff --git a/ml/src/main/scala/spark/ml/MLUtils.scala b/ml/src/main/scala/spark/ml/MLUtils.scala new file mode 100644 index 0000000000..eab5db61bd --- /dev/null +++ b/ml/src/main/scala/spark/ml/MLUtils.scala @@ -0,0 +1,27 @@ +package spark.ml + +import spark.{RDD, SparkContext} + +object MLUtils { + + // Helper methods to load and save data + // Data format: + // , ... + // where , are feature values in Double and + // is the corresponding label as Double + def loadData(sc: SparkContext, dir: String) = { + val data = sc.textFile(dir).map{ line => + val parts = line.split(",") + val label = parts(0).toDouble + val features = parts(1).trim().split(" ").map(_.toDouble) + (label, features) + } + data + } + + def saveData(data: RDD[(Double, Array[Double])], dir: String) { + val dataStr = data.map(x => x._1 + "," + x._2.mkString(" ")) + dataStr.saveAsTextFile(dir) + } + +} diff --git a/ml/src/main/scala/spark/ml/Regression.scala b/ml/src/main/scala/spark/ml/Regression.scala new file mode 100644 index 0000000000..e19d58396b --- /dev/null +++ b/ml/src/main/scala/spark/ml/Regression.scala @@ -0,0 +1,53 @@ +package spark.ml + +import java.io._ + +import spark.{RDD, SparkContext} +import spark.SparkContext._ + +import org.jblas.DoubleMatrix + +abstract class RegressionModel( + val weights: DoubleMatrix, + val intercept: Double) { + + def predict(test_data: RDD[Array[Double]]): RDD[Double] +} + +abstract class RegressionData(val data: RDD[(Double, Array[Double])]) extends Serializable { + var yMean: Double = 0.0 + var xColMean: Array[Double] = null + var xColSd: Array[Double] = null + var nfeatures: Int = 0 + var nexamples: Long = 0 + + // This will populate yMean, xColMean and xColSd + calculateStats() + + def normalizeData(): RDD[(Double, Array[Double])] + def scaleModel(model: RegressionModel): RegressionModel + + def calculateStats() { + this.nexamples = data.count() + this.nfeatures = data.take(1)(0)._2.length + + this.yMean = data.map { case (y, features) => y }.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 { case(y, features) => + val nCols = features.length + // Traverse over every column and emit (col, value, value^2) + (0 until nCols).map(i => (i, (features(i), features(i)*features(i)))) + }.reduceByKey { case(x1, x2) => + (x1._1 + x2._1, x1._2 + x2._2) + } + val xColSumsMap = xColSumSq.collectAsMap() + + // Compute mean and unbiased variance using column sums + this.xColMean = (0 until nfeatures).map(x => xColSumsMap(x)._1 / nexamples).toArray + this.xColSd = (0 until nfeatures).map {x => + val v = (xColSumsMap(x)._2 - (math.pow(xColSumsMap(x)._1, 2) / nexamples)) / (nexamples) + math.sqrt(v) + }.toArray + } +} diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala index 2288a284e2..c030223b85 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -9,25 +9,39 @@ import org.jblas.Solve /** * Ridge Regression from Joseph Gonzalez's implementation in MLBase */ - class RidgeRegressionModel( - val wOpt: DoubleMatrix, - val bOpt: Double, + weights: DoubleMatrix, + intercept: Double, val lambdaOpt: Double, - val lambdas: List[(Double, Double, DoubleMatrix)]) { + val lambdas: List[(Double, Double, DoubleMatrix)]) extends RegressionModel(weights, intercept) { - def predict(test_data: spark.RDD[Array[Double]]) = { - test_data.map(x => (new DoubleMatrix(1, x.length, x:_*).mmul(this.wOpt)).get(0) + this.bOpt) + override def predict(test_data: spark.RDD[Array[Double]]) = { + test_data.map(x => (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept) } } -case class RidgeRegressionData( - val data: RDD[(Double, Array[Double])], - val normalizedData: RDD[(Double, Array[Double])], - val yMean: Double, - val xColMean: Array[Double], - val xColSd: Array[Double] -) +class RidgeRegressionData(data: RDD[(Double, Array[Double])]) extends RegressionData(data) { + override def normalizeData() = { + data.map { case(y, features) => + val yNormalized = y - yMean + val featuresNormalized = (0 until nfeatures).map( + column => (features(column) - xColMean(column)) / xColSd(column) + ).toArray + (yNormalized, featuresNormalized) + } + } + + override def scaleModel(m: RegressionModel) = { + val model = m.asInstanceOf[RidgeRegressionModel] + val colSdMat = new DoubleMatrix(xColSd.length, 1, xColSd:_*) + val colMeanMat = new DoubleMatrix(xColMean.length, 1, xColMean:_*) + + val weights = model.weights.div(colSdMat) + val intercept = yMean - model.weights.transpose().mmul(colMeanMat.div(colSdMat)).get(0) + + new RidgeRegressionModel(weights, intercept, model.lambdaOpt, model.lambdas) + } +} object RidgeRegression extends Logging { @@ -35,12 +49,11 @@ object RidgeRegression extends Logging { lambdaLow: Double = 0.0, lambdaHigh: Double = 10000.0) = { - val ridgeData = normalize(inputData) - val data = ridgeData.normalizedData - - data.cache() - val nfeatures: Int = data.take(1)(0)._2.length - val nexamples: Long = data.count() + inputData.cache() + val ridgeData = new RidgeRegressionData(inputData) + val data = ridgeData.normalizeData() + val nfeatures: Int = ridgeData.nfeatures + val nexamples: Long = ridgeData.nexamples // Compute XtX - Size of XtX is nfeatures by nfeatures val XtX: DoubleMatrix = data.map { case (y, features) => @@ -99,98 +112,27 @@ object RidgeRegression extends Logging { val lambdas = binSearch(lambdaLow, lambdaHigh).sortBy(_._1) // Find the best parameter set by taking the lowest cverror. - val (lambdaOpt, cverror, wOpt) = lambdas.reduce((a, b) => if (a._2 < b._2) a else b) + val (lambdaOpt, cverror, weights) = lambdas.reduce((a, b) => if (a._2 < b._2) a else b) // Return the model which contains the solution - val trainModel = new RidgeRegressionModel(wOpt, 0.0, lambdaOpt, lambdas) - val normModel = normalizeModel(trainModel, ridgeData.xColSd, ridgeData.xColMean, ridgeData.yMean) + val trainModel = new RidgeRegressionModel(weights, 0.0, lambdaOpt, lambdas) + val normModel = ridgeData.scaleModel(trainModel) logInfo("RidgeRegression: optimal lambda " + normModel.lambdaOpt) - logInfo("RidgeRegression: optimal weights " + normModel.wOpt) - logInfo("RidgeRegression: optimal intercept " + normModel.bOpt) + logInfo("RidgeRegression: optimal weights " + normModel.weights) + logInfo("RidgeRegression: optimal intercept " + normModel.intercept) logInfo("RidgeRegression: cross-validation error " + cverror) normModel } - /** - * yMu = Mean[Y] - * xMuVec = Mean[X] - * xSigmaVec = StdDev[X] - * - * // Shift the data - * Xtrain = (X - xMuVec) / xSigmaVec - * Ytrain = Y - yMu - */ - def normalize(data: RDD[(Double, Array[Double])]) = { - data.cache() - - val nexamples: Long = data.count() - val nfeatures: Int = data.take(1)(0)._2.length - - // Calculate the mean for Y - val yMean: Double = data.map { case (y, features) => y }.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 { case(y, features) => - val nCols = features.length - // Traverse over every column and emit (col, value, value^2) - (0 until nCols).map(i => (i, (features(i), features(i)*features(i)))) - }.reduceByKey { case(x1, x2) => - (x1._1 + x2._1, x1._2 + x2._2) - } - val xColSumsMap = xColSumSq.collectAsMap() - - // Compute mean and unbiased variance using column sums - val xColMeans = (0 until nfeatures).map(x => xColSumsMap(x)._1 / nexamples).toArray - val xColSd = (0 until nfeatures).map {x => - val v = (xColSumsMap(x)._2 - (math.pow(xColSumsMap(x)._1, 2) / nexamples)) / (nexamples) - math.sqrt(v) - }.toArray - - // Shift the data - val normalizedData = data.map { case(y, features) => - val yNormalized = y - yMean - val featuresNormalized = (0 until nfeatures).map( - column => (features(column) - xColMeans(column)) / xColSd(column) - ).toArray - (yNormalized, featuresNormalized) - } - new RidgeRegressionData(data, normalizedData, yMean, xColMeans, xColSd) - } - - /** - * Augment and return then final model (derivation): - * y = w' ( (xPred - xMu) / xSigma ) + yMu - * y = w' xPred/sigma + (yMu - w' (xMu/ xSigmaVec) - * Note that the / operator is point wise divions - * - * model.w = w' / sigma // point wise division - * model.b = yMu - w' * (xMu / xSigmaVec) // scalar offset - * - * // Make predictions - * yPred = model.w' * xPred + model.b - */ - def normalizeModel(model: RidgeRegressionModel, - xColSd: Array[Double], xColMeans: Array[Double], - yMean: Double) = { - val colSdMat = new DoubleMatrix(xColSd.length, 1, xColSd:_*) - val colMeanMat = new DoubleMatrix(xColMeans.length, 1, xColMeans:_*) - - val wOpt = model.wOpt.div(colSdMat) - val bOpt = yMean - model.wOpt.transpose().mmul(colMeanMat.div(colSdMat)).get(0) - - new RidgeRegressionModel(wOpt, bOpt, model.lambdaOpt, model.lambdas) - } - - def main(args: Array[String]) { if (args.length != 2) { println("Usage: RidgeRegression ") System.exit(1) } val sc = new SparkContext(args(0), "RidgeRegression") - val data = RidgeRegressionGenerator.loadData(sc, args(1)) + val data = MLUtils.loadData(sc, args(1)) val model = train(data, 0, 1000) sc.stop() } diff --git a/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala b/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala index ff8640bb50..e6d3e94787 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala @@ -8,28 +8,8 @@ import org.jblas.DoubleMatrix object RidgeRegressionGenerator { - // Helper methods to load and save data used for RidgeRegression - // Data format: - // , ... - // where , are feature values in Double and - // is the corresponding label as Double - def loadData(sc: SparkContext, dir: String) = { - val data = sc.textFile(dir).map{ line => - val parts = line.split(",") - val label = parts(0).toDouble - val features = parts(1).trim().split(" ").map(_.toDouble) - (label, features) - } - data - } - - private def saveData(data: RDD[(Double, Array[Double])], dir: String) { - val dataStr = data.map(x => x._1 + "," + x._2.mkString(" ")) - dataStr.saveAsTextFile(dir) - } - def main(args: Array[String]) { - if (args.length != 2) { + if (args.length != 5) { println("Usage: RidgeRegressionGenerator " + " ") System.exit(1) @@ -68,7 +48,7 @@ object RidgeRegressionGenerator { } } - saveData(data, outputPath) + MLUtils.saveData(data, outputPath) sc.stop() } } diff --git a/ml/src/main/scala/spark/ml/Updater.scala b/ml/src/main/scala/spark/ml/Updater.scala new file mode 100644 index 0000000000..bacb059377 --- /dev/null +++ b/ml/src/main/scala/spark/ml/Updater.scala @@ -0,0 +1,16 @@ +package spark.ml + +import org.jblas.DoubleMatrix + +abstract class Updater extends Serializable { + def compute(weightsOlds: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): + (DoubleMatrix, Double) +} + +class SimpleUpdater extends Updater { + override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): + (DoubleMatrix, Double) = { + val normGradient = gradient.mul(stepSize / math.sqrt(iter)) + (weightsOld.sub(normGradient), 0) + } +} diff --git a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala index cc46602edd..50ef472075 100644 --- a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala @@ -29,10 +29,10 @@ class RidgeRegressionSuite extends FunSuite { val testRDD = sc.parallelize(testData, 2) val model = RidgeRegression.train(testRDD, 0, 10) - assert(model.bOpt >= 2.9 && model.bOpt <= 3.1) - assert(model.wOpt.length === 2) - assert(model.wOpt.get(0) >= 0.9 && model.wOpt.get(0) <= 1.1) - assert(model.wOpt.get(1) >= 0.9 && model.wOpt.get(1) <= 1.1) + assert(model.intercept >= 2.9 && model.intercept <= 3.1) + assert(model.weights.length === 2) + assert(model.weights.get(0) >= 0.9 && model.weights.get(0) <= 1.1) + assert(model.weights.get(1) >= 0.9 && model.weights.get(1) <= 1.1) sc.stop() // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown -- cgit v1.2.3 From 48770419bdc09b2cf1ddef3cbdd2c1d23f1748ac Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 10 Jun 2013 23:17:06 -0700 Subject: Add random data used for LR testing. Verified that results match with glm in R --- ml/data/lr-data/random.data | 1000 +++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 1000 insertions(+) create mode 100755 ml/data/lr-data/random.data diff --git a/ml/data/lr-data/random.data b/ml/data/lr-data/random.data new file mode 100755 index 0000000000..29bcb8acba --- /dev/null +++ b/ml/data/lr-data/random.data @@ -0,0 +1,1000 @@ +0.0,-0.19138793197590276 0.7834675900121327 +1.0,3.712420417753061 3.55967640829891 +0.0,-0.3173743619974614 0.9034702789806682 +1.0,4.759494447180777 3.407011867344781 +0.0,-0.7078607074437426 -0.7866705652344417 +1.0,2.6708084832010215 2.5322909406378016 +0.0,-0.07553885038446313 -0.1297104483563081 +1.0,2.759487072285262 2.474689814713741 +0.0,-2.2199161547238107 0.7543109438660762 +1.0,1.922617509832946 1.9412373902594937 +0.0,0.8140942462004225 1.883920822277784 +1.0,1.7649295902120172 3.8195077526061363 +0.0,-1.1173052428096684 -1.468964723960145 +1.0,1.8733449544967458 2.913026590975709 +0.0,-0.11212965215910947 1.068087981775071 +1.0,2.3368459971730227 5.453870208593922 +0.0,-1.2802488543364463 -0.47218504171867676 +1.0,4.1917343620336895 3.5602286778418355 +0.0,0.5995976502137177 -0.797374550890321 +1.0,3.721592294428238 4.824418090974808 +0.0,-0.0721649164244053 -1.3952880192542576 +1.0,3.609764030146346 3.4730043476891277 +0.0,-1.5078269860498976 -2.6460421495665987 +1.0,1.8510254911824193 1.6748364225650059 +0.0,1.021485727769095 -0.14476425336866738 +1.0,4.10105000223134 2.3772502437548493 +0.0,2.6132710211418675 -1.061646527586342 +1.0,2.6444875273854653 4.043302750329545 +0.0,1.115723715938777 0.38401588153403887 +1.0,2.045759949164019 3.156447533448806 +0.0,-1.0543022640565405 -0.6820337845705753 +1.0,3.535337069948117 3.8121122972294965 +0.0,0.9427529503486505 -0.25123516319259886 +1.0,3.9611643301316795 3.3144121016644443 +0.0,-0.15013188927817916 0.8178862482229886 +1.0,3.200504584029051 2.3088398886136057 +0.0,0.819731993393585 -0.47386644109886344 +1.0,3.283317566020217 3.4828146842654513 +0.0,-2.3283941193793303 -0.6148925379529 +1.0,3.901670215294089 3.6356776610143324 +0.0,-0.28635769830042973 0.049586437072917544 +1.0,3.1114746381043927 3.6314805300338775 +0.0,-1.3085536069757229 0.11172767926766304 +1.0,3.3676979357140744 4.689661419564771 +0.0,-1.5820787210442733 1.3226576351191428 +1.0,2.5957586701668207 3.0648240201825923 +0.0,-2.116823743560968 0.272822309954307 +1.0,3.31672509500716 3.870172182480263 +0.0,0.09751166932653511 0.6469052579904877 +1.0,2.0609623373451305 3.9496181906908694 +0.0,0.5238217321419351 -1.2424816480725946 +1.0,3.5731384504449717 5.293293512805712 +0.0,-0.8507917425723299 -1.2243124053200718 +1.0,3.3060954421001867 3.1337045819604565 +0.0,1.5066706426420082 0.04176666807070882 +1.0,4.197316426430547 2.327643377792433 +0.0,-1.8068158696573955 -1.6380836149377855 +1.0,3.568239793850545 3.561688791420822 +0.0,0.4705756905309871 1.1991675114038487 +1.0,4.85003762884306 4.253420553408024 +0.0,0.7595792932847568 0.014062431397674205 +1.0,1.6984862661221896 1.7746925013882613 +0.0,0.1132294255888917 -0.09228036942051128 +1.0,3.766092539171029 2.765647342841482 +0.0,1.053401788561791 -1.0588667339849278 +1.0,2.780021685872393 3.239478188786074 +0.0,0.4042022490052266 1.0982210323828034 +1.0,2.4939569547402063 2.4615506964861273 +0.0,0.4469359967563411 0.3880418183993791 +1.0,2.7943749030887486 3.742182807141721 +0.0,-0.4418685162293727 0.802180923066725 +1.0,3.711213212127241 4.620177703831104 +0.0,0.10737314976605918 -1.5716142960765325 +1.0,4.0522289913808365 3.77562942835957 +0.0,1.4798827061781141 1.1638601205648005 +1.0,3.6758023575825547 3.115500589955362 +0.0,-1.803338141681238 -0.639996207387159 +1.0,2.044667029270621 3.04922768663927 +0.0,-0.06067427095346295 1.394611410740688 +1.0,4.626495834477846 2.995800202291488 +0.0,-0.2770274350630315 0.4521526506693692 +1.0,3.130857841268635 3.76858860814448 +0.0,2.163400739017478 -1.303601716798734 +1.0,2.9131896969824367 3.4288919990054167 +0.0,-0.7145108501670207 1.4189762494365543 +1.0,3.535768896041034 1.4894011726406373 +0.0,1.605614523747256 0.29974289519139824 +1.0,2.413678734728178 2.1826316767457183 +0.0,-0.8821932593373774 0.26432786248412726 +1.0,2.0878695933047116 3.5277388966365177 +0.0,-1.107001191509183 0.38421647065699477 +1.0,2.6462094774496454 2.273786785429519 +0.0,1.0712046043765102 -1.1889735666835115 +1.0,3.7458483094910666 1.3868020542832566 +0.0,-0.8403883736429167 -0.7163969561320671 +1.0,3.3359151000342195 3.2382001552279576 +0.0,0.13309387098922537 0.938761191821517 +1.0,2.083439571838502 3.2204948086228944 +0.0,1.3030219848568272 0.5976630914634896 +1.0,2.7602376200551317 2.200505791897739 +0.0,-0.9458633178207942 0.0490955863627428 +1.0,3.7998466026531883 1.9291683955712686 +0.0,-1.327236501803235 0.06915643957270164 +1.0,3.4740573335685925 2.1080735512507114 +0.0,0.8627688253416859 -1.961802291046532 +1.0,3.5108780392869776 3.9854745964798326 +0.0,-0.69537574439301 0.2436269580373554 +1.0,2.920286302932126 4.704192389485899 +0.0,-2.031190954684878 -0.7843052045579578 +1.0,1.6768848711259499 1.345658047606076 +0.0,0.9234894202027507 -0.38179572928866495 +1.0,3.1710339307651334 4.129874876536583 +0.0,-2.5086697007630376 -0.2638692986795807 +1.0,2.079400422215581 3.124756711992435 +0.0,-0.1388012859869782 0.3698243463601514 +1.0,2.665728164475424 4.574860576068532 +0.0,0.11967116650891912 -0.8792117975750646 +1.0,3.042630437105455 2.7245525508413677 +0.0,0.6078023848042808 -0.7977233104047035 +1.0,3.3340709038589638 4.962729210819017 +0.0,0.6373101353982795 1.1335021278327686 +1.0,3.3821397455119446 4.349379573895378 +0.0,-0.9140176931412027 -0.03428220013900756 +1.0,4.579963977595727 3.8322809335521484 +0.0,-0.43958506434874983 0.21259366700539037 +1.0,2.644701808902675 3.945416465403505 +0.0,-1.119921743746522 -0.2089105317801997 +1.0,2.5480553203091922 3.123344220515146 +0.0,0.8723990414181355 1.11150972420879 +1.0,4.479600967837827 2.8645066949820057 +0.0,-0.003869320481891422 0.24756134775982133 +1.0,3.237294368758498 4.642548547098718 +0.0,0.34643329685515545 0.029869480691029456 +1.0,2.6324740490008893 1.2577448307260846 +0.0,-0.4416403319035849 -1.4597062027342758 +1.0,1.764049052224297 3.649850384544675 +0.0,0.6779287737716254 -1.9489876700506967 +1.0,1.4286669812409405 2.4906452014102416 +0.0,-1.2271599940693638 0.9869686407012563 +1.0,3.6244117441765993 2.36879554315985 +0.0,-0.11422653411940642 0.4741905017884626 +1.0,3.6192153991840694 2.149436181779614 +0.0,0.45425900443207484 -1.357987041493406 +1.0,4.312295702128074 3.7596991900930252 +0.0,-0.35153502234686884 -0.6297451691082592 +1.0,3.4901363450669476 2.0630236379093243 +0.0,-1.5343533005821828 -0.23745688647461852 +1.0,4.775056734905926 5.291243824646301 +0.0,-1.032123659747431 0.8458711875294105 +1.0,2.3091889606097844 3.3688150059111215 +0.0,0.7854236849909306 0.6742463927844289 +1.0,3.284779531346899 2.855746734955609 +0.0,0.380579394855332 -1.2378905330462027 +1.0,2.540193014555953 3.245568950444961 +0.0,-0.5491810448400926 -2.3179482776107894 +1.0,3.481785462949587 1.8870182253717969 +0.0,-0.06833732101790825 2.178923334945784 +1.0,1.1663083809702222 1.8919272314310458 +0.0,-0.7801536433937879 -1.4185984368350903 +1.0,1.457713814592066 3.0323739348144048 +0.0,-0.16377716798970973 0.09678021896691058 +1.0,2.2294515799173094 1.6179126855486068 +0.0,-0.5845552895984718 -0.8095679531228397 +1.0,2.024328902209618 2.4660315284543888 +0.0,0.2037503424802764 1.5767438723426828 +1.0,3.5058983262252643 3.292836693091364 +0.0,-1.4004772080893082 0.6150928060180622 +1.0,4.610936499146778 3.3674445809820313 +0.0,-0.7325641160695897 -3.0469742419403225 +1.0,2.6778956983269926 4.049681967443553 +0.0,-0.3375932473421461 -0.32976087151423067 +1.0,3.975838378562512 1.2032482992228626 +0.0,-1.6622711226380826 -0.6954676646542216 +1.0,3.1601568512397256 2.7472491112914357 +0.0,0.6739969973916968 1.3608866192945286 +1.0,3.097978499063888 3.88429576456391 +0.0,-0.16445244300279913 0.631410854999902 +1.0,4.244875698991619 3.0464568222900477 +0.0,0.1749522197766453 -0.3295077792829936 +1.0,4.158913950688044 1.1836177376726964 +0.0,-1.8286320279969996 -0.6355826362111864 +1.0,2.4795264391445326 0.8073937061906746 +0.0,-0.5095499320702017 -0.8451757050184052 +1.0,3.6489546081475206 2.7405880916534957 +0.0,-0.11733097334574003 0.020300758125140466 +1.0,1.9034123919197892 4.036941742254072 +0.0,-0.4678304671259669 -0.7653895561277071 +1.0,2.555027220737054 4.205906511993216 +0.0,0.1952150967011765 1.2402178923240337 +1.0,3.532371144429582 2.395018092924601 +0.0,1.4682834110821084 2.2292327929025078 +1.0,2.1160331256749663 3.7157102308564824 +0.0,1.3973790173654674 -1.1902799121683607 +1.0,3.4775573554170616 3.0459058509488557 +0.0,-2.215337088722839 0.7693588032777773 +1.0,2.3298220860458976 1.5924630285528396 +0.0,1.260641664088144 1.5474089692944746 +1.0,4.460878990061944 2.595950219349794 +0.0,-1.8214944389802914 -1.9733205363211535 +1.0,4.41874870213851 2.4975116019313264 +0.0,1.2037921250123007 -0.7057578432831773 +1.0,3.042628088030598 3.7366256492570136 +0.0,-0.02609770715133313 -0.01975791007372346 +1.0,1.123824442324706 3.5115607224884466 +0.0,0.3466005704292144 -1.206858960323042 +1.0,3.044152779557358 2.4308738719304266 +0.0,-0.8292396838183249 -0.5768591341562801 +1.0,2.9898679252543325 3.3291086316901484 +0.0,0.6033357093153775 0.18738779274832332 +1.0,3.2777482224094916 2.2676548172839714 +0.0,-0.7104360487845565 -1.0365712508175688 +1.0,2.617802272534323 1.887796671556582 +0.0,-0.21008998836798706 -2.4424443035468957 +1.0,3.9387085143031317 2.368798316318223 +0.0,-0.65027380204969 0.4757828709083824 +1.0,1.6786020855223545 1.62019388696364 +0.0,0.40325101156361803 0.26629562725726075 +1.0,2.4614637796912167 2.778406744842399 +0.0,-0.4327374795655596 0.5643009301153851 +1.0,2.6419358755663103 2.1911675067034206 +0.0,-0.06058610052148417 0.6118154934715632 +1.0,4.134485645832481 4.214482766162727 +0.0,-2.091472947105952 -0.21279450874188077 +1.0,3.7664041746453503 0.5848083052756543 +0.0,0.20187441248519114 0.7310035835212488 +1.0,3.6821251396696817 1.2016937526237272 +0.0,0.16248871053987612 -0.8547163523143474 +1.0,3.1725037691095834 3.051265058839004 +0.0,-1.7466975308858639 -0.048497170816597705 +1.0,4.296665913992498 4.432036327276331 +0.0,-0.49371042139965376 -1.3162216335880739 +1.0,3.0767376272412292 2.4082404056282467 +0.0,0.6517145281009619 -0.15229289422910688 +1.0,3.8556129079007406 4.932746403550176 +0.0,2.467072616559744 -0.6570760874457315 +1.0,3.8722558954619446 2.398547361219584 +0.0,-0.996362973160808 -0.24663573264285635 +1.0,2.058960472055059 0.09020868936476445 +0.0,1.1921444033047794 -1.2205820383864918 +1.0,3.499255855340612 4.26015377680707 +0.0,0.46495431359796363 -0.3535071804767937 +1.0,3.2772715993311534 1.8496849599545144 +0.0,0.9200766227075026 1.0153595739730128 +1.0,3.7395665378166516 4.161859093428991 +0.0,-1.3445731221950805 0.3711182438638966 +1.0,1.974184816991473 2.3758202020218637 +0.0,0.25747673028745044 1.4898729695115611 +1.0,3.643667737073963 2.5171980898063024 +0.0,-0.7491175934837044 1.807998586131331 +1.0,3.024294668483263 2.745713910567566 +0.0,-2.9902104324990075 0.48847563269083094 +1.0,2.693457241550706 4.067192099378729 +0.0,1.0010822910854564 1.065617155304199 +1.0,2.6231328305267576 3.2530925652040796 +0.0,-1.569524799794976 0.10080365850268516 +1.0,5.543177898986999 3.149276748958176 +0.0,-0.2697035609845456 -0.3834981890675749 +1.0,5.5737716796876935 3.134627621089238 +0.0,0.16848836970122472 1.7680681560270155 +1.0,2.984578320659214 3.8081853301923743 +0.0,2.00864307305994 -1.1769936806590435 +1.0,2.4301644281026538 1.5357007015355957 +0.0,-1.251515087462618 -1.0023388301407077 +1.0,2.7783106123714036 3.4753675099443138 +0.0,1.2067779830446301 -1.1138369735803868 +1.0,2.660559526103853 0.9246419639107195 +0.0,-0.2120078291751072 0.553871125085326 +1.0,3.2961674182984613 4.1840551114889655 +0.0,-1.7407002661640898 -0.13494920714243758 +1.0,2.61652747199719 2.606431158365525 +0.0,0.1810536358726569 -0.7041543708042312 +1.0,0.6618977487425206 4.43976232230529 +0.0,-1.1056190552516114 -0.26273698119076755 +1.0,3.245745718364984 0.9585399121419127 +0.0,0.451245033031027 0.3966692171364385 +1.0,0.7000962854359294 2.5787278270774685 +0.0,-0.20657738352563298 -0.3054434424581368 +1.0,2.194893094322135 1.2265276851138993 +0.0,1.6478689673866447 -1.2217538409516264 +1.0,2.6520153534620268 4.253943157694819 +0.0,-1.091459682813003 -1.5933476790183565 +1.0,2.381978388803204 2.5725801073346375 +0.0,-1.7089448316753346 -0.40058783295112843 +1.0,4.692976595302646 2.293610804758882 +0.0,-0.8154594160076379 0.9100123432125261 +1.0,1.8893957859271135 2.365552941116367 +0.0,1.4750445045587657 -0.5730495722105764 +1.0,4.627946484342315 4.01023129091373 +0.0,-0.5740578222548407 -0.9010801407945085 +1.0,1.1844352711236998 1.0077910117111921 +0.0,-1.1904557430938465 -0.972229300373332 +1.0,1.9514043869587852 2.6603232743467817 +0.0,-0.11744191317950421 1.8160954524210857 +1.0,2.796337014232012 3.45131164191957 +0.0,1.1908754571951825 1.37388641966138 +1.0,3.1347230127964805 3.4874636513372774 +0.0,1.4279445191621287 0.4142573535049987 +1.0,3.2845746999649457 2.942571828876143 +0.0,1.0418078095097314 -0.515727237947711 +1.0,3.0672407807876674 3.593602465858237 +0.0,0.1070041194341431 0.013584199138111364 +1.0,2.831124413123504 2.5083468687281196 +0.0,1.9088191143015583 1.1943157723052062 +1.0,2.888463730373365 3.8588231186101716 +0.0,0.3344825700647222 1.4902421889158837 +1.0,5.1805240354926285 2.347000348613805 +0.0,-0.14736761539184529 -1.3764336595247777 +1.0,4.945788020165247 4.520764535128319 +0.0,0.48089579766964224 -1.0406729486881927 +1.0,3.115699146536788 3.0271206455481905 +0.0,0.8816867514268375 -0.7885530518936628 +1.0,3.293642905051253 4.129500570671647 +0.0,0.021019117419869213 -1.0983625263034136 +1.0,3.4712873315273884 2.8896550248710255 +0.0,1.336463967380889 0.1782538924176004 +1.0,2.9674559623039674 2.1702990000666977 +0.0,-0.9137873001694705 -1.6488427315604255 +1.0,2.425720985355789 3.336546225859983 +0.0,-2.3622279944776245 0.33443034793657744 +1.0,3.557057454549674 0.9654984504665607 +0.0,0.4924227412613347 0.8572441753897001 +1.0,2.903599258175698 1.9821387894597133 +0.0,-0.562864152759892 -1.41025535274598 +1.0,2.621542267864135 3.0896861639721602 +0.0,-0.9659016052287058 1.8601390770202668 +1.0,2.73394050343452 1.5908844566159697 +0.0,0.316736908826005 0.2857224419323005 +1.0,2.3312567009140532 5.596694984859762 +0.0,0.3137619371424862 -0.1840942808000176 +1.0,3.857644883242267 1.7425846536145542 +0.0,-0.10204795362718587 3.253153279848385 +1.0,1.991635750012152 3.0091345292604816 +0.0,0.6187841242310289 0.9589700354301842 +1.0,2.9773010080735895 3.723750625441197 +0.0,-0.8890787476930039 0.6057780620635984 +1.0,3.2341068438464773 4.238588226643048 +0.0,-0.6100941277292691 -1.5125630779121992 +1.0,3.378840902739636 2.0705801293719017 +0.0,1.9736225258875286 1.725383750563661 +1.0,1.8874237286900284 3.9061132751393997 +0.0,-0.0823939289302894 1.8958431169469556 +1.0,1.5927855001333566 4.6310125064091965 +0.0,0.3112044157520983 -1.7878471816057036 +1.0,4.34881513764263 3.4693940014863784 +0.0,1.052103622850019 -0.16912252356217902 +1.0,3.167179956507673 2.8792495587252507 +0.0,0.16791453003538387 -0.8546142448164881 +1.0,3.0538805073215953 3.4494667407676842 +0.0,-0.9500475678227512 0.06998146933806365 +1.0,3.8909913837847467 2.6813428719208763 +0.0,-0.09976816220585052 -1.4875944011133129 +1.0,3.1791447205478742 4.424991854067018 +0.0,1.0999643223476656 -1.1200747827607145 +1.0,5.222367041159025 1.2015274537211948 +0.0,-0.2848179798736651 0.401703345435371 +1.0,3.92690552314874 0.5307127426832543 +0.0,-0.6771410319499919 -0.5806616553853885 +1.0,3.611779415106116 3.3322298911093533 +0.0,-1.359189339369671 -0.03773529290863042 +1.0,4.696002594470123 1.4346348756461187 +0.0,-1.0094856636150293 0.19687532044013809 +1.0,3.2169383066148383 3.2307201581236473 +0.0,0.7836015359045666 0.2941037782687062 +1.0,3.7317041306588012 3.7985843457251107 +0.0,-0.3693168101963429 1.4513472421644549 +1.0,4.398703283685875 2.654636797434109 +0.0,0.02043081741683321 0.20805199015337653 +1.0,2.324187503797731 3.8819865944906566 +0.0,1.671377007435211 1.3731572027338659 +1.0,4.534630721644852 1.1543799480085444 +0.0,-0.3253127279932509 -0.8285225286171498 +1.0,3.993821155042294 0.7056403589045206 +0.0,1.194500226045371 0.638917136862092 +1.0,2.72148063695256 3.858678264350294 +0.0,-0.1905653672336637 0.8969404368665279 +1.0,1.9587911397509248 3.937696894952624 +0.0,-1.1358853052995896 1.4443151501322575 +1.0,3.7551091652428026 2.475478572543473 +0.0,-0.9167034706173607 -1.7549316646340103 +1.0,1.4669571532496661 3.2025879996118567 +0.0,-0.9673112226998997 0.13104324478779786 +1.0,5.129589009385082 2.962228456981596 +0.0,-1.038791699676283 0.3394661925580474 +1.0,4.0067362767396055 3.7808733451013863 +0.0,0.4607763000001474 0.3165842402170894 +1.0,3.470781763864157 3.1917117382789906 +0.0,-1.0759836593672722 2.1677955321765423 +1.0,1.8061608083541592 2.1368201192592524 +0.0,0.18913968729195288 -0.6832055159990379 +1.0,2.222086435460701 2.462434683952491 +0.0,1.1697195016246194 -0.6482703204844716 +1.0,0.9469729137532825 2.564223951962673 +0.0,-0.2596612587018774 1.3675954564898984 +1.0,3.3498722540414603 2.8411678301395655 +0.0,0.15549061976540607 -0.8795816620250406 +1.0,3.2166810907529517 3.3909740833940147 +0.0,-0.27777898312342497 1.5708467895548373 +1.0,3.5590852623593734 3.022687446035052 +0.0,0.8854804450462548 -0.1674059547432505 +1.0,5.592380230543062 2.046846128948299 +0.0,-0.38403645419139704 -0.6879614453050698 +1.0,1.2059037878354082 3.1373448113023263 +0.0,-0.9332349591768346 0.3271191223126651 +1.0,2.6941262027196444 2.0016455336591275 +0.0,1.985628476449888 -1.720937514961405 +1.0,1.52678578836386 3.6524268651279113 +0.0,0.14930924959259012 0.3549736192569231 +1.0,2.5081810800507904 4.502494324423253 +0.0,1.3659157029970181 -1.4064298168920828 +1.0,2.8947698041280185 3.871692848909248 +0.0,-0.19002791703482588 0.8099829390725909 +1.0,3.0481549176670555 4.05245395484312 +0.0,-0.014729952199541938 0.43445426055411474 +1.0,3.0874888030440486 3.89317889717026 +0.0,0.9521743475193137 0.16292125350371375 +1.0,3.0564028575123805 3.150394468127784 +0.0,-2.5565867181635724 1.1693524400747453 +1.0,3.963399476624186 2.655863627219969 +0.0,2.0594134768376584 1.4326082874689938 +1.0,3.9415985004601524 4.816989711315565 +0.0,0.4986273362656531 -0.30506819506279537 +1.0,2.7697598834307633 2.0292290332215512 +0.0,-0.4716043983943112 1.4692631198715722 +1.0,3.4127279940145883 3.078218915501194 +0.0,-0.28649487641740207 -0.8009455078808752 +1.0,2.645854233845017 4.028461076417125 +0.0,-1.2333241385253426 -0.2850384355482007 +1.0,2.4938754741404976 1.3466482769013481 +0.0,0.6872021385233428 -0.5159203960430369 +1.0,3.136974388668967 1.69291587793452 +0.0,0.9532239280401443 2.619265789851879 +1.0,2.570576389986536 2.548658346643033 +0.0,-1.030037965987706 0.2814883160676786 +1.0,2.510605023939257 2.3227098241155213 +0.0,2.4171507836629256 1.245606490445435 +1.0,3.5520681299250985 0.7442734445298673 +0.0,1.1940577980770877 1.6319950123919318 +1.0,2.708933998825159 2.118496371335553 +0.0,0.26808250222082186 2.5727974909556437 +1.0,3.221534693193204 3.073316472650363 +0.0,-0.6915734756410544 0.25168141600713434 +1.0,1.839319878312068 1.765565689559382 +0.0,1.708990562782385 1.1196517028520787 +1.0,2.1942131633492643 3.733776318231434 +0.0,1.4884941762679373 -0.5221400677305167 +1.0,2.425026062564176 4.814343944240822 +0.0,-1.3572570451352999 0.04542725800519613 +1.0,3.211869589232063 0.01498355271713292 +0.0,1.6170759581287553 0.7420944718274473 +1.0,1.8096883146020295 1.2063063122336204 +0.0,0.8326608996906895 -0.9760063002065638 +1.0,3.60415819299222 3.905143144181063 +0.0,0.9709971797789466 -1.0644382680658016 +1.0,2.8104103693138778 3.5792951568581017 +0.0,-1.021059644329913 -0.25967578007654707 +1.0,2.4020556940935216 3.8705560506781826 +0.0,-2.704107564850001 -0.14300257306795375 +1.0,3.7681081908063643 2.5433599278958297 +0.0,-0.537043950598385 0.8892208622861 +1.0,3.894301374710518 2.76168141850308 +0.0,-0.8416385593366815 1.3377079857054535 +1.0,1.4560861866861152 1.9464951398785584 +0.0,0.8974462212548237 -0.9027814165394935 +1.0,2.848274393366227 4.089266410865265 +0.0,-1.9874388443190703 -2.0515326123686 +1.0,1.7443330286532606 5.182730816947559 +0.0,1.9345124573698136 0.15482916596109797 +1.0,3.730890742221753 3.4571088485293173 +0.0,-0.7591467032951466 0.7817400181511722 +1.0,1.9612060838774241 1.7874104906670758 +0.0,0.04241602781710118 1.7624663777014242 +1.0,2.983106574446788 2.057794179835603 +0.0,-2.2675373876565272 0.1810247094230928 +1.0,1.8242036739605434 3.2897838599534053 +0.0,0.42135250345103276 0.9201551657148959 +1.0,2.3324158301116547 3.2735600739611406 +0.0,-2.503382611181759 -0.604428052499623 +1.0,2.1068571110070753 1.3987709205712464 +0.0,-0.25006447102137164 1.1597904649452788 +1.0,3.6610503210650105 2.389802330720335 +0.0,0.6655774387829471 -0.7657689612002381 +1.0,3.85820287126228 5.653287382126853 +0.0,0.08244241317513575 0.4755361735454262 +1.0,3.6029514045048234 3.0483730792265247 +0.0,1.0276000901424318 -0.569237094330588 +1.0,2.484863163042475 3.4464671311141046 +0.0,0.24588867824456415 -0.7355421671684942 +1.0,2.8757627634577396 1.3730139621444188 +0.0,0.911649033206053 -1.0562220913143838 +1.0,0.6701966948829261 3.8815519088585195 +0.0,1.0649444423673609 0.5738944212075908 +1.0,3.1272553354329955 5.18450239514651 +0.0,-1.8305691156390467 -1.2811179644895232 +1.0,4.326027257587544 1.9589219729995737 +0.0,-0.2278417247639679 -0.6436775444106994 +1.0,3.9854139754166136 2.8662622299102947 +0.0,-0.33177487577648573 0.7122237484053809 +1.0,2.7631237758865255 2.490470927953921 +0.0,-0.2989203275224733 -0.9063254275476191 +1.0,2.7739570950234254 3.333596743208583 +0.0,-0.12025132003053318 -1.2251715775331837 +1.0,3.9028268386113307 2.580334438085556 +0.0,0.3114518803226873 0.35489645702286177 +1.0,2.8765994073916112 4.251640702192294 +0.0,-3.0895947568085367 -1.0526550179589378 +1.0,3.5182345295490216 2.764855512391279 +0.0,0.5749621254042305 0.7148834016467635 +1.0,4.039448299164001 2.377396087740471 +0.0,1.7077800661629936 -0.23711282974122355 +1.0,2.883211311171089 3.5259606315833287 +0.0,-1.0304518163976537 -0.16271910447066004 +1.0,3.8284470175501504 1.0841759781704199 +0.0,-1.3620621426919217 0.8678141368192274 +1.0,3.831976508070298 2.3592788803510505 +0.0,0.8398199934902235 0.8458121179021545 +1.0,2.166979759191688 4.408250411844058 +0.0,-1.2009412161006234 -0.04486968047943732 +1.0,3.0041897020427517 1.67577082931885 +0.0,-1.0550850035108499 2.6114061208535673 +1.0,1.46399823823424 3.6863318429400627 +0.0,-0.439942118867861 0.8107733517611471 +1.0,2.799907981207793 3.1021389011201244 +0.0,0.40512996190803663 -0.2720769110918539 +1.0,2.936414720731187 2.6121553148876706 +0.0,0.7864503163458285 0.879685137879171 +1.0,3.497848931993103 3.93953696354328 +0.0,1.0898800025299487 -0.3780987477521812 +1.0,3.0737866861658834 3.8281246288654067 +0.0,1.0100369320198321 -0.36412797089680377 +1.0,4.977156552398557 1.9361263628969327 +0.0,1.1948682006514484 -1.0421380659408503 +1.0,2.3707352395183743 3.319087891488442 +0.0,0.14662871945444525 -1.125277513770441 +1.0,4.18636170602371 5.079790109963499 +0.0,0.5213830491310841 2.5489667538554355 +1.0,3.456121838657517 2.9777488007628823 +0.0,1.3942157902546204 -0.7392170745991694 +1.0,4.027857416272539 2.5520251242493615 +0.0,0.6677437543225546 -0.7054702957392922 +1.0,2.419993627501343 3.147115729790262 +0.0,-1.1891285195785104 0.7121837556662985 +1.0,2.6768950566988114 2.746092902448666 +0.0,-0.5581632736462642 -0.8475377022167101 +1.0,2.2877649074222144 3.360822129377224 +0.0,0.12427410923130733 -0.029877611579596446 +1.0,2.1363649823278976 2.040672619624904 +0.0,0.164296403698455 -0.7853340225962958 +1.0,2.2867454265483063 2.920796736914219 +0.0,0.030938689766481568 0.02840531713718885 +1.0,4.935402862397514 4.984097800264938 +0.0,-0.49323021214001667 -0.009344009957387383 +1.0,2.2590589178865788 2.784700488476081 +0.0,-1.7996451721642797 -0.08927843209025701 +1.0,2.7189425454136047 3.366984002518318 +0.0,-0.4732503966611213 2.41667617281343 +1.0,1.914172722581019 2.723688261246487 +0.0,0.6854209215843875 -0.6321377274037409 +1.0,4.7025333481932705 2.6561807763401646 +0.0,0.016511529980536163 -0.4064291762993186 +1.0,1.3841179371371182 3.367159685928979 +0.0,-0.525665902025766 0.3189849885462113 +1.0,2.1237941386456276 3.4141040859263914 +0.0,-1.3977733609952327 1.6180332199555512 +1.0,3.3282228318571496 2.9879449742002184 +0.0,-1.3911999737510374 -0.47876736354905697 +1.0,3.071461319022103 3.902142645231827 +0.0,-1.4616870328596612 0.4234223737141411 +1.0,3.3069543201402576 1.3522887907099401 +0.0,0.1771175002160632 0.7092577154896049 +1.0,2.561517669553921 3.2663130772229185 +0.0,0.8635080818806004 1.7578935533355913 +1.0,3.3054989034355793 3.4205399612822633 +0.0,-0.5525474134214131 -0.008874526853035592 +1.0,5.024607965706471 3.377256085775693 +0.0,0.6499316691799448 0.7636813929956143 +1.0,1.7211648540475015 3.7290596058136307 +0.0,-0.4312096678787339 0.4723353140241522 +1.0,1.6269397815780402 1.9613109767814954 +0.0,0.06589250830042476 0.5659627954925366 +1.0,1.4141705667382305 2.9411215895612255 +0.0,-0.30655047441372724 1.134312621267185 +1.0,4.079371134159225 3.7127217011979767 +0.0,-0.11148410319718746 1.504423362990177 +1.0,3.21908765035085 1.5284527951297098 +0.0,0.38879874604519066 -0.7718569898512835 +1.0,3.0387686435299197 1.9571679686339727 +0.0,0.0432538958325193 -0.609046739618082 +1.0,3.858513576900389 2.3343789318227595 +0.0,-1.594606569379673 2.0291869081775498 +1.0,4.418575803606943 3.634284954659144 +0.0,-1.5657043498774568 0.48528442006547645 +1.0,3.7474369990653518 2.417108621170513 +0.0,-0.4087178618516316 -0.5585629524971241 +1.0,2.8830052178069345 2.714807180476644 +0.0,1.0200529614238536 1.633454495011907 +1.0,2.161101444560085 2.722233198993495 +0.0,0.8905571055499505 0.3531260808046299 +1.0,1.5770402091220281 2.5197577954902615 +0.0,0.19603489193696402 0.4391781215510938 +1.0,3.285302297900197 2.5981032583297274 +0.0,-1.7728311957227578 2.226646036588897 +1.0,2.212402423781055 2.994783519362575 +0.0,-0.26351331835428804 0.6197161896115081 +1.0,2.5101464936050144 2.747453537535198 +0.0,1.083443472210967 -0.7471502465676395 +1.0,2.618022142084275 3.201094589808021 +0.0,-0.10243507468644107 -1.5307780048431203 +1.0,2.0479014235932986 2.7174445598757764 +0.0,-0.2530316183327909 1.5105959457792464 +1.0,2.616239369128394 3.1011058356715644 +0.0,2.0703487677159997 -1.23039689097027 +1.0,2.00559575849234 3.088170264353322 +0.0,0.751453701775929 -0.34079600956200146 +1.0,2.6436129383324625 0.6934715851263205 +0.0,0.4735774669250165 0.24981500600111478 +1.0,3.614102521076285 3.297655445774221 +0.0,-0.8397190394129946 2.0791729859494583 +1.0,2.5800847823336372 2.312770726398467 +0.0,0.9528690775719402 -4.054641847252764 +1.0,1.6631425491523402 4.465488566725185 +0.0,-0.40442215938144854 2.1662912065078923 +1.0,3.2025444402071472 0.954639816329502 +0.0,0.8484611241529962 -0.6531501762867838 +1.0,2.907155165379039 4.494838051538261 +0.0,1.1473298350419248 -0.7604213061923158 +1.0,4.406872541176625 2.616395889868952 +0.0,-1.0643453307576694 0.32269083514118757 +1.0,3.4229771635424653 5.404174358063928 +0.0,0.8223012341648268 -2.0705983787489455 +1.0,0.6519219290294926 3.317297519573949 +0.0,0.6661739745821234 0.21368601256080724 +1.0,2.8092516816651187 2.9407143882873363 +0.0,-2.0396349059310626 0.6660958962860263 +1.0,1.621401319049101 2.120514741629026 +0.0,-0.6673242389540511 -1.033336539766657 +1.0,2.4729967381312257 2.0622671692969314 +0.0,0.318696287733599 0.7696143248064906 +1.0,-0.3310542190127661 2.503572170101248 +0.0,-0.024545405442632163 1.2826535279165514 +1.0,2.08361065329982 1.7709137020843035 +0.0,-0.03325908838419148 2.127731976717063 +1.0,0.8920712229737089 2.267227052639782 +0.0,2.4226620796703706 -1.5422597801969735 +1.0,2.6125707261695665 4.136941962252239 +0.0,0.710000430684373 -0.2365544035810329 +1.0,3.587983407259662 2.371118916918134 +0.0,1.548716105657387 2.6039797648647527 +1.0,2.288647833469394 2.8514285941696564 +0.0,0.5407956769257948 -1.4250712589214616 +1.0,3.9999271279969157 4.647262641336589 +0.0,0.46916438504363506 -0.16114805677977867 +1.0,3.9351714928555133 3.017851089635014 +0.0,-0.24683125971847 0.8686956304798523 +1.0,2.445900548419883 2.601998949302925 +0.0,0.9708272515136681 0.9540365110832763 +1.0,2.0889493306284472 1.670700190658552 +0.0,0.7573519355244429 -0.6731075400854291 +1.0,2.9938559890272676 0.5796453404844417 +0.0,-0.42350233780111274 0.1072223004754211 +1.0,3.22502989165533 3.2744724666391045 +0.0,-0.051171179793716125 0.035749085667007977 +1.0,4.256076524642883 3.956646576238979 +0.0,0.44715068158575316 -0.10904823199444005 +1.0,3.754239074295241 2.4862504435534283 +0.0,-0.12025734941101636 0.6682754649328633 +1.0,2.9673795614648815 3.6207880514009263 +0.0,-2.250093626462795 -0.49148713538228506 +1.0,1.7335315087131171 4.234455598757855 +0.0,-0.5145677322324603 -1.8872464244504652 +1.0,3.1524408905920547 2.534903833671654 +0.0,1.4188237424906527 -1.987300018397619 +1.0,3.025903676999244 2.1652631630581847 +0.0,0.5008343534015861 0.28011601768758965 +1.0,2.0039218613662197 2.3639397631018015 +0.0,1.342528231824729 1.0036076495884643 +1.0,3.3281244751369985 2.4251038991267277 +0.0,-0.38845861664115766 -1.5147629282596704 +1.0,2.613448357242925 4.463712912575443 +0.0,-0.19439583983218703 0.676381234314577 +1.0,1.0400516553104269 2.3981508685333424 +0.0,0.9469554018478826 -0.08144910777086176 +1.0,3.179705969662961 3.768848690124549 +0.0,0.39855441813668835 -1.6301847736954416 +1.0,2.1915941615815226 2.7947789889097763 +0.0,1.6023287643577222 0.05432794979410767 +1.0,1.5758610206949497 3.8709473262823777 +0.0,-1.3109119301269387 -0.8645189055395048 +1.0,3.715865055565244 1.9360512196442488 +0.0,-0.2073998491467907 -1.178882579876182 +1.0,2.565062666629786 2.3121370465462494 +0.0,-0.41397768670851737 -0.6674761320605563 +1.0,2.941938460212705 3.537877403937825 +0.0,0.5954231185191001 1.6839554319972647 +1.0,4.591360208911688 1.4381368838271187 +0.0,-1.3221878199013057 0.786799353955043 +1.0,0.6498018470693379 2.2143413646510095 +0.0,0.5346452265922554 0.45599002729248733 +1.0,2.668100742914233 2.679883986650412 +0.0,-0.22428284967184606 -1.0003823373608314 +1.0,4.233871998643562 3.3423521548333897 +0.0,0.7800144346305873 1.6512542456242612 +1.0,3.3192955924982677 4.664828345688715 +0.0,-0.9059493298933676 -0.42207747354389447 +1.0,3.1776956110847916 1.1393123509452483 +0.0,-0.5246202787832872 1.0246845701853746 +1.0,4.732113325540828 1.29018271893586 +0.0,0.9863596225434407 0.7506968948666005 +1.0,2.911409852038849 2.626474556246977 +0.0,0.8545346747310709 -2.1711133879380955 +1.0,2.476689592134109 4.03136160709651 +0.0,0.43108249592457043 0.4589971218864913 +1.0,3.2333287857145825 2.188137362144206 +0.0,1.4405649581445525 0.4131214094941824 +1.0,2.0631468420251093 3.807898318807702 +0.0,0.43964401099781425 0.6669437158150616 +1.0,2.165843657939062 4.109647016182597 +0.0,-0.9735452695016392 -0.6172105570335473 +1.0,3.169794653766589 3.2721053734106 +0.0,1.3129166037688875 -1.2040138532590103 +1.0,2.211361701514339 1.025981622029549 +0.0,0.3653350359702278 0.5229315457444437 +1.0,3.372206428302252 4.163685355869495 +0.0,-0.8690030167652726 0.3226849491596335 +1.0,4.188509026227427 2.1137749377457076 +0.0,2.2174789916979933 0.8249932442083762 +1.0,3.9224824525785706 2.9436443006575925 +0.0,0.1370905200148926 -0.043320354739616776 +1.0,3.1118662077850807 1.4983207834379917 +0.0,-0.5304073850344787 -0.4219778391981189 +1.0,1.2153552376808336 3.4749521622043438 +0.0,-2.545970043914331 -0.5480647959096547 +1.0,1.8097968872175412 4.733523163055134 +0.0,-0.5599306916727819 0.4648015112295201 +1.0,3.0242901796172204 4.354893518146392 +0.0,-0.49175893973189483 1.8635231981223406 +1.0,3.923889822736733 4.199324033436554 +0.0,0.32931083529824645 -1.2038529291812745 +1.0,2.8430570026355904 3.2581768028655214 +0.0,0.08015643729775149 -0.5281238499521005 +1.0,1.0251176552841985 2.452443183841665 +0.0,-1.4000614002792062 -0.4723026702712555 +1.0,4.642753244692533 3.5777684251625153 +0.0,-0.9732069449126244 -0.7507666182081589 +1.0,2.284811103731081 2.6226837934175817 +0.0,1.4938320459354653 1.2271703303402608 +1.0,2.5217907633717935 1.9804499278889345 +0.0,0.9177851256816916 -1.196945923903535 +1.0,2.650515007788954 0.9818159554114416 +0.0,-0.4172435945582116 0.11930551874205601 +1.0,1.8203127944592765 3.3069324017397594 +0.0,0.08195935202288789 -0.2585763476071969 +1.0,2.14910426585678 4.146147361847687 +0.0,1.578290774885182 0.16149960053586573 +1.0,1.2607405323635168 2.940350340912184 +0.0,1.6722138822230346 -0.5454073192477626 +1.0,0.3769561517619793 4.029314828130509 +0.0,-0.012008811772440746 0.2577932550827986 +1.0,2.330909580388283 3.1650439747088024 +0.0,-1.4224384024201595 -0.6369918128076046 +1.0,3.451178380794735 2.7553545272536746 +0.0,-0.7913135079702314 -0.012217405089490006 +1.0,3.7918310740082424 3.3927876820084033 +0.0,0.41016650792928255 0.3521369094279198 +1.0,2.380867149491576 3.7533007228820754 +0.0,-0.2787273586680994 1.3553543015884186 +1.0,2.8933236071325226 1.7975563396445144 +0.0,-0.4868680345968448 0.058461169788172784 +1.0,3.484434144626577 3.5622013162506683 +0.0,1.171904838026115 0.1162839888503951 +1.0,1.8132727587691455 2.238018140780368 +0.0,0.8114997821213137 -1.712768034302675 +1.0,2.977061410695451 2.802894970831404 +0.0,1.7141760742336318 0.5672102391229309 +1.0,3.2929421353515185 3.3754831695793945 +0.0,-2.280170614413754 -0.4912881923146271 +1.0,4.182771547422101 3.5331418354105812 +0.0,-0.2544453921577854 0.4682744998445509 +1.0,1.9236524545763007 2.628837510538455 +0.0,0.6645491524745186 -2.398604366119661 +1.0,3.50840713613987 3.7182332137428955 +0.0,-1.4532823239751684 -0.9916580822162051 +1.0,2.769613688635247 4.72661442603805 +0.0,-1.090104082054257 0.486265921887567 +1.0,3.4900626627065003 3.03025323652533 +0.0,1.4518716691137106 -0.10218738652959546 +1.0,2.745034544461333 4.366809709694589 +0.0,-0.17197050309086373 0.13673125942508174 +1.0,2.4934379443680985 2.954734256628178 +0.0,0.14078971520128297 -0.5401300324197861 +1.0,3.640563349517043 5.163454382169049 +0.0,1.0264020194022627 -0.8738489740165843 +1.0,3.791458514669831 2.2038333093620834 +0.0,-3.075231830613813 2.04054404065675 +1.0,4.647422323558612 3.5220753128741427 +0.0,-0.6423734479152313 0.5403500050100541 +1.0,1.5985339514690007 2.73447434771563 +0.0,-0.04474684215568748 -0.21477212224970194 +1.0,2.6701891009654792 3.9776885659794505 +0.0,-0.4714276238216119 1.4235807729101415 +1.0,3.5551789183755806 2.7057825768035104 +0.0,1.108254774651522 0.8596053056731966 +1.0,3.0623366138774983 2.718494058918926 +0.0,-1.375827910513567 0.011994162356159788 +1.0,3.841407434840553 2.8434319292302304 +0.0,-0.7149712282755271 0.1811986378283469 +1.0,5.155524316715826 2.1468464150279747 +0.0,-0.06822014690491127 -0.15801546435311806 +1.0,3.4838423066641173 4.211572262022802 +0.0,1.455177312877137 -0.9388697017811595 +1.0,3.917344840727481 3.569507254920478 +0.0,-2.080636526173827 -1.2489913979804321 +1.0,4.904327940183608 3.4289745068714295 +0.0,-1.4744723958060084 0.2930577753686633 +1.0,2.810346752831796 2.4062885063635333 +0.0,-0.17365054648101302 -2.26263747840141 +1.0,4.077713960215311 3.841309768575811 +0.0,1.581178479362914 -0.9672846912018417 +1.0,4.516244757634386 2.9078781629204054 +0.0,-1.5890391289381882 -0.4092245513024253 +1.0,3.359480708344044 3.7375262649030123 +0.0,1.5675385032786122 0.9010632060589036 +1.0,3.8564874267647644 3.060660915266198 +0.0,-0.2482500870678099 0.29655946916337894 +1.0,3.1672692968701397 1.1973226392521306 +0.0,-1.4471523637168304 0.5370395414503478 +1.0,4.814859889188941 2.229750617440331 +0.0,0.2812295731325761 0.6044036116090106 +1.0,2.4884527354338903 1.4171627784171204 +0.0,1.173099753717184 0.7948729712563257 +1.0,1.5092479631180256 4.1412277875509105 +0.0,-1.1453508695714685 -0.15567849492271865 +1.0,1.9397046305500465 3.430755367623314 +0.0,-1.6689604208958047 -1.161942047896626 +1.0,4.287905082572467 2.643797664646416 +0.0,0.5691715436318573 -0.6013793142266736 +1.0,2.622904412483301 1.769830678112635 +0.0,-1.0627706066421603 -1.2962746926911266 +1.0,2.5818494635089886 2.9547836545958663 +0.0,-1.555832778500785 0.6050365213516793 +1.0,0.6877755924513469 3.0627330470806617 +0.0,-0.6945984937358738 -0.5355659085722678 +1.0,3.631758943383 2.6990914911890194 +0.0,-0.10204034384758799 1.2650405538373874 +1.0,2.8618200471403488 2.7676923144816237 +0.0,-1.2337428464512885 -0.7151041760567872 +1.0,3.5209869997316807 3.280763138579491 +0.0,0.3700095159793621 -0.8614396246939711 +1.0,2.698616090611572 3.2205340189872795 +0.0,-0.8069663812258417 -0.07956402748767083 +1.0,2.929873320056276 4.030067053746698 +0.0,-1.2316919288622938 1.245687935224532 +1.0,2.9285679560367055 2.9682906465530783 +0.0,-0.3965578686363537 1.1748126835359254 +1.0,4.002714110052464 4.370338584188975 +0.0,-0.6084107635744659 -0.6092872315132073 +1.0,3.293912876563504 3.5843332356258464 +0.0,-0.8145032742370918 1.4050967895930515 +1.0,1.991600071099763 2.343264260750465 +0.0,-0.9433799779882722 1.5943129187456013 +1.0,2.369037146473894 1.9827898318071764 +0.0,-0.26885731570182714 0.47421918725401946 +1.0,3.263006333756187 3.0441051541001443 +0.0,0.21785408377528742 0.5754303556190559 +1.0,2.941128899266118 1.240818619804987 +0.0,0.736142634408259 -1.3173589352849961 +1.0,3.2027184783050644 2.9218716893221766 +0.0,1.9216539101612737 -2.2400666381338694 +1.0,2.4823406743823426 3.429705681271458 +0.0,0.0666674809216063 -0.976496437708073 +1.0,3.206108328915537 2.0828009180110976 +0.0,-0.11582094814525531 2.5093876016868366 +1.0,2.5373176496966328 2.32926952602907 +0.0,-0.9237765727032562 0.9342845305943139 +1.0,2.5300867778672123 3.2754703213122753 +0.0,0.13837351460348038 0.2533025702882705 +1.0,4.556185356940701 0.7629684714626066 +0.0,-1.8251759895063635 0.6966019254550819 +1.0,4.905392053322123 4.111245902434462 +0.0,0.09886105139472441 1.4093224263552915 +1.0,2.0484713074013223 4.874632770975326 +0.0,-0.040609033066195156 -1.3446008307073973 +1.0,3.678642687565624 4.156505531118834 +0.0,0.052003196801406706 1.2239229001362555 +1.0,3.4376496474012876 2.417529764306501 +0.0,-0.09054032070414311 -1.7571173217955876 +1.0,3.230032966809188 3.5965216835420546 +0.0,0.9100014718072797 0.5615698517199065 +1.0,3.938728443662248 3.2945250621813273 +0.0,-0.9205165004286314 -0.01425448590777016 +1.0,1.907285344344031 3.8629943281683987 +0.0,-0.8160057252300347 -0.2757475590440447 +1.0,2.3076630082503926 3.2283118851645476 +0.0,1.3000520665928303 0.581203895654615 +1.0,3.8425274250736887 3.6133028383400414 +0.0,0.13694776598217193 -1.1659103408047182 +1.0,2.688548985689179 1.5486856086329917 +0.0,-0.14378057635986438 -1.4649914115754739 +1.0,3.923705106138171 3.8281415874634783 +0.0,1.3334544187579878 -0.048721556115349604 +1.0,3.320777445436592 2.947489296620178 +0.0,-0.36251547004650103 -0.2886015741883188 +1.0,3.2163584307843567 2.9285953038088373 +0.0,0.5437339741631225 -0.23459273264636704 +1.0,2.820666118654177 4.0305429519659395 +0.0,0.04808393980018175 0.42285718084497675 +1.0,1.4686721107589078 2.6605885841423067 +0.0,1.1873828480862414 0.5487600196906772 +1.0,3.425690422789916 4.252827757634791 +0.0,-0.7323210179394448 -0.9818194354330615 +1.0,3.018263609974841 2.914037267945018 +0.0,1.005159548514262 -0.5055899932767433 +1.0,4.566046579419102 5.545663797862058 +0.0,-0.7129346827436536 2.2938920919917742 +1.0,2.869336979055624 2.5688122980246684 +0.0,1.5201806096451054 -0.7414084378784415 +1.0,1.71558426191034 2.4576286538624794 +0.0,0.8090326808020629 0.26208059965589425 +1.0,3.0163716479573077 2.4747608384001056 +0.0,0.47627288733283857 1.3085076289292734 +1.0,3.3891272567835684 3.20832981462489 +0.0,1.0488767400026389 1.2318533170755142 +1.0,3.3428160616141853 2.5497426855885075 +0.0,-0.6411040361810151 -0.4290410178863531 +1.0,2.219119637941564 2.6621113083439254 +0.0,1.5621125506487947 0.7273124535333745 +1.0,3.1459765929197636 1.3663869759433418 +0.0,-0.05263982623034547 0.43675636434345644 +1.0,1.890191705836878 3.435071392429276 +0.0,0.28718983621307775 -2.438042507707637 +1.0,5.717207001359904 2.2303522388797035 +0.0,0.17636841934036573 -0.2202348356695646 +1.0,2.7426941364254294 3.9506423829670734 +0.0,-1.118995077703066 0.6062681312772151 +1.0,4.510963440028501 2.4497214672006575 +0.0,0.07601426739661686 1.4712413920907517 +1.0,2.472822799411239 4.045939967967948 +0.0,-2.2061186560242603 0.32560701091997957 +1.0,3.250675248798315 3.268273446922124 +0.0,-0.024542349115316425 1.5505593308513355 +1.0,2.5654508852779654 2.9476923150082874 +0.0,0.8070230851041806 1.0614288963806608 +1.0,4.0121013342203655 1.7608333223695753 +0.0,-0.6895596222836047 0.035498410809669464 +1.0,1.697905057706837 4.053746875797327 +0.0,-0.3311042917990167 -0.09180266122060314 +1.0,3.720796880080382 4.467214289132983 +0.0,-0.318673057944378 -3.1474317710285202 +1.0,4.809204233917482 4.55250051737848 +0.0,0.596445093094233 0.41780789823963405 +1.0,4.432965399675368 3.4638105151117617 +0.0,-0.10285141484897965 1.747950423830727 +1.0,2.1513849154027014 3.9020766404442933 +0.0,1.5988780419195843 -0.08753929889987294 +1.0,0.9867334105272594 3.017081919852008 +0.0,-1.4952194834476749 1.0187701527429442 +1.0,2.2468599817570376 2.5883807516977395 +0.0,-1.804930212071194 0.3519094744696904 +1.0,4.1524048686549975 2.39387437993355 +0.0,0.7077190974093445 0.5703893640810606 +1.0,3.551726989450847 2.4786821848615985 +0.0,1.866022101379231 0.23733176192158173 +1.0,2.636453843734601 3.2607059005922467 +0.0,1.0052825898444602 0.5988275134415102 +1.0,2.643754787324359 3.72363185525656 +0.0,-0.9925822461102075 0.060644514219670244 +1.0,3.8994350969658136 1.9246001662480055 +0.0,0.6513177047637154 0.04450296971216735 +1.0,2.4564101844841106 3.6785165656991596 +0.0,0.2606556093620563 -0.6172755504020078 +1.0,2.4170362032345674 0.8639272362396189 +0.0,-0.6416537078444019 1.8622433251026849 +1.0,2.0247632881021267 2.538336421666863 +0.0,-1.0177991501405648 -0.8522549981552515 +1.0,3.3426117902650185 3.1635532244875586 +0.0,-0.08963512689480763 1.4555128614393191 +1.0,3.7470117779591092 3.414476280017385 +0.0,0.7721815837750134 -0.17297061945116646 +1.0,3.823597567639877 4.2427688079492665 +0.0,-0.6905817293226868 0.5838402640342898 +1.0,3.005258204213709 2.7252310853631125 +0.0,0.963732273262942 -1.3950688358262504 +1.0,3.2803836447761934 3.448945851174787 +0.0,-0.11576488451784747 1.8796627145034757 +1.0,3.905782244273501 3.3853014175990412 +0.0,0.3786078767939069 0.4054987293824608 +1.0,4.251338642737948 3.2212804055347375 +0.0,1.785664685579919 -0.4528337660796719 +1.0,0.9522164714530392 4.648272724469027 +0.0,2.06805484281029 0.3211833348167774 +1.0,3.2063266406360875 3.20907719820361 +0.0,-0.18542396323311192 -0.4721814985954186 +1.0,1.2468417100913183 2.988063666542869 +0.0,-0.9089767150726245 0.049627884005341995 +1.0,3.570670591235201 1.812766580123238 +0.0,1.9973417232460495 -0.17709723581574177 +1.0,2.810527831677345 2.0292239826226717 +0.0,0.06390562956663569 0.9110683296487658 +1.0,4.449308253046676 2.5895593413305997 +0.0,-0.18596846882351442 1.2495641818989083 +1.0,2.1189215966743986 3.7928094437779283 -- cgit v1.2.3 From fd137bd7c6b924dfb818b8ff45b9c6bec0d47700 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 12 Jun 2013 16:30:04 -0700 Subject: Address Reynold's comments. Also use a builder pattern to construct the regression classes. --- ml/src/main/scala/spark/ml/Gradient.scala | 19 +++-- ml/src/main/scala/spark/ml/GradientDescent.scala | 21 +++++- .../main/scala/spark/ml/LogisticRegression.scala | 80 +++++++++++++++++---- ml/src/main/scala/spark/ml/MLUtils.scala | 22 +++--- ml/src/main/scala/spark/ml/Regression.scala | 82 +++++++++++++--------- ml/src/main/scala/spark/ml/RidgeRegression.scala | 77 +++++++++++++++----- ml/src/main/scala/spark/ml/Updater.scala | 15 +++- 7 files changed, 233 insertions(+), 83 deletions(-) diff --git a/ml/src/main/scala/spark/ml/Gradient.scala b/ml/src/main/scala/spark/ml/Gradient.scala index a7e8327133..fe36187000 100644 --- a/ml/src/main/scala/spark/ml/Gradient.scala +++ b/ml/src/main/scala/spark/ml/Gradient.scala @@ -3,6 +3,13 @@ package spark.ml import org.jblas.DoubleMatrix abstract class Gradient extends Serializable { + /** + * Compute the gradient for a given row of data. + * + * @param data - One row of data. Row matrix of size 1xn where n is the number of features. + * @param label - Label for this data item. + * @param weights - Column matrix containing weights for every feature. + */ def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): (DoubleMatrix, Double) } @@ -14,11 +21,13 @@ class LogisticGradient extends Gradient { 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 - } + val loss = + if (margin > 0) { + math.log(1 + math.exp(0 - margin)) + } else { + math.log(1 + math.exp(margin)) - margin + } + (gradient, loss) } } diff --git a/ml/src/main/scala/spark/ml/GradientDescent.scala b/ml/src/main/scala/spark/ml/GradientDescent.scala index 80d4c44a13..a03285bd53 100644 --- a/ml/src/main/scala/spark/ml/GradientDescent.scala +++ b/ml/src/main/scala/spark/ml/GradientDescent.scala @@ -10,6 +10,20 @@ import scala.collection.mutable.ArrayBuffer object GradientDescent { + /** + * 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 numIters - number of iterations that SGD should be run. + * @param miniBatchFraction - fraction of the input data set that should be used for + * one iteration of SGD. Default value 1.0. + * + * @return weights - Column matrix containing weights for every feature. + * @return lossHistory - Array containing the loss computed for every iteration. + */ def runMiniBatchSGD( data: RDD[(Double, Array[Double])], gradient: Gradient, @@ -18,22 +32,23 @@ object GradientDescent { numIters: Int, miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { - val lossHistory = new ArrayBuffer[Double] + val lossHistory = new ArrayBuffer[Double](numIters) val nfeatures: Int = data.take(1)(0)._2.length val nexamples: Long = data.count() val miniBatchSize = nexamples * miniBatchFraction + // Initialize weights as a column matrix var weights = DoubleMatrix.ones(nfeatures) var reg_val = 0.0 for (i <- 1 to numIters) { - val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42).map { + val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42+i).map { case (y, features) => val featuresRow = new DoubleMatrix(features.length, 1, features:_*) val (grad, loss) = gradient.compute(featuresRow, y, weights) (grad, loss) - }.reduce((a, b) => (a._1.add(b._1), a._2 + b._2)) + }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) lossHistory.append(lossSum / miniBatchSize + reg_val) val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i) diff --git a/ml/src/main/scala/spark/ml/LogisticRegression.scala b/ml/src/main/scala/spark/ml/LogisticRegression.scala index 5647bf5a84..e8e4f96dd3 100644 --- a/ml/src/main/scala/spark/ml/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/LogisticRegression.scala @@ -1,7 +1,6 @@ package spark.ml import spark.{Logging, RDD, SparkContext} -import spark.SparkContext._ import org.jblas.DoubleMatrix @@ -14,9 +13,9 @@ class LogisticRegressionModel( val losses: Array[Double]) extends RegressionModel(weights, intercept) { override def predict(test_data: spark.RDD[Array[Double]]) = { - test_data.map { x => - val margin = (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept - 1.0/(1.0 + math.exp(margin * -1)) + test_data.map { x => + val margin = new DoubleMatrix(1, x.length, x:_*).mmul(this.weights).get(0) + this.intercept + 1.0/ (1.0 + math.exp(margin * -1)) } } } @@ -25,9 +24,9 @@ class LogisticRegressionData(data: RDD[(Double, Array[Double])]) extends Regress override def normalizeData() = { // Shift only the features for LogisticRegression data.map { case(y, features) => - val featuresNormalized = (0 until nfeatures).map( - column => (features(column) - xColMean(column)) / xColSd(column) - ).toArray + val featuresNormalized = Array.tabulate(nfeatures) { column => + (features(column) - xColMean(column)) / xColSd(column) + } (y, featuresNormalized) } } @@ -44,25 +43,41 @@ class LogisticRegressionData(data: RDD[(Double, Array[Double])]) extends Regress } } -object LogisticRegression extends Logging { - val STEP_SIZE = 1.0 - val MINI_BATCH_FRACTION = 1.0 +class LogisticRegression(stepSize: Double, miniBatchFraction: Double, numIters: Int) + extends Regression with Logging { - def train(input: RDD[(Double, Array[Double])], numIters: Int) = { + override def train(input: RDD[(Double, Array[Double])]): RegressionModel = { input.cache() val lrData = new LogisticRegressionData(input) val data = lrData.normalizeData() val (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new LogisticGradient(), new SimpleUpdater(), STEP_SIZE, numIters, MINI_BATCH_FRACTION) - + data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, numIters) + val computedModel = new LogisticRegressionModel(weights, 0, losses) val model = lrData.scaleModel(computedModel) logInfo("Final model weights " + model.weights) logInfo("Final model intercept " + model.intercept) - logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(",")) + logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) model } +} + +/** + * Helper classes to build a LogisticRegression object. + */ +object LogisticRegression { + + /** + * Build a logistic regression object with default arguments: + * + * @param stepSize as 1.0 + * @param miniBatchFraction as 1.0 + * @param numIters as 100 + */ + def builder() = { + new LogisticRegressionBuilder(1.0, 1.0, 100) + } def main(args: Array[String]) { if (args.length != 3) { @@ -71,7 +86,42 @@ object LogisticRegression extends Logging { } val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadData(sc, args(1)) - val model = train(data, args(2).toInt) + val lr = LogisticRegression.builder() + .setStepSize(2.0) + .setNumIterations(args(2).toInt) + .build() + val model = lr.train(data) sc.stop() } } + +class LogisticRegressionBuilder(stepSize: Double, miniBatchFraction: Double, numIters: Int) { + + /** + * Set the step size per-iteration of SGD. Default 1.0. + */ + def setStepSize(step: Double) = { + new LogisticRegressionBuilder(step, this.miniBatchFraction, this.numIters) + } + + /** + * Set fraction of data to be used for each SGD iteration. Default 1.0. + */ + def setMiniBatchFraction(fraction: Double) = { + new LogisticRegressionBuilder(this.stepSize, fraction, this.numIters) + } + + /** + * Set the number of iterations for SGD. Default 100. + */ + def setNumIterations(iters: Int) = { + new LogisticRegressionBuilder(this.stepSize, this.miniBatchFraction, iters) + } + + /** + * Build a Logistic regression object. + */ + def build() = { + new LogisticRegression(stepSize, miniBatchFraction, numIters) + } +} diff --git a/ml/src/main/scala/spark/ml/MLUtils.scala b/ml/src/main/scala/spark/ml/MLUtils.scala index eab5db61bd..d62198f426 100644 --- a/ml/src/main/scala/spark/ml/MLUtils.scala +++ b/ml/src/main/scala/spark/ml/MLUtils.scala @@ -2,21 +2,27 @@ package spark.ml import spark.{RDD, SparkContext} +/** + * Helper methods to load and save data + * Data format: + * , ... + * where , are feature values in Double and is the corresponding label as Double. + */ object MLUtils { - // Helper methods to load and save data - // Data format: - // , ... - // where , are feature values in Double and - // is the corresponding label as Double - def loadData(sc: SparkContext, dir: String) = { - val data = sc.textFile(dir).map{ line => + /** + * @param sc SparkContext + * @param dir Directory to the input data files. + * @return An RDD of tuples. For each tuple, the first element is the label, and the second + * element represents the feature values (an array of Double). + */ + def loadData(sc: SparkContext, dir: String): RDD[(Double, Array[Double])] = { + sc.textFile(dir).map { line => val parts = line.split(",") val label = parts(0).toDouble val features = parts(1).trim().split(" ").map(_.toDouble) (label, features) } - data } def saveData(data: RDD[(Double, Array[Double])], dir: String) { diff --git a/ml/src/main/scala/spark/ml/Regression.scala b/ml/src/main/scala/spark/ml/Regression.scala index e19d58396b..26a3a05545 100644 --- a/ml/src/main/scala/spark/ml/Regression.scala +++ b/ml/src/main/scala/spark/ml/Regression.scala @@ -7,47 +7,61 @@ import spark.SparkContext._ import org.jblas.DoubleMatrix -abstract class RegressionModel( - val weights: DoubleMatrix, - val intercept: Double) { - +abstract class RegressionModel(val weights: DoubleMatrix, val intercept: Double) { def predict(test_data: RDD[Array[Double]]): RDD[Double] } abstract class RegressionData(val data: RDD[(Double, Array[Double])]) extends Serializable { - var yMean: Double = 0.0 - var xColMean: Array[Double] = null - var xColSd: Array[Double] = null - var nfeatures: Int = 0 - var nexamples: Long = 0 + val nfeatures: Int = data.take(1)(0)._2.length + val nexamples: Long = data.count() + + val yMean: Double = data.map { case (y, features) => y }.reduce(_ + _) / nexamples + + // NOTE: We shuffle X by column here to compute column sum and sum of squares. + private val xColSumSq: RDD[(Int, (Double, Double))] = data.flatMap { case(y, features) => + val nCols = features.length + // Traverse over every column and emit (col, value, value^2) + Iterator.tabulate(nCols) { i => + (i, (features(i), features(i)*features(i))) + } + }.reduceByKey { case(x1, x2) => + (x1._1 + x2._1, x1._2 + x2._2) + } + private val xColSumsMap = xColSumSq.collectAsMap() - // This will populate yMean, xColMean and xColSd - calculateStats() + // Compute mean and unbiased variance using column sums + val xColMean: Array[Double] = Array.tabulate(nfeatures) { x => + xColSumsMap(x)._1 / nexamples + } + val xColSd: Array[Double] = Array.tabulate(nfeatures) { x => + val v = (xColSumsMap(x)._2 - (math.pow(xColSumsMap(x)._1, 2) / nexamples)) / (nexamples) + math.sqrt(v) + } + /** + * Normalize the provided input data. This function is typically called before + * training a classifier on the input dataset and should be used to center of scale the data + * appropriately. + * + * @return RDD containing the normalized data + */ def normalizeData(): RDD[(Double, Array[Double])] + + /** + * Scale the trained regression model. This function is usually called after training + * to adjust the model based on the normalization performed before. + * + * @return Regression model that can be used for prediction + */ def scaleModel(model: RegressionModel): RegressionModel +} - def calculateStats() { - this.nexamples = data.count() - this.nfeatures = data.take(1)(0)._2.length - - this.yMean = data.map { case (y, features) => y }.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 { case(y, features) => - val nCols = features.length - // Traverse over every column and emit (col, value, value^2) - (0 until nCols).map(i => (i, (features(i), features(i)*features(i)))) - }.reduceByKey { case(x1, x2) => - (x1._1 + x2._1, x1._2 + x2._2) - } - val xColSumsMap = xColSumSq.collectAsMap() - - // Compute mean and unbiased variance using column sums - this.xColMean = (0 until nfeatures).map(x => xColSumsMap(x)._1 / nexamples).toArray - this.xColSd = (0 until nfeatures).map {x => - val v = (xColSumsMap(x)._2 - (math.pow(xColSumsMap(x)._1, 2) / nexamples)) / (nexamples) - math.sqrt(v) - }.toArray - } +trait Regression { + + /** + * Train a model on the provided input dataset. Input data is an RDD of (Label, [Features]) + * + * @return RegressionModel representing the model built. + */ + def train(input: RDD[(Double, Array[Double])]): RegressionModel } diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala index c030223b85..f67fb45134 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -10,13 +10,16 @@ import org.jblas.Solve * Ridge Regression from Joseph Gonzalez's implementation in MLBase */ class RidgeRegressionModel( - weights: DoubleMatrix, - intercept: Double, - val lambdaOpt: Double, - val lambdas: List[(Double, Double, DoubleMatrix)]) extends RegressionModel(weights, intercept) { - - override def predict(test_data: spark.RDD[Array[Double]]) = { - test_data.map(x => (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept) + weights: DoubleMatrix, + intercept: Double, + val lambdaOpt: Double, + val lambdas: List[(Double, Double, DoubleMatrix)]) + extends RegressionModel(weights, intercept) { + + override def predict(test_data: RDD[Array[Double]]) = { + test_data.map { x => + (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept + } } } @@ -24,9 +27,9 @@ class RidgeRegressionData(data: RDD[(Double, Array[Double])]) extends Regression override def normalizeData() = { data.map { case(y, features) => val yNormalized = y - yMean - val featuresNormalized = (0 until nfeatures).map( - column => (features(column) - xColMean(column)) / xColSd(column) - ).toArray + val featuresNormalized = Array.tabulate(nfeatures) { column => + (features(column) - xColMean(column)) / xColSd(column) + }.toArray (yNormalized, featuresNormalized) } } @@ -43,12 +46,9 @@ class RidgeRegressionData(data: RDD[(Double, Array[Double])]) extends Regression } } -object RidgeRegression extends Logging { - - def train(inputData: RDD[(Double, Array[Double])], - lambdaLow: Double = 0.0, - lambdaHigh: Double = 10000.0) = { +class RidgeRegression(lambdaLow: Double, lambdaHigh: Double) extends Regression with Logging { + def train(inputData: RDD[(Double, Array[Double])]): RegressionModel = { inputData.cache() val ridgeData = new RidgeRegressionData(inputData) val data = ridgeData.normalizeData() @@ -125,6 +125,22 @@ object RidgeRegression extends Logging { normModel } +} + +/** + * Helper classes to build a RidgeRegression object. + */ +object RidgeRegression { + + /** + * Build a RidgeRegression object with default arguments as: + * + * @param lowLambda as 0.0 + * @param hiLambda as 100.0 + */ + def builder() = { + new RidgeRegressionBuilder(0.0, 100.0) + } def main(args: Array[String]) { if (args.length != 2) { @@ -133,7 +149,36 @@ object RidgeRegression extends Logging { } val sc = new SparkContext(args(0), "RidgeRegression") val data = MLUtils.loadData(sc, args(1)) - val model = train(data, 0, 1000) + val ridgeReg = RidgeRegression.builder() + .setLowLambda(0) + .setHighLambda(1000) + .build() + + val model = ridgeReg.train(data) sc.stop() } } + +class RidgeRegressionBuilder(lowLambda: Double, hiLambda: Double) { + + /** + * Set the lower bound on binary search for lambda's. Default is 0. + */ + def setLowLambda(low: Double) = { + new RidgeRegressionBuilder(low, this.hiLambda) + } + + /** + * Set the upper bound on binary search for lambda's. Default is 100.0. + */ + def setHighLambda(hi: Double) = { + new RidgeRegressionBuilder(this.lowLambda, hi) + } + + /** + * Build a RidgeRegression object. + */ + def build() = { + new RidgeRegression(lowLambda, hiLambda) + } +} diff --git a/ml/src/main/scala/spark/ml/Updater.scala b/ml/src/main/scala/spark/ml/Updater.scala index bacb059377..3952008e28 100644 --- a/ml/src/main/scala/spark/ml/Updater.scala +++ b/ml/src/main/scala/spark/ml/Updater.scala @@ -3,13 +3,24 @@ package spark.ml import org.jblas.DoubleMatrix abstract class Updater extends Serializable { + /** + * Compute an updated value for weights given the gradient, stepSize and iteration number. + * + * @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 + * + * @return weightsNew - Column matrix containing updated weights + * @return reg_val - regularization value + */ def compute(weightsOlds: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): (DoubleMatrix, Double) } class SimpleUpdater extends Updater { - override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): - (DoubleMatrix, Double) = { + override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, + stepSize: Double, iter: Int): (DoubleMatrix, Double) = { val normGradient = gradient.mul(stepSize / math.sqrt(iter)) (weightsOld.sub(normGradient), 0) } -- cgit v1.2.3 From ed32ec2b3bf93b1090ade63f2be25c2deba890b9 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 12 Jun 2013 16:31:39 -0700 Subject: Update test based on interface changes --- ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala index 50ef472075..eb67974cf0 100644 --- a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala @@ -27,7 +27,12 @@ class RidgeRegressionSuite extends FunSuite { val testData = (0 until 20).map(i => (y(i), xMat(i))).toArray val testRDD = sc.parallelize(testData, 2) - val model = RidgeRegression.train(testRDD, 0, 10) + val ridgeReg = RidgeRegression.builder() + .setLowLambda(0) + .setHighLambda(10) + .build() + + val model = ridgeReg.train(testRDD) assert(model.intercept >= 2.9 && model.intercept <= 3.1) assert(model.weights.length === 2) -- cgit v1.2.3 From 2d0e64900e798db6de5d6f7f725c372a3f20cdd1 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 13 Jun 2013 09:53:34 -0700 Subject: Convert regression classes to builder pattern. Remove extraneous methods and classes --- .../main/scala/spark/ml/LogisticRegression.scala | 77 +++++++++------------- ml/src/main/scala/spark/ml/RidgeRegression.scala | 64 +++++++----------- .../test/scala/spark/ml/RidgeRegressionSuite.scala | 6 +- 3 files changed, 56 insertions(+), 91 deletions(-) diff --git a/ml/src/main/scala/spark/ml/LogisticRegression.scala b/ml/src/main/scala/spark/ml/LogisticRegression.scala index e8e4f96dd3..ddc138cacf 100644 --- a/ml/src/main/scala/spark/ml/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/LogisticRegression.scala @@ -43,9 +43,39 @@ class LogisticRegressionData(data: RDD[(Double, Array[Double])]) extends Regress } } -class LogisticRegression(stepSize: Double, miniBatchFraction: Double, numIters: Int) +class LogisticRegression(var stepSize: Double, var miniBatchFraction: Double, var numIters: Int) extends Regression with Logging { + /** + * Construct a LogisticRegression object with default parameters + */ + def this() = this(1.0, 1.0, 100) + + /** + * Set the step size per-iteration of SGD. Default 1.0. + */ + def setStepSize(step: Double) = { + this.stepSize = step + this + } + + /** + * Set fraction of data to be used for each SGD iteration. Default 1.0. + */ + def setMiniBatchFraction(fraction: Double) = { + this.miniBatchFraction = fraction + this + } + + /** + * Set the number of iterations for SGD. Default 100. + */ + def setNumIterations(iters: Int) = { + this.numIters = iters + this + } + + override def train(input: RDD[(Double, Array[Double])]): RegressionModel = { input.cache() @@ -68,17 +98,6 @@ class LogisticRegression(stepSize: Double, miniBatchFraction: Double, numIters: */ object LogisticRegression { - /** - * Build a logistic regression object with default arguments: - * - * @param stepSize as 1.0 - * @param miniBatchFraction as 1.0 - * @param numIters as 100 - */ - def builder() = { - new LogisticRegressionBuilder(1.0, 1.0, 100) - } - def main(args: Array[String]) { if (args.length != 3) { println("Usage: LogisticRegression ") @@ -86,42 +105,10 @@ object LogisticRegression { } val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadData(sc, args(1)) - val lr = LogisticRegression.builder() + val lr = new LogisticRegression() .setStepSize(2.0) .setNumIterations(args(2).toInt) - .build() val model = lr.train(data) sc.stop() } } - -class LogisticRegressionBuilder(stepSize: Double, miniBatchFraction: Double, numIters: Int) { - - /** - * Set the step size per-iteration of SGD. Default 1.0. - */ - def setStepSize(step: Double) = { - new LogisticRegressionBuilder(step, this.miniBatchFraction, this.numIters) - } - - /** - * Set fraction of data to be used for each SGD iteration. Default 1.0. - */ - def setMiniBatchFraction(fraction: Double) = { - new LogisticRegressionBuilder(this.stepSize, fraction, this.numIters) - } - - /** - * Set the number of iterations for SGD. Default 100. - */ - def setNumIterations(iters: Int) = { - new LogisticRegressionBuilder(this.stepSize, this.miniBatchFraction, iters) - } - - /** - * Build a Logistic regression object. - */ - def build() = { - new LogisticRegression(stepSize, miniBatchFraction, numIters) - } -} diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala index f67fb45134..42b370d43b 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -46,7 +46,26 @@ class RidgeRegressionData(data: RDD[(Double, Array[Double])]) extends Regression } } -class RidgeRegression(lambdaLow: Double, lambdaHigh: Double) extends Regression with Logging { +class RidgeRegression(var lambdaLow: Double, var lambdaHigh: Double) + extends Regression with Logging { + + def this() = this(0.0, 100.0) + + /** + * Set the lower bound on binary search for lambda's. Default is 0. + */ + def setLowLambda(low: Double) = { + this.lambdaLow = low + this + } + + /** + * Set the upper bound on binary search for lambda's. Default is 100.0. + */ + def setHighLambda(hi: Double) = { + this.lambdaHigh = hi + this + } def train(inputData: RDD[(Double, Array[Double])]): RegressionModel = { inputData.cache() @@ -127,21 +146,8 @@ class RidgeRegression(lambdaLow: Double, lambdaHigh: Double) extends Regression } } -/** - * Helper classes to build a RidgeRegression object. - */ object RidgeRegression { - /** - * Build a RidgeRegression object with default arguments as: - * - * @param lowLambda as 0.0 - * @param hiLambda as 100.0 - */ - def builder() = { - new RidgeRegressionBuilder(0.0, 100.0) - } - def main(args: Array[String]) { if (args.length != 2) { println("Usage: RidgeRegression ") @@ -149,36 +155,10 @@ object RidgeRegression { } val sc = new SparkContext(args(0), "RidgeRegression") val data = MLUtils.loadData(sc, args(1)) - val ridgeReg = RidgeRegression.builder() - .setLowLambda(0) - .setHighLambda(1000) - .build() + val ridgeReg = new RidgeRegression().setLowLambda(0) + .setHighLambda(1000) val model = ridgeReg.train(data) sc.stop() } } - -class RidgeRegressionBuilder(lowLambda: Double, hiLambda: Double) { - - /** - * Set the lower bound on binary search for lambda's. Default is 0. - */ - def setLowLambda(low: Double) = { - new RidgeRegressionBuilder(low, this.hiLambda) - } - - /** - * Set the upper bound on binary search for lambda's. Default is 100.0. - */ - def setHighLambda(hi: Double) = { - new RidgeRegressionBuilder(this.lowLambda, hi) - } - - /** - * Build a RidgeRegression object. - */ - def build() = { - new RidgeRegression(lowLambda, hiLambda) - } -} diff --git a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala index eb67974cf0..06b5b1ae31 100644 --- a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala @@ -27,10 +27,8 @@ class RidgeRegressionSuite extends FunSuite { val testData = (0 until 20).map(i => (y(i), xMat(i))).toArray val testRDD = sc.parallelize(testData, 2) - val ridgeReg = RidgeRegression.builder() - .setLowLambda(0) - .setHighLambda(10) - .build() + val ridgeReg = new RidgeRegression().setLowLambda(0) + .setHighLambda(10) val model = ridgeReg.train(testRDD) -- cgit v1.2.3 From 6aadaf4d71e9ed95363638673bda48cecd483c03 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 13 Jun 2013 13:17:20 -0700 Subject: Move normalization to MLUtils and remove Regression trait. --- .../main/scala/spark/ml/LogisticRegression.scala | 64 ++++++++------------- ml/src/main/scala/spark/ml/MLUtils.scala | 46 +++++++++++++++ ml/src/main/scala/spark/ml/Regression.scala | 43 +++----------- ml/src/main/scala/spark/ml/RidgeRegression.scala | 66 ++++++++-------------- .../test/scala/spark/ml/RidgeRegressionSuite.scala | 1 + 5 files changed, 104 insertions(+), 116 deletions(-) diff --git a/ml/src/main/scala/spark/ml/LogisticRegression.scala b/ml/src/main/scala/spark/ml/LogisticRegression.scala index ddc138cacf..76c4fe54c8 100644 --- a/ml/src/main/scala/spark/ml/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/LogisticRegression.scala @@ -8,9 +8,9 @@ import org.jblas.DoubleMatrix * Logistic Regression using Stochastic Gradient Descent. */ class LogisticRegressionModel( - weights: DoubleMatrix, - intercept: Double, - val losses: Array[Double]) extends RegressionModel(weights, intercept) { + val weights: DoubleMatrix, + val intercept: Double, + val losses: Array[Double]) extends RegressionModel { override def predict(test_data: spark.RDD[Array[Double]]) = { test_data.map { x => @@ -20,31 +20,8 @@ class LogisticRegressionModel( } } -class LogisticRegressionData(data: RDD[(Double, Array[Double])]) extends RegressionData(data) { - override def normalizeData() = { - // Shift only the features for LogisticRegression - data.map { case(y, features) => - val featuresNormalized = Array.tabulate(nfeatures) { column => - (features(column) - xColMean(column)) / xColSd(column) - } - (y, featuresNormalized) - } - } - - override def scaleModel(m: RegressionModel) = { - val model = m.asInstanceOf[LogisticRegressionModel] - val colSdMat = new DoubleMatrix(xColSd.length, 1, xColSd:_*) - val colMeanMat = new DoubleMatrix(xColMean.length, 1, xColMean:_*) - - val weights = model.weights.div(colSdMat) - val intercept = -1.0 * model.weights.transpose().mmul(colMeanMat.div(colSdMat)).get(0) - - new LogisticRegressionModel(weights, intercept, model.losses) - } -} - class LogisticRegression(var stepSize: Double, var miniBatchFraction: Double, var numIters: Int) - extends Regression with Logging { + extends Logging { /** * Construct a LogisticRegression object with default parameters @@ -75,17 +52,27 @@ class LogisticRegression(var stepSize: Double, var miniBatchFraction: Double, va this } + def train(input: RDD[(Double, Array[Double])]): LogisticRegressionModel = { + val nfeatures: Int = input.take(1)(0)._2.length + val nexamples: Long = input.count() - override def train(input: RDD[(Double, Array[Double])]): RegressionModel = { - input.cache() + val (yMean, xColMean, xColSd) = MLUtils.computeStats(input, nfeatures, nexamples) + + // Shift only the features for LogisticRegression + val data = input.map { case(y, features) => + val featuresMat = new DoubleMatrix(nfeatures, 1, features:_*) + val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd) + (y, featuresNormalized.toArray) + } - val lrData = new LogisticRegressionData(input) - val data = lrData.normalizeData() val (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, numIters) + data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, miniBatchFraction) + + val weightsScaled = weights.div(xColSd) + val intercept = -1.0 * weights.transpose().mmul(xColMean.div(xColSd)).get(0) + + val model = new LogisticRegressionModel(weightsScaled, intercept, losses) - val computedModel = new LogisticRegressionModel(weights, 0, losses) - val model = lrData.scaleModel(computedModel) logInfo("Final model weights " + model.weights) logInfo("Final model intercept " + model.intercept) logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) @@ -99,15 +86,14 @@ class LogisticRegression(var stepSize: Double, var miniBatchFraction: Double, va object LogisticRegression { def main(args: Array[String]) { - if (args.length != 3) { - println("Usage: LogisticRegression ") + if (args.length != 4) { + println("Usage: LogisticRegression ") System.exit(1) } val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadData(sc, args(1)) - val lr = new LogisticRegression() - .setStepSize(2.0) - .setNumIterations(args(2).toInt) + val lr = new LogisticRegression().setStepSize(args(2).toDouble) + .setNumIterations(args(3).toInt) val model = lr.train(data) sc.stop() } diff --git a/ml/src/main/scala/spark/ml/MLUtils.scala b/ml/src/main/scala/spark/ml/MLUtils.scala index d62198f426..b2361e5564 100644 --- a/ml/src/main/scala/spark/ml/MLUtils.scala +++ b/ml/src/main/scala/spark/ml/MLUtils.scala @@ -1,6 +1,9 @@ package spark.ml import spark.{RDD, SparkContext} +import spark.SparkContext._ + +import org.jblas.DoubleMatrix /** * Helper methods to load and save data @@ -30,4 +33,47 @@ object MLUtils { 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[(Double, Array[Double])], nfeatures: Int, nexamples: Long): + (Double, DoubleMatrix, DoubleMatrix) = { + val yMean: Double = data.map { case (y, features) => y }.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 { case(y, features) => + val nCols = features.length + // Traverse over every column and emit (col, value, value^2) + Iterator.tabulate(nCols) { i => + (i, (features(i), features(i)*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) + } } diff --git a/ml/src/main/scala/spark/ml/Regression.scala b/ml/src/main/scala/spark/ml/Regression.scala index 26a3a05545..f23524bfa8 100644 --- a/ml/src/main/scala/spark/ml/Regression.scala +++ b/ml/src/main/scala/spark/ml/Regression.scala @@ -7,36 +7,17 @@ import spark.SparkContext._ import org.jblas.DoubleMatrix -abstract class RegressionModel(val weights: DoubleMatrix, val intercept: Double) { +trait RegressionModel { + /** + * Predict values for the given data set using the model trained. + * + * @param test_data RDD representing data points to be predicted + * @return RDD[Double] where each entry contains the corresponding prediction + */ def predict(test_data: RDD[Array[Double]]): RDD[Double] } abstract class RegressionData(val data: RDD[(Double, Array[Double])]) extends Serializable { - val nfeatures: Int = data.take(1)(0)._2.length - val nexamples: Long = data.count() - - val yMean: Double = data.map { case (y, features) => y }.reduce(_ + _) / nexamples - - // NOTE: We shuffle X by column here to compute column sum and sum of squares. - private val xColSumSq: RDD[(Int, (Double, Double))] = data.flatMap { case(y, features) => - val nCols = features.length - // Traverse over every column and emit (col, value, value^2) - Iterator.tabulate(nCols) { i => - (i, (features(i), features(i)*features(i))) - } - }.reduceByKey { case(x1, x2) => - (x1._1 + x2._1, x1._2 + x2._2) - } - private val xColSumsMap = xColSumSq.collectAsMap() - - // Compute mean and unbiased variance using column sums - val xColMean: Array[Double] = Array.tabulate(nfeatures) { x => - xColSumsMap(x)._1 / nexamples - } - val xColSd: Array[Double] = Array.tabulate(nfeatures) { x => - val v = (xColSumsMap(x)._2 - (math.pow(xColSumsMap(x)._1, 2) / nexamples)) / (nexamples) - math.sqrt(v) - } /** * Normalize the provided input data. This function is typically called before @@ -55,13 +36,3 @@ abstract class RegressionData(val data: RDD[(Double, Array[Double])]) extends Se */ def scaleModel(model: RegressionModel): RegressionModel } - -trait Regression { - - /** - * Train a model on the provided input dataset. Input data is an RDD of (Label, [Features]) - * - * @return RegressionModel representing the model built. - */ - def train(input: RDD[(Double, Array[Double])]): RegressionModel -} diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala index 42b370d43b..36db2570d9 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -10,11 +10,11 @@ import org.jblas.Solve * Ridge Regression from Joseph Gonzalez's implementation in MLBase */ class RidgeRegressionModel( - weights: DoubleMatrix, - intercept: Double, + val weights: DoubleMatrix, + val intercept: Double, val lambdaOpt: Double, val lambdas: List[(Double, Double, DoubleMatrix)]) - extends RegressionModel(weights, intercept) { + extends RegressionModel { override def predict(test_data: RDD[Array[Double]]) = { test_data.map { x => @@ -23,31 +23,8 @@ class RidgeRegressionModel( } } -class RidgeRegressionData(data: RDD[(Double, Array[Double])]) extends RegressionData(data) { - override def normalizeData() = { - data.map { case(y, features) => - val yNormalized = y - yMean - val featuresNormalized = Array.tabulate(nfeatures) { column => - (features(column) - xColMean(column)) / xColSd(column) - }.toArray - (yNormalized, featuresNormalized) - } - } - - override def scaleModel(m: RegressionModel) = { - val model = m.asInstanceOf[RidgeRegressionModel] - val colSdMat = new DoubleMatrix(xColSd.length, 1, xColSd:_*) - val colMeanMat = new DoubleMatrix(xColMean.length, 1, xColMean:_*) - - val weights = model.weights.div(colSdMat) - val intercept = yMean - model.weights.transpose().mmul(colMeanMat.div(colSdMat)).get(0) - - new RidgeRegressionModel(weights, intercept, model.lambdaOpt, model.lambdas) - } -} - class RidgeRegression(var lambdaLow: Double, var lambdaHigh: Double) - extends Regression with Logging { + extends Logging { def this() = this(0.0, 100.0) @@ -67,23 +44,29 @@ class RidgeRegression(var lambdaLow: Double, var lambdaHigh: Double) this } - def train(inputData: RDD[(Double, Array[Double])]): RegressionModel = { - inputData.cache() - val ridgeData = new RidgeRegressionData(inputData) - val data = ridgeData.normalizeData() - val nfeatures: Int = ridgeData.nfeatures - val nexamples: Long = ridgeData.nexamples + def train(input: RDD[(Double, Array[Double])]): RidgeRegressionModel = { + val nfeatures: Int = input.take(1)(0)._2.length + val nexamples: Long = input.count() + + val (yMean, xColMean, xColSd) = MLUtils.computeStats(input, nfeatures, nexamples) + + val data = input.map { case(y, features) => + val yNormalized = y - yMean + val featuresMat = new DoubleMatrix(nfeatures, 1, features:_*) + val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd) + (yNormalized, featuresNormalized.toArray) + } // Compute XtX - Size of XtX is nfeatures by nfeatures val XtX: DoubleMatrix = data.map { case (y, features) => val x = new DoubleMatrix(1, features.length, features:_*) x.transpose().mmul(x) - }.reduce(_.add(_)) + }.reduce(_.addi(_)) // Compute Xt*y - Size of Xty is nfeatures by 1 val Xty: DoubleMatrix = data.map { case (y, features) => new DoubleMatrix(features.length, 1, features:_*).mul(y) - }.reduce(_.add(_)) + }.reduce(_.addi(_)) // Define a function to compute the leave one out cross validation error // for a single example @@ -134,15 +117,16 @@ class RidgeRegression(var lambdaLow: Double, var lambdaHigh: Double) val (lambdaOpt, cverror, weights) = lambdas.reduce((a, b) => if (a._2 < b._2) a else b) // Return the model which contains the solution - val trainModel = new RidgeRegressionModel(weights, 0.0, lambdaOpt, lambdas) - val normModel = ridgeData.scaleModel(trainModel) + val weightsScaled = weights.div(xColSd) + val intercept = yMean - (weights.transpose().mmul(xColMean.div(xColSd)).get(0)) + val model = new RidgeRegressionModel(weightsScaled, intercept, lambdaOpt, lambdas) - logInfo("RidgeRegression: optimal lambda " + normModel.lambdaOpt) - logInfo("RidgeRegression: optimal weights " + normModel.weights) - logInfo("RidgeRegression: optimal intercept " + normModel.intercept) + logInfo("RidgeRegression: optimal lambda " + model.lambdaOpt) + logInfo("RidgeRegression: optimal weights " + model.weights) + logInfo("RidgeRegression: optimal intercept " + model.intercept) logInfo("RidgeRegression: cross-validation error " + cverror) - normModel + model } } diff --git a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala index 06b5b1ae31..07da5081b3 100644 --- a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala @@ -27,6 +27,7 @@ class RidgeRegressionSuite extends FunSuite { val testData = (0 until 20).map(i => (y(i), xMat(i))).toArray val testRDD = sc.parallelize(testData, 2) + testRDD.cache() val ridgeReg = new RidgeRegression().setLowLambda(0) .setHighLambda(10) -- cgit v1.2.3 From 3a6924cb8ff082eed0243b0684de88a858beee7d Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 13 Jun 2013 13:25:10 -0700 Subject: Clean up some comments. --- ml/src/main/scala/spark/ml/GradientDescent.scala | 1 + ml/src/main/scala/spark/ml/LogisticRegression.scala | 4 +--- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/ml/src/main/scala/spark/ml/GradientDescent.scala b/ml/src/main/scala/spark/ml/GradientDescent.scala index a03285bd53..564f89e5ee 100644 --- a/ml/src/main/scala/spark/ml/GradientDescent.scala +++ b/ml/src/main/scala/spark/ml/GradientDescent.scala @@ -12,6 +12,7 @@ object GradientDescent { /** * Run gradient descent in parallel using mini batches. + * Based on Matlab code written by John Duchi. * * @param data - Input data for SGD. RDD of form (label, [feature values]). * @param gradient - Gradient object that will be used to compute the gradient. diff --git a/ml/src/main/scala/spark/ml/LogisticRegression.scala b/ml/src/main/scala/spark/ml/LogisticRegression.scala index 76c4fe54c8..c134f8d77d 100644 --- a/ml/src/main/scala/spark/ml/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/LogisticRegression.scala @@ -6,6 +6,7 @@ import org.jblas.DoubleMatrix /** * Logistic Regression using Stochastic Gradient Descent. + * Based on Matlab code written by John Duchi. */ class LogisticRegressionModel( val weights: DoubleMatrix, @@ -80,9 +81,6 @@ class LogisticRegression(var stepSize: Double, var miniBatchFraction: Double, va } } -/** - * Helper classes to build a LogisticRegression object. - */ object LogisticRegression { def main(args: Array[String]) { -- cgit v1.2.3 From 76acc9fe9d76c43ee063642db3ea2a9190d97eb6 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 13 Jun 2013 14:15:03 -0700 Subject: Make regression arguments private and add method to predict one point --- .../main/scala/spark/ml/LogisticRegression.scala | 13 ++++++--- ml/src/main/scala/spark/ml/Regression.scala | 31 +++++----------------- ml/src/main/scala/spark/ml/RidgeRegression.scala | 10 ++++--- 3 files changed, 24 insertions(+), 30 deletions(-) diff --git a/ml/src/main/scala/spark/ml/LogisticRegression.scala b/ml/src/main/scala/spark/ml/LogisticRegression.scala index c134f8d77d..4dc883fbf8 100644 --- a/ml/src/main/scala/spark/ml/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/LogisticRegression.scala @@ -13,15 +13,22 @@ class LogisticRegressionModel( val intercept: Double, val losses: Array[Double]) extends RegressionModel { - override def predict(test_data: spark.RDD[Array[Double]]) = { - test_data.map { x => + override def predict(testData: spark.RDD[Array[Double]]) = { + testData.map { x => val margin = new DoubleMatrix(1, x.length, x:_*).mmul(this.weights).get(0) + this.intercept 1.0/ (1.0 + math.exp(margin * -1)) } } + + override def predict(testData: Array[Double]): Double = { + val dataMat = new DoubleMatrix(1, testData.length, testData:_*) + val margin = dataMat.mmul(this.weights).get(0) + this.intercept + 1.0/ (1.0 + math.exp(margin * -1)) + } } -class LogisticRegression(var stepSize: Double, var miniBatchFraction: Double, var numIters: Int) +class LogisticRegression(private var stepSize: Double, private var miniBatchFraction: Double, + private var numIters: Int) extends Logging { /** diff --git a/ml/src/main/scala/spark/ml/Regression.scala b/ml/src/main/scala/spark/ml/Regression.scala index f23524bfa8..9d7424d656 100644 --- a/ml/src/main/scala/spark/ml/Regression.scala +++ b/ml/src/main/scala/spark/ml/Regression.scala @@ -1,38 +1,21 @@ package spark.ml -import java.io._ - -import spark.{RDD, SparkContext} -import spark.SparkContext._ - -import org.jblas.DoubleMatrix +import spark.RDD trait RegressionModel { /** * Predict values for the given data set using the model trained. * - * @param test_data RDD representing data points to be predicted + * @param testData RDD representing data points to be predicted * @return RDD[Double] where each entry contains the corresponding prediction */ - def predict(test_data: RDD[Array[Double]]): RDD[Double] -} - -abstract class RegressionData(val data: RDD[(Double, Array[Double])]) extends Serializable { - - /** - * Normalize the provided input data. This function is typically called before - * training a classifier on the input dataset and should be used to center of scale the data - * appropriately. - * - * @return RDD containing the normalized data - */ - def normalizeData(): RDD[(Double, Array[Double])] + def predict(testData: RDD[Array[Double]]): RDD[Double] /** - * Scale the trained regression model. This function is usually called after training - * to adjust the model based on the normalization performed before. + * Predict values for a single data point using the model trained. * - * @return Regression model that can be used for prediction + * @param testData array representing a single data point + * @return Double prediction from the trained model */ - def scaleModel(model: RegressionModel): RegressionModel + def predict(testData: Array[Double]): Double } diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala index 36db2570d9..628fc25933 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -16,14 +16,18 @@ class RidgeRegressionModel( val lambdas: List[(Double, Double, DoubleMatrix)]) extends RegressionModel { - override def predict(test_data: RDD[Array[Double]]) = { - test_data.map { x => + override def predict(testData: RDD[Array[Double]]): RDD[Double] = { + testData.map { x => (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept } } + + override def predict(testData: Array[Double]): Double = { + (new DoubleMatrix(1, testData.length, testData:_*).mmul(this.weights)).get(0) + this.intercept + } } -class RidgeRegression(var lambdaLow: Double, var lambdaHigh: Double) +class RidgeRegression(private var lambdaLow: Double, private var lambdaHigh: Double) extends Logging { def this() = this(0.0, 100.0) -- cgit v1.2.3 From 6dd3a816c8d438e1d8beebe0a1cbf5d90ef6e3bc Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 13 Jun 2013 15:11:21 -0700 Subject: Use a private constructor instead of private vars --- ml/src/main/scala/spark/ml/LogisticRegression.scala | 4 ++-- ml/src/main/scala/spark/ml/RidgeRegression.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/ml/src/main/scala/spark/ml/LogisticRegression.scala b/ml/src/main/scala/spark/ml/LogisticRegression.scala index 4dc883fbf8..3417bfe882 100644 --- a/ml/src/main/scala/spark/ml/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/LogisticRegression.scala @@ -27,8 +27,8 @@ class LogisticRegressionModel( } } -class LogisticRegression(private var stepSize: Double, private var miniBatchFraction: Double, - private var numIters: Int) +class LogisticRegression private (var stepSize: Double, var miniBatchFraction: Double, + var numIters: Int) extends Logging { /** diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala index 628fc25933..3cbb0653c3 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/RidgeRegression.scala @@ -27,7 +27,7 @@ class RidgeRegressionModel( } } -class RidgeRegression(private var lambdaLow: Double, private var lambdaHigh: Double) +class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) extends Logging { def this() = this(0.0, 100.0) -- cgit v1.2.3 From 43b398db6a2008a9dc0343df0ec0b24f96fa65c1 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 14 Jun 2013 16:13:12 -0700 Subject: Fix logistic regression to not center data. Also add a feature to get the intercept correct and test these using a small unit test. --- .../main/scala/spark/ml/LogisticRegression.scala | 17 ++----- .../scala/spark/ml/LogisticRegressionSuite.scala | 54 ++++++++++++++++++++++ 2 files changed, 59 insertions(+), 12 deletions(-) create mode 100644 ml/src/test/scala/spark/ml/LogisticRegressionSuite.scala diff --git a/ml/src/main/scala/spark/ml/LogisticRegression.scala b/ml/src/main/scala/spark/ml/LogisticRegression.scala index 3417bfe882..14c237c5ed 100644 --- a/ml/src/main/scala/spark/ml/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/LogisticRegression.scala @@ -61,23 +61,16 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D } def train(input: RDD[(Double, Array[Double])]): LogisticRegressionModel = { - val nfeatures: Int = input.take(1)(0)._2.length - val nexamples: Long = input.count() - - val (yMean, xColMean, xColSd) = MLUtils.computeStats(input, nfeatures, nexamples) - - // Shift only the features for LogisticRegression - val data = input.map { case(y, features) => - val featuresMat = new DoubleMatrix(nfeatures, 1, features:_*) - val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd) - (y, featuresNormalized.toArray) + // Add a extra variable consisting of all 1.0's for the intercept. + val data = input.map { case (y, features) => + (y, Array(1.0, features:_*)) } val (weights, losses) = GradientDescent.runMiniBatchSGD( data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, miniBatchFraction) - val weightsScaled = weights.div(xColSd) - val intercept = -1.0 * weights.transpose().mmul(xColMean.div(xColSd)).get(0) + val weightsScaled = weights.getRange(1, weights.length) + val intercept = weights.get(0) val model = new LogisticRegressionModel(weightsScaled, intercept, losses) diff --git a/ml/src/test/scala/spark/ml/LogisticRegressionSuite.scala b/ml/src/test/scala/spark/ml/LogisticRegressionSuite.scala new file mode 100644 index 0000000000..d66c2c8b35 --- /dev/null +++ b/ml/src/test/scala/spark/ml/LogisticRegressionSuite.scala @@ -0,0 +1,54 @@ +package spark.ml + +import spark.SparkContext +import spark.SparkContext._ +import spark.Logging + +import org.apache.commons.math3.distribution.NormalDistribution +import org.scalatest.FunSuite + +class LogisticRegressionSuite extends FunSuite with Logging { + + // Test if we can correctly learn A, B where Y = logistic(A + B*X) + test("logistic regression") { + val nPoints = 10000 + val rnd = new NormalDistribution(0, 1) + rnd.reseedRandomGenerator(42) + + val sc = new SparkContext("local", "test") + val x1 = Array.fill[Double](nPoints)(rnd.sample()) + + val A = 2.0 + val B = -1.5 + + // 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(100) + // y <- as.numeric(y > 0) + val y = (0 until nPoints).map { i => + val yVal = A + B * x1(i) + rLogis(i) + if (yVal > 0) 1.0 else 0.0 + } + + val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i)))).toArray + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + val lr = new LogisticRegression().setStepSize(1.0) + .setNumIterations(1000) + + val model = lr.train(testRDD) + + assert(model.weights.get(0) >= -1.60 && model.weights.get(0) <= -1.40) + assert(model.intercept >= 1.9 && model.intercept <= 2.1) + + sc.stop() + // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown + System.clearProperty("spark.driver.port") + } +} -- cgit v1.2.3 From 39ed41652beaad63c03540411a51ed82c1126e6d Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sat, 15 Jun 2013 09:03:20 -0700 Subject: Move to regression, util and gradient packages --- ml/src/main/scala/spark/ml/Gradient.scala | 33 ----- ml/src/main/scala/spark/ml/GradientDescent.scala | 62 --------- .../main/scala/spark/ml/LogisticRegression.scala | 98 ------------- .../spark/ml/LogisticRegressionGenerator.scala | 40 ------ ml/src/main/scala/spark/ml/MLUtils.scala | 79 ----------- ml/src/main/scala/spark/ml/Regression.scala | 21 --- ml/src/main/scala/spark/ml/RidgeRegression.scala | 152 -------------------- .../scala/spark/ml/RidgeRegressionGenerator.scala | 54 -------- ml/src/main/scala/spark/ml/Updater.scala | 27 ---- ml/src/main/scala/spark/ml/gradient/Gradient.scala | 33 +++++ .../scala/spark/ml/gradient/GradientDescent.scala | 62 +++++++++ ml/src/main/scala/spark/ml/gradient/Updater.scala | 27 ++++ .../spark/ml/regression/LogisticRegression.scala | 100 ++++++++++++++ .../regression/LogisticRegressionGenerator.scala | 41 ++++++ .../scala/spark/ml/regression/Regression.scala | 21 +++ .../spark/ml/regression/RidgeRegression.scala | 153 +++++++++++++++++++++ .../ml/regression/RidgeRegressionGenerator.scala | 55 ++++++++ ml/src/main/scala/spark/ml/util/MLUtils.scala | 79 +++++++++++ .../scala/spark/ml/LogisticRegressionSuite.scala | 54 -------- .../test/scala/spark/ml/RidgeRegressionSuite.scala | 45 ------ .../ml/regression/LogisticRegressionSuite.scala | 54 ++++++++ .../spark/ml/regression/RidgeRegressionSuite.scala | 45 ++++++ 22 files changed, 670 insertions(+), 665 deletions(-) delete mode 100644 ml/src/main/scala/spark/ml/Gradient.scala delete mode 100644 ml/src/main/scala/spark/ml/GradientDescent.scala delete mode 100644 ml/src/main/scala/spark/ml/LogisticRegression.scala delete mode 100644 ml/src/main/scala/spark/ml/LogisticRegressionGenerator.scala delete mode 100644 ml/src/main/scala/spark/ml/MLUtils.scala delete mode 100644 ml/src/main/scala/spark/ml/Regression.scala delete mode 100644 ml/src/main/scala/spark/ml/RidgeRegression.scala delete mode 100644 ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala delete mode 100644 ml/src/main/scala/spark/ml/Updater.scala create mode 100644 ml/src/main/scala/spark/ml/gradient/Gradient.scala create mode 100644 ml/src/main/scala/spark/ml/gradient/GradientDescent.scala create mode 100644 ml/src/main/scala/spark/ml/gradient/Updater.scala create mode 100644 ml/src/main/scala/spark/ml/regression/LogisticRegression.scala create mode 100644 ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala create mode 100644 ml/src/main/scala/spark/ml/regression/Regression.scala create mode 100644 ml/src/main/scala/spark/ml/regression/RidgeRegression.scala create mode 100644 ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala create mode 100644 ml/src/main/scala/spark/ml/util/MLUtils.scala delete mode 100644 ml/src/test/scala/spark/ml/LogisticRegressionSuite.scala delete mode 100644 ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala create mode 100644 ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala create mode 100644 ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala diff --git a/ml/src/main/scala/spark/ml/Gradient.scala b/ml/src/main/scala/spark/ml/Gradient.scala deleted file mode 100644 index fe36187000..0000000000 --- a/ml/src/main/scala/spark/ml/Gradient.scala +++ /dev/null @@ -1,33 +0,0 @@ -package spark.ml - -import org.jblas.DoubleMatrix - -abstract class Gradient extends Serializable { - /** - * Compute the gradient for a given row of data. - * - * @param data - One row of data. Row matrix of size 1xn where n is the number of features. - * @param label - Label for this data item. - * @param weights - Column matrix containing weights for every feature. - */ - def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): - (DoubleMatrix, Double) -} - -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) - } -} diff --git a/ml/src/main/scala/spark/ml/GradientDescent.scala b/ml/src/main/scala/spark/ml/GradientDescent.scala deleted file mode 100644 index 564f89e5ee..0000000000 --- a/ml/src/main/scala/spark/ml/GradientDescent.scala +++ /dev/null @@ -1,62 +0,0 @@ -package spark.ml - -import spark.{Logging, RDD, SparkContext} -import spark.SparkContext._ - -import org.jblas.DoubleMatrix - -import scala.collection.mutable.ArrayBuffer - - -object GradientDescent { - - /** - * Run gradient descent in parallel using mini batches. - * Based on Matlab code written by John Duchi. - * - * @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 numIters - number of iterations that SGD should be run. - * @param miniBatchFraction - fraction of the input data set that should be used for - * one iteration of SGD. Default value 1.0. - * - * @return weights - Column matrix containing weights for every feature. - * @return lossHistory - Array containing the loss computed for every iteration. - */ - def runMiniBatchSGD( - data: RDD[(Double, Array[Double])], - gradient: Gradient, - updater: Updater, - stepSize: Double, - numIters: Int, - miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { - - val lossHistory = new ArrayBuffer[Double](numIters) - - val nfeatures: Int = data.take(1)(0)._2.length - val nexamples: Long = data.count() - val miniBatchSize = nexamples * miniBatchFraction - - // Initialize weights as a column matrix - var weights = DoubleMatrix.ones(nfeatures) - var reg_val = 0.0 - - for (i <- 1 to numIters) { - val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42+i).map { - case (y, features) => - val featuresRow = new DoubleMatrix(features.length, 1, features:_*) - val (grad, loss) = gradient.compute(featuresRow, y, weights) - (grad, loss) - }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) - - lossHistory.append(lossSum / miniBatchSize + reg_val) - val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i) - weights = update._1 - reg_val = update._2 - } - - (weights, lossHistory.toArray) - } -} diff --git a/ml/src/main/scala/spark/ml/LogisticRegression.scala b/ml/src/main/scala/spark/ml/LogisticRegression.scala deleted file mode 100644 index 14c237c5ed..0000000000 --- a/ml/src/main/scala/spark/ml/LogisticRegression.scala +++ /dev/null @@ -1,98 +0,0 @@ -package spark.ml - -import spark.{Logging, RDD, SparkContext} - -import org.jblas.DoubleMatrix - -/** - * Logistic Regression using Stochastic Gradient Descent. - * Based on Matlab code written by John Duchi. - */ -class LogisticRegressionModel( - val weights: DoubleMatrix, - val intercept: Double, - val losses: Array[Double]) extends RegressionModel { - - override def predict(testData: spark.RDD[Array[Double]]) = { - testData.map { x => - val margin = new DoubleMatrix(1, x.length, x:_*).mmul(this.weights).get(0) + this.intercept - 1.0/ (1.0 + math.exp(margin * -1)) - } - } - - override def predict(testData: Array[Double]): Double = { - val dataMat = new DoubleMatrix(1, testData.length, testData:_*) - val margin = dataMat.mmul(this.weights).get(0) + this.intercept - 1.0/ (1.0 + math.exp(margin * -1)) - } -} - -class LogisticRegression private (var stepSize: Double, var miniBatchFraction: Double, - var numIters: Int) - extends Logging { - - /** - * Construct a LogisticRegression object with default parameters - */ - def this() = this(1.0, 1.0, 100) - - /** - * Set the step size per-iteration of SGD. Default 1.0. - */ - def setStepSize(step: Double) = { - this.stepSize = step - this - } - - /** - * Set fraction of data to be used for each SGD iteration. Default 1.0. - */ - def setMiniBatchFraction(fraction: Double) = { - this.miniBatchFraction = fraction - this - } - - /** - * Set the number of iterations for SGD. Default 100. - */ - def setNumIterations(iters: Int) = { - this.numIters = iters - this - } - - def train(input: RDD[(Double, Array[Double])]): LogisticRegressionModel = { - // Add a extra variable consisting of all 1.0's for the intercept. - val data = input.map { case (y, features) => - (y, Array(1.0, features:_*)) - } - - val (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, miniBatchFraction) - - val weightsScaled = weights.getRange(1, weights.length) - val intercept = weights.get(0) - - val model = new LogisticRegressionModel(weightsScaled, intercept, losses) - - logInfo("Final model weights " + model.weights) - logInfo("Final model intercept " + model.intercept) - logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) - model - } -} - -object LogisticRegression { - - 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.loadData(sc, args(1)) - val lr = new LogisticRegression().setStepSize(args(2).toDouble) - .setNumIterations(args(3).toInt) - val model = lr.train(data) - sc.stop() - } -} diff --git a/ml/src/main/scala/spark/ml/LogisticRegressionGenerator.scala b/ml/src/main/scala/spark/ml/LogisticRegressionGenerator.scala deleted file mode 100644 index ff46cfa46d..0000000000 --- a/ml/src/main/scala/spark/ml/LogisticRegressionGenerator.scala +++ /dev/null @@ -1,40 +0,0 @@ -package spark.ml - -import spark.{RDD, SparkContext} - -import org.apache.commons.math3.distribution.NormalDistribution -import org.jblas.DoubleMatrix - -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 NormalDistribution(0, 1) - rnd.reseedRandomGenerator(42 + idx) - - val y = if (idx % 2 == 0) 0 else 1 - val x = Array.fill[Double](nfeatures) { - rnd.sample() + (y * eps) - } - (y, x) - } - - MLUtils.saveData(data, outputPath) - sc.stop() - } -} diff --git a/ml/src/main/scala/spark/ml/MLUtils.scala b/ml/src/main/scala/spark/ml/MLUtils.scala deleted file mode 100644 index b2361e5564..0000000000 --- a/ml/src/main/scala/spark/ml/MLUtils.scala +++ /dev/null @@ -1,79 +0,0 @@ -package spark.ml - -import spark.{RDD, SparkContext} -import spark.SparkContext._ - -import org.jblas.DoubleMatrix - -/** - * Helper methods to load and save data - * Data format: - * , ... - * where , are feature values in Double and is the corresponding label as Double. - */ -object MLUtils { - - /** - * @param sc SparkContext - * @param dir Directory to the input data files. - * @return An RDD of tuples. For each tuple, the first element is the label, and the second - * element represents the feature values (an array of Double). - */ - def loadData(sc: SparkContext, dir: String): RDD[(Double, Array[Double])] = { - sc.textFile(dir).map { line => - val parts = line.split(",") - val label = parts(0).toDouble - val features = parts(1).trim().split(" ").map(_.toDouble) - (label, features) - } - } - - def saveData(data: RDD[(Double, Array[Double])], dir: String) { - val dataStr = data.map(x => x._1 + "," + x._2.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[(Double, Array[Double])], nfeatures: Int, nexamples: Long): - (Double, DoubleMatrix, DoubleMatrix) = { - val yMean: Double = data.map { case (y, features) => y }.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 { case(y, features) => - val nCols = features.length - // Traverse over every column and emit (col, value, value^2) - Iterator.tabulate(nCols) { i => - (i, (features(i), features(i)*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) - } -} diff --git a/ml/src/main/scala/spark/ml/Regression.scala b/ml/src/main/scala/spark/ml/Regression.scala deleted file mode 100644 index 9d7424d656..0000000000 --- a/ml/src/main/scala/spark/ml/Regression.scala +++ /dev/null @@ -1,21 +0,0 @@ -package spark.ml - -import spark.RDD - -trait RegressionModel { - /** - * 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/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/RidgeRegression.scala deleted file mode 100644 index 3cbb0653c3..0000000000 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ /dev/null @@ -1,152 +0,0 @@ -package spark.ml - -import spark.{Logging, RDD, SparkContext} -import spark.SparkContext._ - -import org.jblas.DoubleMatrix -import org.jblas.Solve - -/** - * Ridge Regression from Joseph Gonzalez's implementation in MLBase - */ -class RidgeRegressionModel( - val weights: DoubleMatrix, - val intercept: Double, - val lambdaOpt: Double, - val lambdas: List[(Double, Double, DoubleMatrix)]) - extends RegressionModel { - - override def predict(testData: RDD[Array[Double]]): RDD[Double] = { - testData.map { x => - (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept - } - } - - override def predict(testData: Array[Double]): Double = { - (new DoubleMatrix(1, testData.length, testData:_*).mmul(this.weights)).get(0) + this.intercept - } -} - -class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) - extends Logging { - - def this() = this(0.0, 100.0) - - /** - * Set the lower bound on binary search for lambda's. Default is 0. - */ - def setLowLambda(low: Double) = { - this.lambdaLow = low - this - } - - /** - * Set the upper bound on binary search for lambda's. Default is 100.0. - */ - def setHighLambda(hi: Double) = { - this.lambdaHigh = hi - this - } - - def train(input: RDD[(Double, Array[Double])]): RidgeRegressionModel = { - val nfeatures: Int = input.take(1)(0)._2.length - val nexamples: Long = input.count() - - val (yMean, xColMean, xColSd) = MLUtils.computeStats(input, nfeatures, nexamples) - - val data = input.map { case(y, features) => - val yNormalized = y - yMean - val featuresMat = new DoubleMatrix(nfeatures, 1, features:_*) - val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd) - (yNormalized, featuresNormalized.toArray) - } - - // Compute XtX - Size of XtX is nfeatures by nfeatures - val XtX: DoubleMatrix = data.map { case (y, features) => - val x = new DoubleMatrix(1, features.length, features:_*) - x.transpose().mmul(x) - }.reduce(_.addi(_)) - - // Compute Xt*y - Size of Xty is nfeatures by 1 - val Xty: DoubleMatrix = data.map { case (y, features) => - new DoubleMatrix(features.length, 1, features:_*).mul(y) - }.reduce(_.addi(_)) - - // Define a function to compute the leave one out cross validation error - // for a single example - def crossValidate(lambda: Double): (Double, Double, DoubleMatrix) = { - // Compute the MLE ridge regression parameter value - - // Ridge Regression parameter = inv(XtX + \lambda*I) * Xty - val XtXlambda = DoubleMatrix.eye(nfeatures).muli(lambda).addi(XtX) - val w = Solve.solveSymmetric(XtXlambda, Xty) - - val invXtX = Solve.solveSymmetric(XtXlambda, DoubleMatrix.eye(nfeatures)) - - // compute the generalized cross validation score - val cverror = data.map { - case (y, features) => - val x = new DoubleMatrix(features.length, 1, features:_*) - val yhat = w.transpose().mmul(x).get(0) - val H_ii = x.transpose().mmul(invXtX).mmul(x).get(0) - val residual = (y - yhat) / (1.0 - H_ii) - residual * residual - }.reduce(_ + _) / nexamples - - (lambda, cverror, w) - } - - // Binary search for the best assignment to lambda. - def binSearch(low: Double, high: Double): List[(Double, Double, DoubleMatrix)] = { - val mid = (high - low) / 2 + low - val lowValue = crossValidate((mid - low) / 2 + low) - val highValue = crossValidate((high - mid) / 2 + mid) - val (newLow, newHigh) = if (lowValue._2 < highValue._2) { - (low, mid + (high-low)/4) - } else { - (mid - (high-low)/4, high) - } - if (newHigh - newLow > 1.0E-7) { - // :: is list prepend in Scala. - lowValue :: highValue :: binSearch(newLow, newHigh) - } else { - List(lowValue, highValue) - } - } - - // Actually compute the best lambda - val lambdas = binSearch(lambdaLow, lambdaHigh).sortBy(_._1) - - // Find the best parameter set by taking the lowest cverror. - val (lambdaOpt, cverror, weights) = lambdas.reduce((a, b) => if (a._2 < b._2) a else b) - - // Return the model which contains the solution - val weightsScaled = weights.div(xColSd) - val intercept = yMean - (weights.transpose().mmul(xColMean.div(xColSd)).get(0)) - val model = new RidgeRegressionModel(weightsScaled, intercept, lambdaOpt, lambdas) - - logInfo("RidgeRegression: optimal lambda " + model.lambdaOpt) - logInfo("RidgeRegression: optimal weights " + model.weights) - logInfo("RidgeRegression: optimal intercept " + model.intercept) - logInfo("RidgeRegression: cross-validation error " + cverror) - - model - } -} - -object RidgeRegression { - - def main(args: Array[String]) { - if (args.length != 2) { - println("Usage: RidgeRegression ") - System.exit(1) - } - val sc = new SparkContext(args(0), "RidgeRegression") - val data = MLUtils.loadData(sc, args(1)) - val ridgeReg = new RidgeRegression().setLowLambda(0) - .setHighLambda(1000) - - val model = ridgeReg.train(data) - sc.stop() - } -} diff --git a/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala b/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala deleted file mode 100644 index e6d3e94787..0000000000 --- a/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala +++ /dev/null @@ -1,54 +0,0 @@ -package spark.ml - -import spark.{RDD, SparkContext} - -import org.apache.commons.math3.distribution.NormalDistribution -import org.jblas.DoubleMatrix - - -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 NormalDistribution(0, eps) - rnd.reseedRandomGenerator(42 + p) - - val normalValues = Array.fill[Double](examplesInPartition)(rnd.sample()) - val yObs = new DoubleMatrix(normalValues).addi(y) - - Iterator.tabulate(examplesInPartition) { i => - (yObs.get(i, 0), X.getRow(i).toArray) - } - } - - MLUtils.saveData(data, outputPath) - sc.stop() - } -} diff --git a/ml/src/main/scala/spark/ml/Updater.scala b/ml/src/main/scala/spark/ml/Updater.scala deleted file mode 100644 index 3952008e28..0000000000 --- a/ml/src/main/scala/spark/ml/Updater.scala +++ /dev/null @@ -1,27 +0,0 @@ -package spark.ml - -import org.jblas.DoubleMatrix - -abstract class Updater extends Serializable { - /** - * Compute an updated value for weights given the gradient, stepSize and iteration number. - * - * @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 - * - * @return weightsNew - Column matrix containing updated weights - * @return reg_val - regularization value - */ - def compute(weightsOlds: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): - (DoubleMatrix, Double) -} - -class SimpleUpdater extends Updater { - override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, - stepSize: Double, iter: Int): (DoubleMatrix, Double) = { - val normGradient = gradient.mul(stepSize / math.sqrt(iter)) - (weightsOld.sub(normGradient), 0) - } -} diff --git a/ml/src/main/scala/spark/ml/gradient/Gradient.scala b/ml/src/main/scala/spark/ml/gradient/Gradient.scala new file mode 100644 index 0000000000..50795c01a2 --- /dev/null +++ b/ml/src/main/scala/spark/ml/gradient/Gradient.scala @@ -0,0 +1,33 @@ +package spark.ml.gradient + +import org.jblas.DoubleMatrix + +abstract class Gradient extends Serializable { + /** + * Compute the gradient for a given row of data. + * + * @param data - One row of data. Row matrix of size 1xn where n is the number of features. + * @param label - Label for this data item. + * @param weights - Column matrix containing weights for every feature. + */ + def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): + (DoubleMatrix, Double) +} + +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) + } +} diff --git a/ml/src/main/scala/spark/ml/gradient/GradientDescent.scala b/ml/src/main/scala/spark/ml/gradient/GradientDescent.scala new file mode 100644 index 0000000000..8cd2a69a3a --- /dev/null +++ b/ml/src/main/scala/spark/ml/gradient/GradientDescent.scala @@ -0,0 +1,62 @@ +package spark.ml.gradient + +import spark.{Logging, RDD, SparkContext} +import spark.SparkContext._ + +import org.jblas.DoubleMatrix + +import scala.collection.mutable.ArrayBuffer + + +object GradientDescent { + + /** + * Run gradient descent in parallel using mini batches. + * Based on Matlab code written by John Duchi. + * + * @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 numIters - number of iterations that SGD should be run. + * @param miniBatchFraction - fraction of the input data set that should be used for + * one iteration of SGD. Default value 1.0. + * + * @return weights - Column matrix containing weights for every feature. + * @return lossHistory - Array containing the loss computed for every iteration. + */ + def runMiniBatchSGD( + data: RDD[(Double, Array[Double])], + gradient: Gradient, + updater: Updater, + stepSize: Double, + numIters: Int, + miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { + + val lossHistory = new ArrayBuffer[Double](numIters) + + val nfeatures: Int = data.take(1)(0)._2.length + val nexamples: Long = data.count() + val miniBatchSize = nexamples * miniBatchFraction + + // Initialize weights as a column matrix + var weights = DoubleMatrix.ones(nfeatures) + var reg_val = 0.0 + + for (i <- 1 to numIters) { + val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42+i).map { + case (y, features) => + val featuresRow = new DoubleMatrix(features.length, 1, features:_*) + val (grad, loss) = gradient.compute(featuresRow, y, weights) + (grad, loss) + }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) + + lossHistory.append(lossSum / miniBatchSize + reg_val) + val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i) + weights = update._1 + reg_val = update._2 + } + + (weights, lossHistory.toArray) + } +} diff --git a/ml/src/main/scala/spark/ml/gradient/Updater.scala b/ml/src/main/scala/spark/ml/gradient/Updater.scala new file mode 100644 index 0000000000..27f03897b0 --- /dev/null +++ b/ml/src/main/scala/spark/ml/gradient/Updater.scala @@ -0,0 +1,27 @@ +package spark.ml.gradient + +import org.jblas.DoubleMatrix + +abstract class Updater extends Serializable { + /** + * Compute an updated value for weights given the gradient, stepSize and iteration number. + * + * @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 + * + * @return weightsNew - Column matrix containing updated weights + * @return reg_val - regularization value + */ + def compute(weightsOlds: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): + (DoubleMatrix, Double) +} + +class SimpleUpdater extends Updater { + override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, + stepSize: Double, iter: Int): (DoubleMatrix, Double) = { + val normGradient = gradient.mul(stepSize / math.sqrt(iter)) + (weightsOld.sub(normGradient), 0) + } +} diff --git a/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala b/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala new file mode 100644 index 0000000000..1508f6934a --- /dev/null +++ b/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala @@ -0,0 +1,100 @@ +package spark.ml.regression + +import spark.{Logging, RDD, SparkContext} +import spark.ml.gradient._ +import spark.ml.util.MLUtils + +import org.jblas.DoubleMatrix + +/** + * Logistic Regression using Stochastic Gradient Descent. + * Based on Matlab code written by John Duchi. + */ +class LogisticRegressionModel( + val weights: DoubleMatrix, + val intercept: Double, + val losses: Array[Double]) extends RegressionModel { + + override def predict(testData: spark.RDD[Array[Double]]) = { + testData.map { x => + val margin = new DoubleMatrix(1, x.length, x:_*).mmul(this.weights).get(0) + this.intercept + 1.0/ (1.0 + math.exp(margin * -1)) + } + } + + override def predict(testData: Array[Double]): Double = { + val dataMat = new DoubleMatrix(1, testData.length, testData:_*) + val margin = dataMat.mmul(this.weights).get(0) + this.intercept + 1.0/ (1.0 + math.exp(margin * -1)) + } +} + +class LogisticRegression private (var stepSize: Double, var miniBatchFraction: Double, + var numIters: Int) + extends Logging { + + /** + * Construct a LogisticRegression object with default parameters + */ + def this() = this(1.0, 1.0, 100) + + /** + * Set the step size per-iteration of SGD. Default 1.0. + */ + def setStepSize(step: Double) = { + this.stepSize = step + this + } + + /** + * Set fraction of data to be used for each SGD iteration. Default 1.0. + */ + def setMiniBatchFraction(fraction: Double) = { + this.miniBatchFraction = fraction + this + } + + /** + * Set the number of iterations for SGD. Default 100. + */ + def setNumIterations(iters: Int) = { + this.numIters = iters + this + } + + def train(input: RDD[(Double, Array[Double])]): LogisticRegressionModel = { + // Add a extra variable consisting of all 1.0's for the intercept. + val data = input.map { case (y, features) => + (y, Array(1.0, features:_*)) + } + + val (weights, losses) = GradientDescent.runMiniBatchSGD( + data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, miniBatchFraction) + + val weightsScaled = weights.getRange(1, weights.length) + val intercept = weights.get(0) + + val model = new LogisticRegressionModel(weightsScaled, intercept, losses) + + logInfo("Final model weights " + model.weights) + logInfo("Final model intercept " + model.intercept) + logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) + model + } +} + +object LogisticRegression { + + 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.loadData(sc, args(1)) + val lr = new LogisticRegression().setStepSize(args(2).toDouble) + .setNumIterations(args(3).toInt) + val model = lr.train(data) + sc.stop() + } +} diff --git a/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala b/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala new file mode 100644 index 0000000000..1617eac205 --- /dev/null +++ b/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala @@ -0,0 +1,41 @@ +package spark.ml.regression + +import spark.{RDD, SparkContext} +import spark.ml.util.MLUtils + +import org.apache.commons.math3.distribution.NormalDistribution +import org.jblas.DoubleMatrix + +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 NormalDistribution(0, 1) + rnd.reseedRandomGenerator(42 + idx) + + val y = if (idx % 2 == 0) 0 else 1 + val x = Array.fill[Double](nfeatures) { + rnd.sample() + (y * eps) + } + (y, x) + } + + MLUtils.saveData(data, outputPath) + sc.stop() + } +} diff --git a/ml/src/main/scala/spark/ml/regression/Regression.scala b/ml/src/main/scala/spark/ml/regression/Regression.scala new file mode 100644 index 0000000000..4a20f513b7 --- /dev/null +++ b/ml/src/main/scala/spark/ml/regression/Regression.scala @@ -0,0 +1,21 @@ +package spark.ml.regression + +import spark.RDD + +trait RegressionModel { + /** + * 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/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala b/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala new file mode 100644 index 0000000000..16d146ac27 --- /dev/null +++ b/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala @@ -0,0 +1,153 @@ +package spark.ml.regression + +import spark.{Logging, RDD, SparkContext} +import spark.SparkContext._ +import spark.ml.util.MLUtils + +import org.jblas.DoubleMatrix +import org.jblas.Solve + +/** + * Ridge Regression from Joseph Gonzalez's implementation in MLBase + */ +class RidgeRegressionModel( + val weights: DoubleMatrix, + val intercept: Double, + val lambdaOpt: Double, + val lambdas: List[(Double, Double, DoubleMatrix)]) + extends RegressionModel { + + override def predict(testData: RDD[Array[Double]]): RDD[Double] = { + testData.map { x => + (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept + } + } + + override def predict(testData: Array[Double]): Double = { + (new DoubleMatrix(1, testData.length, testData:_*).mmul(this.weights)).get(0) + this.intercept + } +} + +class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) + extends Logging { + + def this() = this(0.0, 100.0) + + /** + * Set the lower bound on binary search for lambda's. Default is 0. + */ + def setLowLambda(low: Double) = { + this.lambdaLow = low + this + } + + /** + * Set the upper bound on binary search for lambda's. Default is 100.0. + */ + def setHighLambda(hi: Double) = { + this.lambdaHigh = hi + this + } + + def train(input: RDD[(Double, Array[Double])]): RidgeRegressionModel = { + val nfeatures: Int = input.take(1)(0)._2.length + val nexamples: Long = input.count() + + val (yMean, xColMean, xColSd) = MLUtils.computeStats(input, nfeatures, nexamples) + + val data = input.map { case(y, features) => + val yNormalized = y - yMean + val featuresMat = new DoubleMatrix(nfeatures, 1, features:_*) + val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd) + (yNormalized, featuresNormalized.toArray) + } + + // Compute XtX - Size of XtX is nfeatures by nfeatures + val XtX: DoubleMatrix = data.map { case (y, features) => + val x = new DoubleMatrix(1, features.length, features:_*) + x.transpose().mmul(x) + }.reduce(_.addi(_)) + + // Compute Xt*y - Size of Xty is nfeatures by 1 + val Xty: DoubleMatrix = data.map { case (y, features) => + new DoubleMatrix(features.length, 1, features:_*).mul(y) + }.reduce(_.addi(_)) + + // Define a function to compute the leave one out cross validation error + // for a single example + def crossValidate(lambda: Double): (Double, Double, DoubleMatrix) = { + // Compute the MLE ridge regression parameter value + + // Ridge Regression parameter = inv(XtX + \lambda*I) * Xty + val XtXlambda = DoubleMatrix.eye(nfeatures).muli(lambda).addi(XtX) + val w = Solve.solveSymmetric(XtXlambda, Xty) + + val invXtX = Solve.solveSymmetric(XtXlambda, DoubleMatrix.eye(nfeatures)) + + // compute the generalized cross validation score + val cverror = data.map { + case (y, features) => + val x = new DoubleMatrix(features.length, 1, features:_*) + val yhat = w.transpose().mmul(x).get(0) + val H_ii = x.transpose().mmul(invXtX).mmul(x).get(0) + val residual = (y - yhat) / (1.0 - H_ii) + residual * residual + }.reduce(_ + _) / nexamples + + (lambda, cverror, w) + } + + // Binary search for the best assignment to lambda. + def binSearch(low: Double, high: Double): List[(Double, Double, DoubleMatrix)] = { + val mid = (high - low) / 2 + low + val lowValue = crossValidate((mid - low) / 2 + low) + val highValue = crossValidate((high - mid) / 2 + mid) + val (newLow, newHigh) = if (lowValue._2 < highValue._2) { + (low, mid + (high-low)/4) + } else { + (mid - (high-low)/4, high) + } + if (newHigh - newLow > 1.0E-7) { + // :: is list prepend in Scala. + lowValue :: highValue :: binSearch(newLow, newHigh) + } else { + List(lowValue, highValue) + } + } + + // Actually compute the best lambda + val lambdas = binSearch(lambdaLow, lambdaHigh).sortBy(_._1) + + // Find the best parameter set by taking the lowest cverror. + val (lambdaOpt, cverror, weights) = lambdas.reduce((a, b) => if (a._2 < b._2) a else b) + + // Return the model which contains the solution + val weightsScaled = weights.div(xColSd) + val intercept = yMean - (weights.transpose().mmul(xColMean.div(xColSd)).get(0)) + val model = new RidgeRegressionModel(weightsScaled, intercept, lambdaOpt, lambdas) + + logInfo("RidgeRegression: optimal lambda " + model.lambdaOpt) + logInfo("RidgeRegression: optimal weights " + model.weights) + logInfo("RidgeRegression: optimal intercept " + model.intercept) + logInfo("RidgeRegression: cross-validation error " + cverror) + + model + } +} + +object RidgeRegression { + + def main(args: Array[String]) { + if (args.length != 2) { + println("Usage: RidgeRegression ") + System.exit(1) + } + val sc = new SparkContext(args(0), "RidgeRegression") + val data = MLUtils.loadData(sc, args(1)) + val ridgeReg = new RidgeRegression().setLowLambda(0) + .setHighLambda(1000) + + val model = ridgeReg.train(data) + sc.stop() + } +} diff --git a/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala b/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala new file mode 100644 index 0000000000..ac7f1e7320 --- /dev/null +++ b/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala @@ -0,0 +1,55 @@ +package spark.ml.regression + +import spark.{RDD, SparkContext} +import spark.ml.util.MLUtils + +import org.apache.commons.math3.distribution.NormalDistribution +import org.jblas.DoubleMatrix + + +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 NormalDistribution(0, eps) + rnd.reseedRandomGenerator(42 + p) + + val normalValues = Array.fill[Double](examplesInPartition)(rnd.sample()) + val yObs = new DoubleMatrix(normalValues).addi(y) + + Iterator.tabulate(examplesInPartition) { i => + (yObs.get(i, 0), X.getRow(i).toArray) + } + } + + MLUtils.saveData(data, outputPath) + sc.stop() + } +} diff --git a/ml/src/main/scala/spark/ml/util/MLUtils.scala b/ml/src/main/scala/spark/ml/util/MLUtils.scala new file mode 100644 index 0000000000..76e1bbd36d --- /dev/null +++ b/ml/src/main/scala/spark/ml/util/MLUtils.scala @@ -0,0 +1,79 @@ +package spark.ml.util + +import spark.{RDD, SparkContext} +import spark.SparkContext._ + +import org.jblas.DoubleMatrix + +/** + * Helper methods to load and save data + * Data format: + * , ... + * where , are feature values in Double and is the corresponding label as Double. + */ +object MLUtils { + + /** + * @param sc SparkContext + * @param dir Directory to the input data files. + * @return An RDD of tuples. For each tuple, the first element is the label, and the second + * element represents the feature values (an array of Double). + */ + def loadData(sc: SparkContext, dir: String): RDD[(Double, Array[Double])] = { + sc.textFile(dir).map { line => + val parts = line.split(",") + val label = parts(0).toDouble + val features = parts(1).trim().split(" ").map(_.toDouble) + (label, features) + } + } + + def saveData(data: RDD[(Double, Array[Double])], dir: String) { + val dataStr = data.map(x => x._1 + "," + x._2.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[(Double, Array[Double])], nfeatures: Int, nexamples: Long): + (Double, DoubleMatrix, DoubleMatrix) = { + val yMean: Double = data.map { case (y, features) => y }.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 { case(y, features) => + val nCols = features.length + // Traverse over every column and emit (col, value, value^2) + Iterator.tabulate(nCols) { i => + (i, (features(i), features(i)*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) + } +} diff --git a/ml/src/test/scala/spark/ml/LogisticRegressionSuite.scala b/ml/src/test/scala/spark/ml/LogisticRegressionSuite.scala deleted file mode 100644 index d66c2c8b35..0000000000 --- a/ml/src/test/scala/spark/ml/LogisticRegressionSuite.scala +++ /dev/null @@ -1,54 +0,0 @@ -package spark.ml - -import spark.SparkContext -import spark.SparkContext._ -import spark.Logging - -import org.apache.commons.math3.distribution.NormalDistribution -import org.scalatest.FunSuite - -class LogisticRegressionSuite extends FunSuite with Logging { - - // Test if we can correctly learn A, B where Y = logistic(A + B*X) - test("logistic regression") { - val nPoints = 10000 - val rnd = new NormalDistribution(0, 1) - rnd.reseedRandomGenerator(42) - - val sc = new SparkContext("local", "test") - val x1 = Array.fill[Double](nPoints)(rnd.sample()) - - val A = 2.0 - val B = -1.5 - - // 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(100) - // y <- as.numeric(y > 0) - val y = (0 until nPoints).map { i => - val yVal = A + B * x1(i) + rLogis(i) - if (yVal > 0) 1.0 else 0.0 - } - - val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i)))).toArray - - val testRDD = sc.parallelize(testData, 2) - testRDD.cache() - val lr = new LogisticRegression().setStepSize(1.0) - .setNumIterations(1000) - - val model = lr.train(testRDD) - - assert(model.weights.get(0) >= -1.60 && model.weights.get(0) <= -1.40) - assert(model.intercept >= 1.9 && model.intercept <= 2.1) - - sc.stop() - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port") - } -} diff --git a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala b/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala deleted file mode 100644 index 07da5081b3..0000000000 --- a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala +++ /dev/null @@ -1,45 +0,0 @@ -package spark.ml - -import spark.SparkContext -import spark.SparkContext._ - -import org.apache.commons.math3.distribution.NormalDistribution -import org.scalatest.FunSuite - -class RidgeRegressionSuite extends FunSuite { - - // Test if we can correctly learn Y = 3 + X1 + X2 when - // X1 and X2 are collinear. - test("multi-collinear variables") { - val rnd = new NormalDistribution(0, 1) - rnd.reseedRandomGenerator(43) - val sc = new SparkContext("local", "test") - val x1 = Array.fill[Double](20)(rnd.sample()) - - // Pick a mean close to mean of x1 - val rnd1 = new NormalDistribution(0.1, 0.01) - rnd1.reseedRandomGenerator(42) - val x2 = Array.fill[Double](20)(rnd1.sample()) - - val xMat = (0 until 20).map(i => Array(x1(i), x2(i))).toArray - - val y = xMat.map(i => 3 + i(0) + i(1)) - val testData = (0 until 20).map(i => (y(i), xMat(i))).toArray - - val testRDD = sc.parallelize(testData, 2) - testRDD.cache() - val ridgeReg = new RidgeRegression().setLowLambda(0) - .setHighLambda(10) - - val model = ridgeReg.train(testRDD) - - assert(model.intercept >= 2.9 && model.intercept <= 3.1) - assert(model.weights.length === 2) - assert(model.weights.get(0) >= 0.9 && model.weights.get(0) <= 1.1) - assert(model.weights.get(1) >= 0.9 && model.weights.get(1) <= 1.1) - - sc.stop() - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port") - } -} diff --git a/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala b/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala new file mode 100644 index 0000000000..ce388ecc26 --- /dev/null +++ b/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala @@ -0,0 +1,54 @@ +package spark.ml.regression + +import spark.SparkContext +import spark.SparkContext._ +import spark.Logging + +import org.apache.commons.math3.distribution.NormalDistribution +import org.scalatest.FunSuite + +class LogisticRegressionSuite extends FunSuite with Logging { + + // Test if we can correctly learn A, B where Y = logistic(A + B*X) + test("logistic regression") { + val nPoints = 10000 + val rnd = new NormalDistribution(0, 1) + rnd.reseedRandomGenerator(42) + + val sc = new SparkContext("local", "test") + val x1 = Array.fill[Double](nPoints)(rnd.sample()) + + val A = 2.0 + val B = -1.5 + + // 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(100) + // y <- as.numeric(y > 0) + val y = (0 until nPoints).map { i => + val yVal = A + B * x1(i) + rLogis(i) + if (yVal > 0) 1.0 else 0.0 + } + + val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i)))).toArray + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + val lr = new LogisticRegression().setStepSize(1.0) + .setNumIterations(1000) + + val model = lr.train(testRDD) + + assert(model.weights.get(0) >= -1.60 && model.weights.get(0) <= -1.40) + assert(model.intercept >= 1.9 && model.intercept <= 2.1) + + sc.stop() + // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown + System.clearProperty("spark.driver.port") + } +} diff --git a/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala b/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala new file mode 100644 index 0000000000..6d5f13d6f5 --- /dev/null +++ b/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala @@ -0,0 +1,45 @@ +package spark.ml.regression + +import spark.SparkContext +import spark.SparkContext._ + +import org.apache.commons.math3.distribution.NormalDistribution +import org.scalatest.FunSuite + +class RidgeRegressionSuite extends FunSuite { + + // Test if we can correctly learn Y = 3 + X1 + X2 when + // X1 and X2 are collinear. + test("multi-collinear variables") { + val rnd = new NormalDistribution(0, 1) + rnd.reseedRandomGenerator(43) + val sc = new SparkContext("local", "test") + val x1 = Array.fill[Double](20)(rnd.sample()) + + // Pick a mean close to mean of x1 + val rnd1 = new NormalDistribution(0.1, 0.01) + rnd1.reseedRandomGenerator(42) + val x2 = Array.fill[Double](20)(rnd1.sample()) + + val xMat = (0 until 20).map(i => Array(x1(i), x2(i))).toArray + + val y = xMat.map(i => 3 + i(0) + i(1)) + val testData = (0 until 20).map(i => (y(i), xMat(i))).toArray + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + val ridgeReg = new RidgeRegression().setLowLambda(0) + .setHighLambda(10) + + val model = ridgeReg.train(testRDD) + + assert(model.intercept >= 2.9 && model.intercept <= 3.1) + assert(model.weights.length === 2) + assert(model.weights.get(0) >= 0.9 && model.weights.get(0) <= 1.1) + assert(model.weights.get(1) >= 0.9 && model.weights.get(1) <= 1.1) + + sc.stop() + // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown + System.clearProperty("spark.driver.port") + } +} -- cgit v1.2.3 From 05be233ce2716fe57cf44433d52734ded29e3506 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 21 Jun 2013 15:58:01 +0200 Subject: Removed dependency on Apache Commons Math --- .../ml/regression/LogisticRegressionGenerator.scala | 12 ++++++------ .../spark/ml/regression/RidgeRegressionGenerator.scala | 12 ++++++------ .../spark/ml/regression/LogisticRegressionSuite.scala | 11 ++++++----- .../scala/spark/ml/regression/RidgeRegressionSuite.scala | 16 ++++++++-------- project/SparkBuild.scala | 3 +-- 5 files changed, 27 insertions(+), 27 deletions(-) diff --git a/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala b/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala index 1617eac205..6d37aad047 100644 --- a/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala +++ b/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala @@ -1,11 +1,12 @@ package spark.ml.regression -import spark.{RDD, SparkContext} -import spark.ml.util.MLUtils +import scala.util.Random -import org.apache.commons.math3.distribution.NormalDistribution import org.jblas.DoubleMatrix +import spark.{RDD, SparkContext} +import spark.ml.util.MLUtils + object LogisticRegressionGenerator { def main(args: Array[String]) { @@ -25,12 +26,11 @@ object LogisticRegressionGenerator { val sc = new SparkContext(sparkMaster, "LogisticRegressionGenerator") val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => - val rnd = new NormalDistribution(0, 1) - rnd.reseedRandomGenerator(42 + idx) + val rnd = new Random(42 + idx) val y = if (idx % 2 == 0) 0 else 1 val x = Array.fill[Double](nfeatures) { - rnd.sample() + (y * eps) + rnd.nextGaussian() + (y * eps) } (y, x) } diff --git a/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala b/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala index ac7f1e7320..75854fe1de 100644 --- a/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala +++ b/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala @@ -1,11 +1,12 @@ package spark.ml.regression -import spark.{RDD, SparkContext} -import spark.ml.util.MLUtils +import scala.util.Random -import org.apache.commons.math3.distribution.NormalDistribution import org.jblas.DoubleMatrix +import spark.{RDD, SparkContext} +import spark.ml.util.MLUtils + object RidgeRegressionGenerator { @@ -38,10 +39,9 @@ object RidgeRegressionGenerator { val X = DoubleMatrix.rand(examplesInPartition, nfeatures) val y = X.mmul(w) - val rnd = new NormalDistribution(0, eps) - rnd.reseedRandomGenerator(42 + p) + val rnd = new Random(42 + p) - val normalValues = Array.fill[Double](examplesInPartition)(rnd.sample()) + val normalValues = Array.fill[Double](examplesInPartition)(rnd.nextGaussian() * eps) val yObs = new DoubleMatrix(normalValues).addi(y) Iterator.tabulate(examplesInPartition) { i => diff --git a/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala b/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala index ce388ecc26..53d9789979 100644 --- a/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala @@ -1,22 +1,23 @@ package spark.ml.regression +import scala.util.Random + +import org.scalatest.FunSuite + import spark.SparkContext import spark.SparkContext._ import spark.Logging -import org.apache.commons.math3.distribution.NormalDistribution -import org.scalatest.FunSuite class LogisticRegressionSuite extends FunSuite with Logging { // Test if we can correctly learn A, B where Y = logistic(A + B*X) test("logistic regression") { val nPoints = 10000 - val rnd = new NormalDistribution(0, 1) - rnd.reseedRandomGenerator(42) + val rnd = new Random(42) val sc = new SparkContext("local", "test") - val x1 = Array.fill[Double](nPoints)(rnd.sample()) + val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) val A = 2.0 val B = -1.5 diff --git a/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala b/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala index 6d5f13d6f5..795cda1379 100644 --- a/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala @@ -1,25 +1,25 @@ package spark.ml.regression +import scala.util.Random + +import org.scalatest.FunSuite + import spark.SparkContext import spark.SparkContext._ -import org.apache.commons.math3.distribution.NormalDistribution -import org.scalatest.FunSuite class RidgeRegressionSuite extends FunSuite { // Test if we can correctly learn Y = 3 + X1 + X2 when // X1 and X2 are collinear. test("multi-collinear variables") { - val rnd = new NormalDistribution(0, 1) - rnd.reseedRandomGenerator(43) + val rnd = new Random(43) val sc = new SparkContext("local", "test") - val x1 = Array.fill[Double](20)(rnd.sample()) + val x1 = Array.fill[Double](20)(rnd.nextGaussian()) // Pick a mean close to mean of x1 - val rnd1 = new NormalDistribution(0.1, 0.01) - rnd1.reseedRandomGenerator(42) - val x2 = Array.fill[Double](20)(rnd1.sample()) + val rnd1 = new Random(42) //new NormalDistribution(0.1, 0.01) + val x2 = Array.fill[Double](20)(0.1 + rnd1.nextGaussian() * 0.01) val xMat = (0 until 20).map(i => Array(x1(i), x2(i))).toArray diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index aa877ad4a7..5dbb5d4a65 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -224,8 +224,7 @@ object SparkBuild extends Build { def mlSettings = sharedSettings ++ Seq( name := "spark-ml", libraryDependencies ++= Seq( - "org.jblas" % "jblas" % "1.2.3", - "org.apache.commons" % "commons-math3" % "3.2" + "org.jblas" % "jblas" % "1.2.3" ) ) -- cgit v1.2.3 From d903b3887f13b5ef500cd31227b4563d372e6cd8 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 21 Jun 2013 19:09:17 +0200 Subject: Initial implementation of Alternating Least Squares. Includes unit tests and sample data to run on. --- ml/data/als/test.data | 16 + ml/src/main/scala/spark/ml/als/ALS.scala | 386 +++++++++++++++++++++ .../spark/ml/als/MatrixFactorizationModel.scala | 22 ++ ml/src/test/scala/spark/ml/als/ALSSuite.scala | 80 +++++ 4 files changed, 504 insertions(+) create mode 100644 ml/data/als/test.data create mode 100644 ml/src/main/scala/spark/ml/als/ALS.scala create mode 100644 ml/src/main/scala/spark/ml/als/MatrixFactorizationModel.scala create mode 100644 ml/src/test/scala/spark/ml/als/ALSSuite.scala diff --git a/ml/data/als/test.data b/ml/data/als/test.data new file mode 100644 index 0000000000..e476cc23e0 --- /dev/null +++ b/ml/data/als/test.data @@ -0,0 +1,16 @@ +1,1,5.0 +1,2,1.0 +1,3,5.0 +1,4,1.0 +2,1,5.0 +2,2,1.0 +2,3,5.0 +2,4,1.0 +3,1,1.0 +3,2,5.0 +3,3,1.0 +3,4,5.0 +4,1,1.0 +4,2,5.0 +4,3,1.0 +4,4,5.0 diff --git a/ml/src/main/scala/spark/ml/als/ALS.scala b/ml/src/main/scala/spark/ml/als/ALS.scala new file mode 100644 index 0000000000..84339780e2 --- /dev/null +++ b/ml/src/main/scala/spark/ml/als/ALS.scala @@ -0,0 +1,386 @@ +package spark.ml.als + +import scala.collection.mutable.{ArrayBuffer, BitSet} +import scala.util.Random + +import spark.{HashPartitioner, Partitioner, SparkContext, RDD} +import spark.storage.StorageLevel +import spark.SparkContext._ + +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[als] 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[als] case class InLinkBlock( + elementIds: Array[Int], ratingsForBlock: Array[Array[(Array[Int], Array[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. + */ + def setBlocks(numBlocks: Int): ALS = { + this.numBlocks = numBlocks + this + } + + /** Set the rank of the feature matrices computed (number of features). */ + def setRank(rank: Int): ALS = { + this.rank = rank + this + } + + /** Set the total number of iterations */ + def setIterations(iterations: Int): ALS = { + this.iterations = iterations + this + } + + /** Set the regularization parameter, lambda */ + def setLambda(lambda: Double): ALS = { + this.lambda = lambda + this + } + + /** + * Run ALS with the configured parmeters on an input RDD of (user, product, rating) triples. + * Returns a MatrixFactorizationModel with feature vectors for each user and product. + */ + def train(ratings: RDD[(Int, Int, Double)]): MatrixFactorizationModel = { + val numBlocks = if (this.numBlocks == -1) { + math.max(ratings.context.defaultParallelism, ratings.partitions.size) + } else { + this.numBlocks + } + + val partitioner = new HashPartitioner(numBlocks) + + val ratingsByUserBlock = ratings.map{ case (u, p, r) => (u % numBlocks, (u, p, r)) } + val ratingsByProductBlock = ratings.map{ case (u, p, r) => (p % numBlocks, (p, u, r)) } + + val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock) + val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) + + // Initialize user and product factors deterministically. + var users = userOutLinks.mapValues(_.elementIds.map(u => makeInitialFactor(rank, u))) + var products = productOutLinks.mapValues(_.elementIds.map(p => makeInitialFactor(rank, -p))) + + 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[(Int, Int, Double)]): OutLinkBlock = { + val userIds = ratings.map(_._1).distinct.sorted + val numUsers = userIds.length + val userIdToPos = userIds.zipWithIndex.toMap + val shouldSend = Array.fill(numUsers)(new BitSet(numBlocks)) + for ((u, p, r) <- ratings) { + shouldSend(userIdToPos(u))(p % 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[(Int, Int, Double)]): InLinkBlock = { + val userIds = ratings.map(_._1).distinct.sorted + val numUsers = userIds.length + val userIdToPos = userIds.zipWithIndex.toMap + val ratingsForBlock = new Array[Array[(Array[Int], Array[Double])]](numBlocks) + for (productBlock <- 0 until numBlocks) { + val ratingsInBlock = ratings.filter(t => t._2 % numBlocks == productBlock) + val ratingsByProduct = ratingsInBlock.groupBy(_._2) // (p, Seq[(u, p, r)]) + .toArray + .sortBy(_._1) + .map{case (p, rs) => (rs.map(t => userIdToPos(t._1)), rs.map(_._3))} + ratingsForBlock(productBlock) = ratingsByProduct + } + 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, (Int, Int, Double))]) + : (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 + Iterator((blockId, (makeInLinkBlock(numBlocks, ratings), makeOutLinkBlock(numBlocks, ratings)))) + }, true) + links.persist() + (links.mapValues(_._1), links.mapValues(_._2)) + } + + /** + * Make a random initial factor vector with the given seed. + * TODO: Initialize things using mapPartitionsWithIndex to make it faster? + */ + private def makeInitialFactor(rank: Int, seed: Int): Array[Double] = { + val rand = new Random(seed) + 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[(Int, Int, Double)], + rank: Int, + iterations: Int, + lambda: Double, + blocks: Int) + : MatrixFactorizationModel = + { + new ALS(blocks, rank, iterations, lambda).train(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[(Int, Int, Double)], 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[(Int, Int, Double)], rank: Int, iterations: Int) + : MatrixFactorizationModel = + { + train(ratings, rank, iterations, 0.01, -1) + } + + def main(args: Array[String]) { + if (args.length != 5) { + 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 sc = new SparkContext(master, "ALS") + val ratings = sc.textFile(ratingsFile).map { line => + val fields = line.split(',') + (fields(0).toInt, fields(1).toInt, fields(2).toDouble) + } + val model = ALS.train(ratings, rank, iters) + 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/ml/src/main/scala/spark/ml/als/MatrixFactorizationModel.scala b/ml/src/main/scala/spark/ml/als/MatrixFactorizationModel.scala new file mode 100644 index 0000000000..52a8984bc7 --- /dev/null +++ b/ml/src/main/scala/spark/ml/als/MatrixFactorizationModel.scala @@ -0,0 +1,22 @@ +package spark.ml.als + +import spark.RDD +import spark.SparkContext._ + +import org.jblas._ + +class MatrixFactorizationModel( + val rank: Int, + val userFeatures: RDD[(Int, Array[Double])], + val productFeatures: RDD[(Int, Array[Double])]) +{ + /** 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/ml/src/test/scala/spark/ml/als/ALSSuite.scala b/ml/src/test/scala/spark/ml/als/ALSSuite.scala new file mode 100644 index 0000000000..70f350857f --- /dev/null +++ b/ml/src/test/scala/spark/ml/als/ALSSuite.scala @@ -0,0 +1,80 @@ +package spark.ml.als + +import scala.util.Random + +import org.scalatest.FunSuite + +import spark.SparkContext +import spark.SparkContext._ +import spark.Logging + +import org.jblas._ + + +class ALSSuite extends FunSuite with Logging { + + test("rank-1 matrices") { + testALS(10, 20, 1, 5, 0.5, 0.1) + } + + test("rank-2 matrices") { + testALS(20, 30, 2, 10, 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 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 (u, p, trueRatings.get(u, p)) + } + + val sc = new SparkContext("local", "test") + + 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)) + } + } + + sc.stop() + // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown + System.clearProperty("spark.driver.port") + } +} + -- cgit v1.2.3 From 6f0ebb2db24272390ebd481e50a08336e37d2b1d Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 21 Jun 2013 19:13:22 +0200 Subject: Remove unused import --- ml/src/main/scala/spark/ml/als/ALS.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/ml/src/main/scala/spark/ml/als/ALS.scala b/ml/src/main/scala/spark/ml/als/ALS.scala index 84339780e2..dca5bfc8f1 100644 --- a/ml/src/main/scala/spark/ml/als/ALS.scala +++ b/ml/src/main/scala/spark/ml/als/ALS.scala @@ -4,7 +4,6 @@ import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.util.Random import spark.{HashPartitioner, Partitioner, SparkContext, RDD} -import spark.storage.StorageLevel import spark.SparkContext._ import org.jblas.{DoubleMatrix, SimpleBlas, Solve} -- cgit v1.2.3 From 3c046a6eca182d720b58f5cdfd17b6201664e716 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 21 Jun 2013 19:24:06 +0200 Subject: Some small fixes to ALS. --- ml/src/main/scala/spark/ml/als/ALS.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/ml/src/main/scala/spark/ml/als/ALS.scala b/ml/src/main/scala/spark/ml/als/ALS.scala index dca5bfc8f1..a1ae28a38d 100644 --- a/ml/src/main/scala/spark/ml/als/ALS.scala +++ b/ml/src/main/scala/spark/ml/als/ALS.scala @@ -4,6 +4,7 @@ import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.util.Random import spark.{HashPartitioner, Partitioner, SparkContext, RDD} +import spark.storage.StorageLevel import spark.SparkContext._ import org.jblas.{DoubleMatrix, SimpleBlas, Solve} @@ -52,26 +53,26 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l /** * Set the number of blocks to parallelize the computation into; pass -1 for an auto-configured - * number of blocks. + * 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). */ + /** Set the rank of the feature matrices computed (number of features). Default: 10. */ def setRank(rank: Int): ALS = { this.rank = rank this } - /** Set the total number of iterations */ + /** Set the number of iterations to run. Default: 10. */ def setIterations(iterations: Int): ALS = { this.iterations = iterations this } - /** Set the regularization parameter, lambda */ + /** Set the regularization parameter, lambda. Default: 0.01. */ def setLambda(lambda: Double): ALS = { this.lambda = lambda this @@ -168,7 +169,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l val ratings = elements.map(_._2).toArray Iterator((blockId, (makeInLinkBlock(numBlocks, ratings), makeOutLinkBlock(numBlocks, ratings)))) }, true) - links.persist() + links.persist(StorageLevel.MEMORY_AND_DISK) (links.mapValues(_._1), links.mapValues(_._2)) } -- cgit v1.2.3 From d3ce898b8eec066d69892d29b0c6531790902d78 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 21 Jun 2013 20:00:56 +0200 Subject: Scaffolding and model for K-means --- ml/src/main/scala/spark/ml/clustering/KMeans.scala | 117 +++++++++++++++++++++ .../scala/spark/ml/clustering/KMeansModel.scala | 25 +++++ ml/src/main/scala/spark/ml/util/MLUtils.scala | 15 +++ 3 files changed, 157 insertions(+) create mode 100644 ml/src/main/scala/spark/ml/clustering/KMeans.scala create mode 100644 ml/src/main/scala/spark/ml/clustering/KMeansModel.scala diff --git a/ml/src/main/scala/spark/ml/clustering/KMeans.scala b/ml/src/main/scala/spark/ml/clustering/KMeans.scala new file mode 100644 index 0000000000..fd15da9b43 --- /dev/null +++ b/ml/src/main/scala/spark/ml/clustering/KMeans.scala @@ -0,0 +1,117 @@ +package spark.ml.clustering + +import scala.collection.mutable.ArrayBuffer +import scala.util.Random + +import spark.{SparkContext, RDD} +import spark.SparkContext._ +import spark.Logging + +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 TODO). 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) + extends Serializable with Logging +{ + // Initialization mode names + private val RANDOM = "random" + private val K_MEANS_PARALLEL = "k-means||" + + def this() = this(2, 20, 1, "k-means||", 5) + + /** 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 version of k-means++ discussed + * in (TODO). Default: k-means||. + */ + def setInitializationMode(initializationMode: String): KMeans = { + if (initializationMode != RANDOM && initializationMode != 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 = { + 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 = { + this.initializationSteps = initializationSteps + 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 train(data: RDD[Array[Double]]): KMeansModel = { + // TODO: check whether data is persistent; this needs RDD.storageLevel to be publicly readable + + null + } +} + + +/** + * Top-level methods for calling K-means clustering. + */ +object KMeans { + 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) + .train(data) + } + + def train(data: RDD[Array[Double]], k: Int, maxIterations: Int, runs: Int): KMeansModel = { + train(data, k, maxIterations, runs, "k-means||") + } + + def train(data: RDD[Array[Double]], k: Int, maxIterations: Int): KMeansModel = { + train(data, k, maxIterations, 1, "k-means||") + } +} diff --git a/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala b/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala new file mode 100644 index 0000000000..f2f26c47b8 --- /dev/null +++ b/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala @@ -0,0 +1,25 @@ +package spark.ml.clustering + +import spark.ml.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]]) { + /** 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 = { + var bestDist = Double.PositiveInfinity + var bestIndex = -1 + for (i <- 0 until k) { + val dist = MLUtils.squaredDistance(clusterCenters(i), point) + if (dist < bestDist) { + bestDist = dist + bestIndex = i + } + } + bestIndex + } +} diff --git a/ml/src/main/scala/spark/ml/util/MLUtils.scala b/ml/src/main/scala/spark/ml/util/MLUtils.scala index 76e1bbd36d..48292649c3 100644 --- a/ml/src/main/scala/spark/ml/util/MLUtils.scala +++ b/ml/src/main/scala/spark/ml/util/MLUtils.scala @@ -76,4 +76,19 @@ object MLUtils { (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)) + } + sum + } } -- cgit v1.2.3 From 43dae967d7095d4ebd1ab5919e8907da640e48d3 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 15:16:24 -0400 Subject: Renamed "als" package to "recommendation" --- ml/src/main/scala/spark/ml/als/ALS.scala | 386 --------------------- .../spark/ml/als/MatrixFactorizationModel.scala | 22 -- .../main/scala/spark/ml/recommendation/ALS.scala | 386 +++++++++++++++++++++ .../recommendation/MatrixFactorizationModel.scala | 22 ++ ml/src/test/scala/spark/ml/als/ALSSuite.scala | 80 ----- .../scala/spark/ml/recommendation/ALSSuite.scala | 80 +++++ 6 files changed, 488 insertions(+), 488 deletions(-) delete mode 100644 ml/src/main/scala/spark/ml/als/ALS.scala delete mode 100644 ml/src/main/scala/spark/ml/als/MatrixFactorizationModel.scala create mode 100644 ml/src/main/scala/spark/ml/recommendation/ALS.scala create mode 100644 ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala delete mode 100644 ml/src/test/scala/spark/ml/als/ALSSuite.scala create mode 100644 ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala diff --git a/ml/src/main/scala/spark/ml/als/ALS.scala b/ml/src/main/scala/spark/ml/als/ALS.scala deleted file mode 100644 index a1ae28a38d..0000000000 --- a/ml/src/main/scala/spark/ml/als/ALS.scala +++ /dev/null @@ -1,386 +0,0 @@ -package spark.ml.als - -import scala.collection.mutable.{ArrayBuffer, BitSet} -import scala.util.Random - -import spark.{HashPartitioner, Partitioner, SparkContext, RDD} -import spark.storage.StorageLevel -import spark.SparkContext._ - -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[als] 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[als] case class InLinkBlock( - elementIds: Array[Int], ratingsForBlock: Array[Array[(Array[Int], Array[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 parmeters on an input RDD of (user, product, rating) triples. - * Returns a MatrixFactorizationModel with feature vectors for each user and product. - */ - def train(ratings: RDD[(Int, Int, Double)]): MatrixFactorizationModel = { - val numBlocks = if (this.numBlocks == -1) { - math.max(ratings.context.defaultParallelism, ratings.partitions.size) - } else { - this.numBlocks - } - - val partitioner = new HashPartitioner(numBlocks) - - val ratingsByUserBlock = ratings.map{ case (u, p, r) => (u % numBlocks, (u, p, r)) } - val ratingsByProductBlock = ratings.map{ case (u, p, r) => (p % numBlocks, (p, u, r)) } - - val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock) - val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) - - // Initialize user and product factors deterministically. - var users = userOutLinks.mapValues(_.elementIds.map(u => makeInitialFactor(rank, u))) - var products = productOutLinks.mapValues(_.elementIds.map(p => makeInitialFactor(rank, -p))) - - 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[(Int, Int, Double)]): OutLinkBlock = { - val userIds = ratings.map(_._1).distinct.sorted - val numUsers = userIds.length - val userIdToPos = userIds.zipWithIndex.toMap - val shouldSend = Array.fill(numUsers)(new BitSet(numBlocks)) - for ((u, p, r) <- ratings) { - shouldSend(userIdToPos(u))(p % 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[(Int, Int, Double)]): InLinkBlock = { - val userIds = ratings.map(_._1).distinct.sorted - val numUsers = userIds.length - val userIdToPos = userIds.zipWithIndex.toMap - val ratingsForBlock = new Array[Array[(Array[Int], Array[Double])]](numBlocks) - for (productBlock <- 0 until numBlocks) { - val ratingsInBlock = ratings.filter(t => t._2 % numBlocks == productBlock) - val ratingsByProduct = ratingsInBlock.groupBy(_._2) // (p, Seq[(u, p, r)]) - .toArray - .sortBy(_._1) - .map{case (p, rs) => (rs.map(t => userIdToPos(t._1)), rs.map(_._3))} - ratingsForBlock(productBlock) = ratingsByProduct - } - 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, (Int, Int, Double))]) - : (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 - Iterator((blockId, (makeInLinkBlock(numBlocks, ratings), makeOutLinkBlock(numBlocks, ratings)))) - }, true) - links.persist(StorageLevel.MEMORY_AND_DISK) - (links.mapValues(_._1), links.mapValues(_._2)) - } - - /** - * Make a random initial factor vector with the given seed. - * TODO: Initialize things using mapPartitionsWithIndex to make it faster? - */ - private def makeInitialFactor(rank: Int, seed: Int): Array[Double] = { - val rand = new Random(seed) - 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[(Int, Int, Double)], - rank: Int, - iterations: Int, - lambda: Double, - blocks: Int) - : MatrixFactorizationModel = - { - new ALS(blocks, rank, iterations, lambda).train(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[(Int, Int, Double)], 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[(Int, Int, Double)], rank: Int, iterations: Int) - : MatrixFactorizationModel = - { - train(ratings, rank, iterations, 0.01, -1) - } - - def main(args: Array[String]) { - if (args.length != 5) { - 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 sc = new SparkContext(master, "ALS") - val ratings = sc.textFile(ratingsFile).map { line => - val fields = line.split(',') - (fields(0).toInt, fields(1).toInt, fields(2).toDouble) - } - val model = ALS.train(ratings, rank, iters) - 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/ml/src/main/scala/spark/ml/als/MatrixFactorizationModel.scala b/ml/src/main/scala/spark/ml/als/MatrixFactorizationModel.scala deleted file mode 100644 index 52a8984bc7..0000000000 --- a/ml/src/main/scala/spark/ml/als/MatrixFactorizationModel.scala +++ /dev/null @@ -1,22 +0,0 @@ -package spark.ml.als - -import spark.RDD -import spark.SparkContext._ - -import org.jblas._ - -class MatrixFactorizationModel( - val rank: Int, - val userFeatures: RDD[(Int, Array[Double])], - val productFeatures: RDD[(Int, Array[Double])]) -{ - /** 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/ml/src/main/scala/spark/ml/recommendation/ALS.scala b/ml/src/main/scala/spark/ml/recommendation/ALS.scala new file mode 100644 index 0000000000..6717315fa7 --- /dev/null +++ b/ml/src/main/scala/spark/ml/recommendation/ALS.scala @@ -0,0 +1,386 @@ +package spark.ml.recommendation + +import scala.collection.mutable.{ArrayBuffer, BitSet} +import scala.util.Random + +import spark.{HashPartitioner, Partitioner, SparkContext, RDD} +import spark.storage.StorageLevel +import spark.SparkContext._ + +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])]]) + + +/** + * 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 parmeters on an input RDD of (user, product, rating) triples. + * Returns a MatrixFactorizationModel with feature vectors for each user and product. + */ + def train(ratings: RDD[(Int, Int, Double)]): MatrixFactorizationModel = { + val numBlocks = if (this.numBlocks == -1) { + math.max(ratings.context.defaultParallelism, ratings.partitions.size) + } else { + this.numBlocks + } + + val partitioner = new HashPartitioner(numBlocks) + + val ratingsByUserBlock = ratings.map{ case (u, p, r) => (u % numBlocks, (u, p, r)) } + val ratingsByProductBlock = ratings.map{ case (u, p, r) => (p % numBlocks, (p, u, r)) } + + val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock) + val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) + + // Initialize user and product factors deterministically. + var users = userOutLinks.mapValues(_.elementIds.map(u => makeInitialFactor(rank, u))) + var products = productOutLinks.mapValues(_.elementIds.map(p => makeInitialFactor(rank, -p))) + + 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[(Int, Int, Double)]): OutLinkBlock = { + val userIds = ratings.map(_._1).distinct.sorted + val numUsers = userIds.length + val userIdToPos = userIds.zipWithIndex.toMap + val shouldSend = Array.fill(numUsers)(new BitSet(numBlocks)) + for ((u, p, r) <- ratings) { + shouldSend(userIdToPos(u))(p % 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[(Int, Int, Double)]): InLinkBlock = { + val userIds = ratings.map(_._1).distinct.sorted + val numUsers = userIds.length + val userIdToPos = userIds.zipWithIndex.toMap + val ratingsForBlock = new Array[Array[(Array[Int], Array[Double])]](numBlocks) + for (productBlock <- 0 until numBlocks) { + val ratingsInBlock = ratings.filter(t => t._2 % numBlocks == productBlock) + val ratingsByProduct = ratingsInBlock.groupBy(_._2) // (p, Seq[(u, p, r)]) + .toArray + .sortBy(_._1) + .map{case (p, rs) => (rs.map(t => userIdToPos(t._1)), rs.map(_._3))} + ratingsForBlock(productBlock) = ratingsByProduct + } + 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, (Int, Int, Double))]) + : (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 + Iterator((blockId, (makeInLinkBlock(numBlocks, ratings), makeOutLinkBlock(numBlocks, ratings)))) + }, true) + links.persist(StorageLevel.MEMORY_AND_DISK) + (links.mapValues(_._1), links.mapValues(_._2)) + } + + /** + * Make a random initial factor vector with the given seed. + * TODO: Initialize things using mapPartitionsWithIndex to make it faster? + */ + private def makeInitialFactor(rank: Int, seed: Int): Array[Double] = { + val rand = new Random(seed) + 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[(Int, Int, Double)], + rank: Int, + iterations: Int, + lambda: Double, + blocks: Int) + : MatrixFactorizationModel = + { + new ALS(blocks, rank, iterations, lambda).train(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[(Int, Int, Double)], 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[(Int, Int, Double)], rank: Int, iterations: Int) + : MatrixFactorizationModel = + { + train(ratings, rank, iterations, 0.01, -1) + } + + def main(args: Array[String]) { + if (args.length != 5) { + 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 sc = new SparkContext(master, "ALS") + val ratings = sc.textFile(ratingsFile).map { line => + val fields = line.split(',') + (fields(0).toInt, fields(1).toInt, fields(2).toDouble) + } + val model = ALS.train(ratings, rank, iters) + 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/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala b/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala new file mode 100644 index 0000000000..7bd33a87ee --- /dev/null +++ b/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala @@ -0,0 +1,22 @@ +package spark.ml.recommendation + +import spark.RDD +import spark.SparkContext._ + +import org.jblas._ + +class MatrixFactorizationModel( + val rank: Int, + val userFeatures: RDD[(Int, Array[Double])], + val productFeatures: RDD[(Int, Array[Double])]) +{ + /** 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/ml/src/test/scala/spark/ml/als/ALSSuite.scala b/ml/src/test/scala/spark/ml/als/ALSSuite.scala deleted file mode 100644 index 70f350857f..0000000000 --- a/ml/src/test/scala/spark/ml/als/ALSSuite.scala +++ /dev/null @@ -1,80 +0,0 @@ -package spark.ml.als - -import scala.util.Random - -import org.scalatest.FunSuite - -import spark.SparkContext -import spark.SparkContext._ -import spark.Logging - -import org.jblas._ - - -class ALSSuite extends FunSuite with Logging { - - test("rank-1 matrices") { - testALS(10, 20, 1, 5, 0.5, 0.1) - } - - test("rank-2 matrices") { - testALS(20, 30, 2, 10, 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 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 (u, p, trueRatings.get(u, p)) - } - - val sc = new SparkContext("local", "test") - - 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)) - } - } - - sc.stop() - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port") - } -} - diff --git a/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala b/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala new file mode 100644 index 0000000000..957ab51feb --- /dev/null +++ b/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala @@ -0,0 +1,80 @@ +package spark.ml.recommendation + +import scala.util.Random + +import org.scalatest.FunSuite + +import spark.SparkContext +import spark.SparkContext._ +import spark.Logging + +import org.jblas._ + + +class ALSSuite extends FunSuite with Logging { + + test("rank-1 matrices") { + testALS(10, 20, 1, 5, 0.5, 0.1) + } + + test("rank-2 matrices") { + testALS(20, 30, 2, 10, 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 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 (u, p, trueRatings.get(u, p)) + } + + val sc = new SparkContext("local", "test") + + 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)) + } + } + + sc.stop() + // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown + System.clearProperty("spark.driver.port") + } +} + -- cgit v1.2.3 From 6586c5e28b4ccbb569ca8cf149bd181d0d8e1e33 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 16:52:51 -0400 Subject: Added a SparkContext accessor to RDD --- core/src/main/scala/spark/RDD.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index f336c2ea1e..8e1e18c06c 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -105,6 +105,9 @@ abstract class RDD[T: ClassManifest]( // Methods and fields available on all RDDs // ======================================================================= + /** The SparkContext that created this RDD. */ + def sparkContext: SparkContext = sc + /** A unique ID for this RDD (within its SparkContext). */ val id: Int = sc.newRddId() -- cgit v1.2.3 From 39684eafe3dbfdc0563c95c05e69350fe39c71f9 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 18:08:52 -0400 Subject: Formatting --- ml/src/main/scala/spark/ml/recommendation/ALS.scala | 2 +- .../scala/spark/ml/recommendation/MatrixFactorizationModel.scala | 7 ++++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/ml/src/main/scala/spark/ml/recommendation/ALS.scala b/ml/src/main/scala/spark/ml/recommendation/ALS.scala index 6717315fa7..abf54216a5 100644 --- a/ml/src/main/scala/spark/ml/recommendation/ALS.scala +++ b/ml/src/main/scala/spark/ml/recommendation/ALS.scala @@ -365,7 +365,7 @@ object ALS { def main(args: Array[String]) { if (args.length != 5) { - println("Usage: ALS ") + println("Usage: ALS ") System.exit(1) } val (master, ratingsFile, rank, iters, outputDir) = diff --git a/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala b/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala index 7bd33a87ee..cfdf2ba523 100644 --- a/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala +++ b/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala @@ -6,9 +6,10 @@ import spark.SparkContext._ import org.jblas._ class MatrixFactorizationModel( - val rank: Int, - val userFeatures: RDD[(Int, Array[Double])], - val productFeatures: RDD[(Int, Array[Double])]) + 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 = { -- cgit v1.2.3 From 52f491125ecc26821842fe88786b4b0e7c01499b Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 18:09:19 -0400 Subject: Implementation of k-means and k-means|| --- ml/src/main/scala/spark/ml/clustering/KMeans.scala | 230 +++++++++++++++++++-- .../scala/spark/ml/clustering/KMeansModel.scala | 13 +- .../scala/spark/ml/clustering/LocalKMeans.scala | 88 ++++++++ ml/src/main/scala/spark/ml/util/MLUtils.scala | 1 + 4 files changed, 316 insertions(+), 16 deletions(-) create mode 100644 ml/src/main/scala/spark/ml/clustering/LocalKMeans.scala diff --git a/ml/src/main/scala/spark/ml/clustering/KMeans.scala b/ml/src/main/scala/spark/ml/clustering/KMeans.scala index fd15da9b43..d1786b0812 100644 --- a/ml/src/main/scala/spark/ml/clustering/KMeans.scala +++ b/ml/src/main/scala/spark/ml/clustering/KMeans.scala @@ -6,14 +6,15 @@ import scala.util.Random import spark.{SparkContext, RDD} import spark.SparkContext._ import spark.Logging +import spark.ml.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 TODO). When multiple concurrent runs are requested, they are - * executed together with joint passes over the data for efficiency. + * 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. @@ -23,14 +24,13 @@ class KMeans private ( var maxIterations: Int, var runs: Int, var initializationMode: String, - var initializationSteps: Int) + var initializationSteps: Int, + var epsilon: Double) extends Serializable with Logging { - // Initialization mode names - private val RANDOM = "random" - private val K_MEANS_PARALLEL = "k-means||" + private type ClusterCenters = Array[Array[Double]] - def this() = this(2, 20, 1, "k-means||", 5) + 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 = { @@ -46,11 +46,11 @@ class KMeans private ( /** * Set the initialization algorithm. This can be either "random" to choose random points as - * initial cluster centers, or "k-means||" to use a parallel version of k-means++ discussed - * in (TODO). Default: k-means||. + * 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 != RANDOM && initializationMode != K_MEANS_PARALLEL) { + if (initializationMode != KMeans.RANDOM && initializationMode != KMeans.K_MEANS_PARALLEL) { throw new IllegalArgumentException("Invalid initialization mode: " + initializationMode) } this.initializationMode = initializationMode @@ -63,6 +63,9 @@ class KMeans private ( * 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 } @@ -72,10 +75,22 @@ class KMeans private ( * 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. @@ -83,7 +98,139 @@ class KMeans private ( def train(data: RDD[Array[Double]]): KMeansModel = { // TODO: check whether data is persistent; this needs RDD.storageLevel to be publicly readable - null + val sc = data.sparkContext + + var 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) { + for ((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, 1) + Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k)) + } + + /** + * 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 sample = data.takeSample(true, runs, 1) + 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((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((r, i))).toArray + LocalKMeans.kMeansPlusPlus(r, myCenters, myWeights, k, 30) + } + + finalCenters.toArray } } @@ -92,6 +239,10 @@ class KMeans private ( * 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, @@ -106,12 +257,61 @@ object KMeans { .setInitializationMode(initializationMode) .train(data) } - + def train(data: RDD[Array[Double]], k: Int, maxIterations: Int, runs: Int): KMeansModel = { - train(data, k, maxIterations, runs, "k-means||") + 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||") + 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[ml] 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[ml] 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 sc = new SparkContext(master, "KMeans") + val data = sc.textFile(inputFile).map(line => line.split(' ').map(_.toDouble)) + val model = KMeans.train(data, k, iters) + val cost = model.computeCost(data) + println("Cluster centers:") + for (c <- model.clusterCenters) { + println(" " + c.mkString(" ")) + } + println("Cost: " + cost) + System.exit(0) } } diff --git a/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala b/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala index f2f26c47b8..84bfd0f99a 100644 --- a/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala +++ b/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala @@ -1,11 +1,14 @@ package spark.ml.clustering +import spark.RDD +import spark.SparkContext._ import spark.ml.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]]) { +class KMeansModel(val clusterCenters: Array[Array[Double]]) extends Serializable { /** Total number of clusters. */ def k: Int = clusterCenters.length @@ -22,4 +25,12 @@ class KMeansModel(val clusterCenters: Array[Array[Double]]) { } bestIndex } + + /** + * 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/ml/src/main/scala/spark/ml/clustering/LocalKMeans.scala b/ml/src/main/scala/spark/ml/clustering/LocalKMeans.scala new file mode 100644 index 0000000000..03129ef552 --- /dev/null +++ b/ml/src/main/scala/spark/ml/clustering/LocalKMeans.scala @@ -0,0 +1,88 @@ +package spark.ml.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[ml] 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/ml/src/main/scala/spark/ml/util/MLUtils.scala b/ml/src/main/scala/spark/ml/util/MLUtils.scala index 48292649c3..6efa7c81ad 100644 --- a/ml/src/main/scala/spark/ml/util/MLUtils.scala +++ b/ml/src/main/scala/spark/ml/util/MLUtils.scala @@ -88,6 +88,7 @@ object MLUtils { var sum = 0.0 while (i < v1.length) { sum += (v1(i) - v2(i)) * (v1(i) - v2(i)) + i += 1 } sum } -- cgit v1.2.3 From 496c7548bb3af6581f9bdb53cfa5be600d5849fb Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sat, 22 Jun 2013 13:51:30 -0700 Subject: Change test to use fewer iterations --- ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala b/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala index 53d9789979..6216523a9e 100644 --- a/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala @@ -40,8 +40,8 @@ class LogisticRegressionSuite extends FunSuite with Logging { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val lr = new LogisticRegression().setStepSize(1.0) - .setNumIterations(1000) + val lr = new LogisticRegression().setStepSize(5.0) + .setNumIterations(20) val model = lr.train(testRDD) -- cgit v1.2.3 From cffe3340c5346ab7c21d82cd9c456e9ad17e2e32 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 23:17:15 -0400 Subject: Fix logistic regression test failure and test suite cleanup --- .../scala/spark/ml/recommendation/ALSSuite.scala | 16 ++++++------- .../ml/regression/LogisticRegressionSuite.scala | 28 ++++++++++++---------- .../spark/ml/regression/RidgeRegressionSuite.scala | 14 ++++++----- 3 files changed, 31 insertions(+), 27 deletions(-) diff --git a/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala b/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala index 957ab51feb..c450d5315c 100644 --- a/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala +++ b/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala @@ -2,16 +2,22 @@ package spark.ml.recommendation import scala.util.Random +import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext import spark.SparkContext._ -import spark.Logging import org.jblas._ -class ALSSuite extends FunSuite with Logging { +class ALSSuite extends FunSuite with BeforeAndAfterAll { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } test("rank-1 matrices") { testALS(10, 20, 1, 5, 0.5, 0.1) @@ -49,8 +55,6 @@ class ALSSuite extends FunSuite with Logging { yield (u, p, trueRatings.get(u, p)) } - val sc = new SparkContext("local", "test") - val model = ALS.train(sc.parallelize(sampledRatings), features, iterations) val predictedU = new DoubleMatrix(users, features) @@ -71,10 +75,6 @@ class ALSSuite extends FunSuite with Logging { u, p, correct, prediction, trueRatings, predictedRatings, predictedU, predictedP)) } } - - sc.stop() - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port") } } diff --git a/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala b/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala index 6216523a9e..55f2c5c18e 100644 --- a/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala @@ -2,21 +2,26 @@ package spark.ml.regression import scala.util.Random +import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext import spark.SparkContext._ -import spark.Logging -class LogisticRegressionSuite extends FunSuite with Logging { +class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } // Test if we can correctly learn A, B where Y = logistic(A + B*X) test("logistic regression") { val nPoints = 10000 val rnd = new Random(42) - val sc = new SparkContext("local", "test") val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) val A = 2.0 @@ -24,14 +29,14 @@ class LogisticRegressionSuite extends FunSuite with Logging { // 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 rLogis = (0 until nPoints).map { i => val u = unifRand.nextDouble() math.log(u) - math.log(1.0-u) } - + // y <- A + B*x + rlogis(100) // y <- as.numeric(y > 0) - val y = (0 until nPoints).map { i => + val y = (0 until nPoints).map { i => val yVal = A + B * x1(i) + rLogis(i) if (yVal > 0) 1.0 else 0.0 } @@ -40,16 +45,13 @@ class LogisticRegressionSuite extends FunSuite with Logging { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val lr = new LogisticRegression().setStepSize(5.0) + val lr = new LogisticRegression().setStepSize(10.0) .setNumIterations(20) val model = lr.train(testRDD) - assert(model.weights.get(0) >= -1.60 && model.weights.get(0) <= -1.40) - assert(model.intercept >= 1.9 && model.intercept <= 2.1) - - sc.stop() - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port") + val weight0 = model.weights.get(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]") } } diff --git a/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala b/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala index 795cda1379..aed5cbec24 100644 --- a/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala @@ -2,19 +2,25 @@ package spark.ml.regression import scala.util.Random +import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext import spark.SparkContext._ -class RidgeRegressionSuite extends FunSuite { +class RidgeRegressionSuite extends FunSuite with BeforeAndAfterAll { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } // Test if we can correctly learn Y = 3 + X1 + X2 when // X1 and X2 are collinear. test("multi-collinear variables") { val rnd = new Random(43) - val sc = new SparkContext("local", "test") val x1 = Array.fill[Double](20)(rnd.nextGaussian()) // Pick a mean close to mean of x1 @@ -37,9 +43,5 @@ class RidgeRegressionSuite extends FunSuite { assert(model.weights.length === 2) assert(model.weights.get(0) >= 0.9 && model.weights.get(0) <= 1.1) assert(model.weights.get(1) >= 0.9 && model.weights.get(1) <= 1.1) - - sc.stop() - // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown - System.clearProperty("spark.driver.port") } } -- cgit v1.2.3 From 652ea0f1d86e77503942a7846f89236b15da2ce9 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 11:15:13 -0700 Subject: Allow RDD.takeSample to give samples bigger than the RDD Before, when withReplacement was set to true, we would not get a sample bigger than the RDD's count(). Conflicts: core/src/main/scala/spark/RDD.scala core/src/test/scala/spark/RDDSuite.scala --- core/src/main/scala/spark/RDD.scala | 44 +++++++++++++++++--------------- core/src/test/scala/spark/RDDSuite.scala | 34 ++++++++++++++++++++++++ 2 files changed, 58 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 8e1e18c06c..106fb2960f 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -122,7 +122,7 @@ abstract class RDD[T: ClassManifest]( /** User-defined generator of this RDD*/ var generator = Utils.getCallSiteInfo.firstUserClass - + /** Reset generator*/ def setGenerator(_generator: String) = { generator = _generator @@ -284,31 +284,35 @@ abstract class RDD[T: ClassManifest]( def takeSample(withReplacement: Boolean, num: Int, seed: Int): Array[T] = { var fraction = 0.0 var total = 0 - val multiplier = 3.0 - val initialCount = count() + var multiplier = 3.0 + var initialCount = this.count() var maxSelected = 0 + if (num < 0) { + throw new IllegalArgumentException("Negative number of elements requested") + } + if (initialCount > Integer.MAX_VALUE - 1) { maxSelected = Integer.MAX_VALUE - 1 } else { maxSelected = initialCount.toInt } - if (num > initialCount) { + if (num > initialCount && !withReplacement) { total = maxSelected - fraction = math.min(multiplier * (maxSelected + 1) / initialCount, 1.0) - } else if (num < 0) { - throw(new IllegalArgumentException("Negative number of elements requested")) + fraction = multiplier * (maxSelected + 1) / initialCount } else { - fraction = math.min(multiplier * (num + 1) / initialCount, 1.0) + fraction = multiplier * (num + 1) / initialCount total = num } val rand = new Random(seed) - var samples = this.sample(withReplacement, fraction, rand.nextInt).collect() + var samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() + // If the first sample didn't turn out large enough, keep trying to take samples; + // this shouldn't happen often because we use a big multiplier for thei initial size while (samples.length < total) { - samples = this.sample(withReplacement, fraction, rand.nextInt).collect() + samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() } Utils.randomizeInPlace(samples, rand).take(total) @@ -366,7 +370,7 @@ abstract class RDD[T: ClassManifest]( /** * Return an RDD created by piping elements to a forked external process. */ - def pipe(command: String, env: Map[String, String]): RDD[String] = + def pipe(command: String, env: Map[String, String]): RDD[String] = new PipedRDD(this, command, env) @@ -377,24 +381,24 @@ abstract class RDD[T: ClassManifest]( * @param command command to run in forked process. * @param env environment variables to set. * @param printPipeContext Before piping elements, this function is called as an oppotunity - * to pipe context data. Print line function (like out.println) will be + * to pipe context data. Print line function (like out.println) will be * passed as printPipeContext's parameter. - * @param printRDDElement Use this function to customize how to pipe elements. This function - * will be called with each RDD element as the 1st parameter, and the + * @param printRDDElement Use this function to customize how to pipe elements. This function + * will be called with each RDD element as the 1st parameter, and the * print line function (like out.println()) as the 2nd parameter. * An example of pipe the RDD data of groupBy() in a streaming way, * instead of constructing a huge String to concat all the elements: - * def printRDDElement(record:(String, Seq[String]), f:String=>Unit) = + * def printRDDElement(record:(String, Seq[String]), f:String=>Unit) = * for (e <- record._2){f(e)} * @return the result RDD */ def pipe( - command: Seq[String], - env: Map[String, String] = Map(), + command: Seq[String], + env: Map[String, String] = Map(), printPipeContext: (String => Unit) => Unit = null, - printRDDElement: (T, String => Unit) => Unit = null): RDD[String] = - new PipedRDD(this, command, env, - if (printPipeContext ne null) sc.clean(printPipeContext) else null, + printRDDElement: (T, String => Unit) => Unit = null): RDD[String] = + new PipedRDD(this, command, env, + if (printPipeContext ne null) sc.clean(printPipeContext) else null, if (printRDDElement ne null) sc.clean(printRDDElement) else null) /** diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index d8db69b1c9..919bbb2ed8 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -251,4 +251,38 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(topK.size === 2) assert(topK.sorted === Array("b", "a")) } + + test("takeSample") { + sc = new SparkContext("local", "test") + val data = sc.parallelize(1 to 100, 2) + for (seed <- 1 to 5) { + val sample = data.takeSample(withReplacement=false, 20, seed) + assert(sample.size === 20) // Got exactly 20 elements + assert(sample.toSet.size === 20) // Elements are distinct + assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + } + for (seed <- 1 to 5) { + val sample = data.takeSample(withReplacement=false, 200, seed) + assert(sample.size === 100) // Got only 100 elements + assert(sample.toSet.size === 100) // Elements are distinct + assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + } + for (seed <- 1 to 5) { + val sample = data.takeSample(withReplacement=true, 20, seed) + assert(sample.size === 20) // Got exactly 20 elements + assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + } + for (seed <- 1 to 5) { + val sample = data.takeSample(withReplacement=true, 100, seed) + assert(sample.size === 100) // Got exactly 100 elements + // Chance of getting all distinct elements is astronomically low, so test we got < 100 + assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements") + } + for (seed <- 1 to 5) { + val sample = data.takeSample(withReplacement=true, 200, seed) + assert(sample.size === 200) // Got exactly 200 elements + // Chance of getting all distinct elements is still quite low, so test we got < 100 + assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements") + } + } } -- cgit v1.2.3 From e7d49388e3df2cd3abb8fedc8a097831d4f33e3e Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 23 Jun 2013 00:01:10 -0400 Subject: Added unit test for K-means, and fixed some bugs --- ml/src/main/scala/spark/ml/clustering/KMeans.scala | 2 +- .../scala/spark/ml/clustering/KMeansModel.scala | 11 +- .../scala/spark/ml/clustering/KMeansSuite.scala | 150 +++++++++++++++++++++ 3 files changed, 152 insertions(+), 11 deletions(-) create mode 100644 ml/src/test/scala/spark/ml/clustering/KMeansSuite.scala diff --git a/ml/src/main/scala/spark/ml/clustering/KMeans.scala b/ml/src/main/scala/spark/ml/clustering/KMeans.scala index d1786b0812..8c4bec2a26 100644 --- a/ml/src/main/scala/spark/ml/clustering/KMeans.scala +++ b/ml/src/main/scala/spark/ml/clustering/KMeans.scala @@ -226,7 +226,7 @@ class KMeans private ( }.reduceByKey(_ + _).collectAsMap() val finalCenters = (0 until runs).map { r => val myCenters = centers(r).toArray - val myWeights = (0 until myCenters.length).map(i => weightMap((r, i))).toArray + val myWeights = (0 until myCenters.length).map(i => weightMap.getOrElse((r, i), 0.0)).toArray LocalKMeans.kMeansPlusPlus(r, myCenters, myWeights, k, 30) } diff --git a/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala b/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala index 84bfd0f99a..8244ccc55b 100644 --- a/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala +++ b/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala @@ -14,16 +14,7 @@ class KMeansModel(val clusterCenters: Array[Array[Double]]) extends Serializable /** Return the cluster index that a given point belongs to. */ def predict(point: Array[Double]): Int = { - var bestDist = Double.PositiveInfinity - var bestIndex = -1 - for (i <- 0 until k) { - val dist = MLUtils.squaredDistance(clusterCenters(i), point) - if (dist < bestDist) { - bestDist = dist - bestIndex = i - } - } - bestIndex + KMeans.findClosest(clusterCenters, point)._1 } /** diff --git a/ml/src/test/scala/spark/ml/clustering/KMeansSuite.scala b/ml/src/test/scala/spark/ml/clustering/KMeansSuite.scala new file mode 100644 index 0000000000..f3bd1d599f --- /dev/null +++ b/ml/src/test/scala/spark/ml/clustering/KMeansSuite.scala @@ -0,0 +1,150 @@ +package spark.ml.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 { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + val EPSILON = 1e-4 + + 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||") + 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||") + 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) + } +} -- cgit v1.2.3 From 9441d3ef09273436044a8a8b37706eab8df58995 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 23 Jun 2013 00:50:14 -0400 Subject: Use random seeds for K-means and ALS, and increase tolerance in tests Random seeds make more sense by default for a machine learning library because other libraries behave the same way (people expect to be able to run the algorithm multiple times and get a better answer), but we can add configuration later if needed. Tests that depend on specific seed choices seem brittle. --- ml/src/main/scala/spark/ml/clustering/KMeans.scala | 7 ++++--- ml/src/main/scala/spark/ml/recommendation/ALS.scala | 11 ++++++----- ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala | 4 ++-- 3 files changed, 12 insertions(+), 10 deletions(-) diff --git a/ml/src/main/scala/spark/ml/clustering/KMeans.scala b/ml/src/main/scala/spark/ml/clustering/KMeans.scala index 8c4bec2a26..d35f942c01 100644 --- a/ml/src/main/scala/spark/ml/clustering/KMeans.scala +++ b/ml/src/main/scala/spark/ml/clustering/KMeans.scala @@ -179,7 +179,7 @@ class KMeans private ( */ 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, 1) + val sample = data.takeSample(true, runs * k, new Random().nextInt()) Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k)) } @@ -194,7 +194,8 @@ class KMeans private ( */ private def initKMeansParallel(data: RDD[Array[Double]]): Array[ClusterCenters] = { // Initialize each run's center to a random point - val sample = data.takeSample(true, runs, 1) + val seed = new Random().nextInt() + val sample = data.takeSample(true, runs, seed) val centers = Array.tabulate(runs)(r => ArrayBuffer(sample(r))) // On each step, sample 2 * k points on average for each run with probability proportional @@ -205,7 +206,7 @@ class KMeans private ( for (r <- 0 until runs) yield (r, KMeans.pointCost(centerArrays(r), point)) }.reduceByKey(_ + _).collectAsMap() val chosen = data.mapPartitionsWithIndex { (index, points) => - val rand = new Random((step << 16) ^ index) + val rand = new Random(seed ^ (step << 16) ^ index) for { p <- points r <- 0 until runs diff --git a/ml/src/main/scala/spark/ml/recommendation/ALS.scala b/ml/src/main/scala/spark/ml/recommendation/ALS.scala index abf54216a5..8d5c16847a 100644 --- a/ml/src/main/scala/spark/ml/recommendation/ALS.scala +++ b/ml/src/main/scala/spark/ml/recommendation/ALS.scala @@ -97,9 +97,10 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock) val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) - // Initialize user and product factors deterministically. - var users = userOutLinks.mapValues(_.elementIds.map(u => makeInitialFactor(rank, u))) - var products = productOutLinks.mapValues(_.elementIds.map(p => makeInitialFactor(rank, -p))) + // Initialize user and product factors randomly + val seed = new Random().nextInt() + var users = userOutLinks.mapValues(_.elementIds.map(u => randomFactor(rank, seed ^ u))) + var products = productOutLinks.mapValues(_.elementIds.map(p => randomFactor(rank, seed ^ ~p))) for (iter <- 0 until iterations) { // perform ALS update @@ -174,10 +175,10 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l } /** - * Make a random initial factor vector with the given seed. + * Make a random factor vector with the given seed. * TODO: Initialize things using mapPartitionsWithIndex to make it faster? */ - private def makeInitialFactor(rank: Int, seed: Int): Array[Double] = { + private def randomFactor(rank: Int, seed: Int): Array[Double] = { val rand = new Random(seed) Array.fill(rank)(rand.nextDouble) } diff --git a/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala b/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala index c450d5315c..f3f56c4357 100644 --- a/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala +++ b/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala @@ -20,11 +20,11 @@ class ALSSuite extends FunSuite with BeforeAndAfterAll { } test("rank-1 matrices") { - testALS(10, 20, 1, 5, 0.5, 0.1) + testALS(10, 20, 1, 15, 0.7, 0.3) } test("rank-2 matrices") { - testALS(20, 30, 2, 10, 0.7, 0.3) + testALS(20, 30, 2, 15, 0.7, 0.3) } /** -- cgit v1.2.3 From 09f187a400623020b520a4cc5c8fc5814746dab1 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 30 Jun 2013 14:40:24 -0700 Subject: Add top-level methods for regression methods. Also add multiple versions of them to make it easier to call them from java. --- .../spark/ml/regression/LogisticRegression.scala | 64 +++++++++++++++++++++- .../spark/ml/regression/RidgeRegression.scala | 40 ++++++++++++-- 2 files changed, 96 insertions(+), 8 deletions(-) diff --git a/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala b/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala index 1508f6934a..435669f72a 100644 --- a/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala @@ -83,8 +83,67 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D } } +/** + * Top-level methods for calling Logistic Regression. + */ object LogisticRegression { + /** + * 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. + * + * @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[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double) + : LogisticRegressionModel = + { + new LogisticRegression(stepSize, miniBatchFraction, numIterations).train(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. + * + * @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[(Double, Array[Double])], + 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. + * + * @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[(Double, Array[Double])], + numIterations: Int) + : LogisticRegressionModel = + { + train(input, numIterations, 1.0, 1.0) + } + def main(args: Array[String]) { if (args.length != 4) { println("Usage: LogisticRegression ") @@ -92,9 +151,8 @@ object LogisticRegression { } val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadData(sc, args(1)) - val lr = new LogisticRegression().setStepSize(args(2).toDouble) - .setNumIterations(args(3).toInt) - val model = lr.train(data) + val model = LogisticRegression.train(data, args(3).toInt, args(2).toDouble) + sc.stop() } } diff --git a/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala b/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala index 16d146ac27..dae224144e 100644 --- a/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala @@ -134,9 +134,42 @@ class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) model } } - +/** + * Top-level methods for calling Ridge Regression. + */ object RidgeRegression { + /** + * Train a ridge regression model given an RDD of (response, features) pairs. + * We use the closed form solution to compute the cross-validation score for + * a given lambda. The optimal lambda is computed by performing binary search + * between the provided bounds of lambda. + * + * @param input RDD of (response, array of features) pairs. + * @param lambdaLow lower bound used in binary search for lambda + * @param lambdaHigh upper bound used in binary search for lambda + */ + def train( + input: RDD[(Double, Array[Double])], + lambdaLow: Double, + lambdaHigh: Double) + : RidgeRegressionModel = + { + new RidgeRegression(lambdaLow, lambdaHigh).train(input) + } + + /** + * Train a ridge regression model given an RDD of (response, features) pairs. + * We use the closed form solution to compute the cross-validation score for + * a given lambda. The optimal lambda is computed by performing binary search + * between lambda values of 0 and 100. + * + * @param input RDD of (response, array of features) pairs. + */ + def train(input: RDD[(Double, Array[Double])]) : RidgeRegressionModel = { + train(input, 0.0, 100.0) + } + def main(args: Array[String]) { if (args.length != 2) { println("Usage: RidgeRegression ") @@ -144,10 +177,7 @@ object RidgeRegression { } val sc = new SparkContext(args(0), "RidgeRegression") val data = MLUtils.loadData(sc, args(1)) - val ridgeReg = new RidgeRegression().setLowLambda(0) - .setHighLambda(1000) - - val model = ridgeReg.train(data) + val model = RidgeRegression.train(data, 0, 1000) sc.stop() } } -- cgit v1.2.3 From 0e33c88cbd85d3c9aa73b4c66a03372169abf3a1 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 30 Jun 2013 14:43:08 -0700 Subject: Rename package gradient to optimization --- ml/src/main/scala/spark/ml/gradient/Gradient.scala | 33 ------------ .../scala/spark/ml/gradient/GradientDescent.scala | 62 ---------------------- ml/src/main/scala/spark/ml/gradient/Updater.scala | 27 ---------- .../scala/spark/ml/optimization/Gradient.scala | 33 ++++++++++++ .../spark/ml/optimization/GradientDescent.scala | 62 ++++++++++++++++++++++ .../main/scala/spark/ml/optimization/Updater.scala | 27 ++++++++++ .../spark/ml/regression/LogisticRegression.scala | 2 +- 7 files changed, 123 insertions(+), 123 deletions(-) delete mode 100644 ml/src/main/scala/spark/ml/gradient/Gradient.scala delete mode 100644 ml/src/main/scala/spark/ml/gradient/GradientDescent.scala delete mode 100644 ml/src/main/scala/spark/ml/gradient/Updater.scala create mode 100644 ml/src/main/scala/spark/ml/optimization/Gradient.scala create mode 100644 ml/src/main/scala/spark/ml/optimization/GradientDescent.scala create mode 100644 ml/src/main/scala/spark/ml/optimization/Updater.scala diff --git a/ml/src/main/scala/spark/ml/gradient/Gradient.scala b/ml/src/main/scala/spark/ml/gradient/Gradient.scala deleted file mode 100644 index 50795c01a2..0000000000 --- a/ml/src/main/scala/spark/ml/gradient/Gradient.scala +++ /dev/null @@ -1,33 +0,0 @@ -package spark.ml.gradient - -import org.jblas.DoubleMatrix - -abstract class Gradient extends Serializable { - /** - * Compute the gradient for a given row of data. - * - * @param data - One row of data. Row matrix of size 1xn where n is the number of features. - * @param label - Label for this data item. - * @param weights - Column matrix containing weights for every feature. - */ - def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): - (DoubleMatrix, Double) -} - -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) - } -} diff --git a/ml/src/main/scala/spark/ml/gradient/GradientDescent.scala b/ml/src/main/scala/spark/ml/gradient/GradientDescent.scala deleted file mode 100644 index 8cd2a69a3a..0000000000 --- a/ml/src/main/scala/spark/ml/gradient/GradientDescent.scala +++ /dev/null @@ -1,62 +0,0 @@ -package spark.ml.gradient - -import spark.{Logging, RDD, SparkContext} -import spark.SparkContext._ - -import org.jblas.DoubleMatrix - -import scala.collection.mutable.ArrayBuffer - - -object GradientDescent { - - /** - * Run gradient descent in parallel using mini batches. - * Based on Matlab code written by John Duchi. - * - * @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 numIters - number of iterations that SGD should be run. - * @param miniBatchFraction - fraction of the input data set that should be used for - * one iteration of SGD. Default value 1.0. - * - * @return weights - Column matrix containing weights for every feature. - * @return lossHistory - Array containing the loss computed for every iteration. - */ - def runMiniBatchSGD( - data: RDD[(Double, Array[Double])], - gradient: Gradient, - updater: Updater, - stepSize: Double, - numIters: Int, - miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { - - val lossHistory = new ArrayBuffer[Double](numIters) - - val nfeatures: Int = data.take(1)(0)._2.length - val nexamples: Long = data.count() - val miniBatchSize = nexamples * miniBatchFraction - - // Initialize weights as a column matrix - var weights = DoubleMatrix.ones(nfeatures) - var reg_val = 0.0 - - for (i <- 1 to numIters) { - val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42+i).map { - case (y, features) => - val featuresRow = new DoubleMatrix(features.length, 1, features:_*) - val (grad, loss) = gradient.compute(featuresRow, y, weights) - (grad, loss) - }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) - - lossHistory.append(lossSum / miniBatchSize + reg_val) - val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i) - weights = update._1 - reg_val = update._2 - } - - (weights, lossHistory.toArray) - } -} diff --git a/ml/src/main/scala/spark/ml/gradient/Updater.scala b/ml/src/main/scala/spark/ml/gradient/Updater.scala deleted file mode 100644 index 27f03897b0..0000000000 --- a/ml/src/main/scala/spark/ml/gradient/Updater.scala +++ /dev/null @@ -1,27 +0,0 @@ -package spark.ml.gradient - -import org.jblas.DoubleMatrix - -abstract class Updater extends Serializable { - /** - * Compute an updated value for weights given the gradient, stepSize and iteration number. - * - * @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 - * - * @return weightsNew - Column matrix containing updated weights - * @return reg_val - regularization value - */ - def compute(weightsOlds: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): - (DoubleMatrix, Double) -} - -class SimpleUpdater extends Updater { - override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, - stepSize: Double, iter: Int): (DoubleMatrix, Double) = { - val normGradient = gradient.mul(stepSize / math.sqrt(iter)) - (weightsOld.sub(normGradient), 0) - } -} diff --git a/ml/src/main/scala/spark/ml/optimization/Gradient.scala b/ml/src/main/scala/spark/ml/optimization/Gradient.scala new file mode 100644 index 0000000000..6d062ebddf --- /dev/null +++ b/ml/src/main/scala/spark/ml/optimization/Gradient.scala @@ -0,0 +1,33 @@ +package spark.ml.optimization + +import org.jblas.DoubleMatrix + +abstract class Gradient extends Serializable { + /** + * Compute the gradient for a given row of data. + * + * @param data - One row of data. Row matrix of size 1xn where n is the number of features. + * @param label - Label for this data item. + * @param weights - Column matrix containing weights for every feature. + */ + def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): + (DoubleMatrix, Double) +} + +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) + } +} diff --git a/ml/src/main/scala/spark/ml/optimization/GradientDescent.scala b/ml/src/main/scala/spark/ml/optimization/GradientDescent.scala new file mode 100644 index 0000000000..d959ebf71c --- /dev/null +++ b/ml/src/main/scala/spark/ml/optimization/GradientDescent.scala @@ -0,0 +1,62 @@ +package spark.ml.optimization + +import spark.{Logging, RDD, SparkContext} +import spark.SparkContext._ + +import org.jblas.DoubleMatrix + +import scala.collection.mutable.ArrayBuffer + + +object GradientDescent { + + /** + * Run gradient descent in parallel using mini batches. + * Based on Matlab code written by John Duchi. + * + * @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 numIters - number of iterations that SGD should be run. + * @param miniBatchFraction - fraction of the input data set that should be used for + * one iteration of SGD. Default value 1.0. + * + * @return weights - Column matrix containing weights for every feature. + * @return lossHistory - Array containing the loss computed for every iteration. + */ + def runMiniBatchSGD( + data: RDD[(Double, Array[Double])], + gradient: Gradient, + updater: Updater, + stepSize: Double, + numIters: Int, + miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { + + val lossHistory = new ArrayBuffer[Double](numIters) + + val nfeatures: Int = data.take(1)(0)._2.length + val nexamples: Long = data.count() + val miniBatchSize = nexamples * miniBatchFraction + + // Initialize weights as a column matrix + var weights = DoubleMatrix.ones(nfeatures) + var reg_val = 0.0 + + for (i <- 1 to numIters) { + val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42+i).map { + case (y, features) => + val featuresRow = new DoubleMatrix(features.length, 1, features:_*) + val (grad, loss) = gradient.compute(featuresRow, y, weights) + (grad, loss) + }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) + + lossHistory.append(lossSum / miniBatchSize + reg_val) + val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i) + weights = update._1 + reg_val = update._2 + } + + (weights, lossHistory.toArray) + } +} diff --git a/ml/src/main/scala/spark/ml/optimization/Updater.scala b/ml/src/main/scala/spark/ml/optimization/Updater.scala new file mode 100644 index 0000000000..dfc7bf2025 --- /dev/null +++ b/ml/src/main/scala/spark/ml/optimization/Updater.scala @@ -0,0 +1,27 @@ +package spark.ml.optimization + +import org.jblas.DoubleMatrix + +abstract class Updater extends Serializable { + /** + * Compute an updated value for weights given the gradient, stepSize and iteration number. + * + * @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 + * + * @return weightsNew - Column matrix containing updated weights + * @return reg_val - regularization value + */ + def compute(weightsOlds: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): + (DoubleMatrix, Double) +} + +class SimpleUpdater extends Updater { + override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, + stepSize: Double, iter: Int): (DoubleMatrix, Double) = { + val normGradient = gradient.mul(stepSize / math.sqrt(iter)) + (weightsOld.sub(normGradient), 0) + } +} diff --git a/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala b/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala index 435669f72a..3c471ab652 100644 --- a/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala @@ -1,7 +1,7 @@ package spark.ml.regression import spark.{Logging, RDD, SparkContext} -import spark.ml.gradient._ +import spark.ml.optimization._ import spark.ml.util.MLUtils import org.jblas.DoubleMatrix -- cgit v1.2.3 From 399bd65ef5f780c2796f6facf9fac8fd9ec2c2f4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 11:27:06 -0700 Subject: Fixed compile error due to merge --- core/src/test/scala/spark/RDDSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 919bbb2ed8..e41ae385c0 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -253,7 +253,6 @@ class RDDSuite extends FunSuite with SharedSparkContext { } test("takeSample") { - sc = new SparkContext("local", "test") val data = sc.parallelize(1 to 100, 2) for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement=false, 20, seed) -- cgit v1.2.3 From 43b24635ee45a845f2432bc13c11fcf2eb02f2f3 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 11:38:53 -0700 Subject: Renamed ML package to MLlib and added it to classpath --- bin/compute-classpath.cmd | 2 + bin/compute-classpath.sh | 2 + ml/data/als/test.data | 16 - ml/data/lr-data/random.data | 1000 -------------------- ml/data/ridge-data/lpsa.data | 67 -- ml/src/main/scala/spark/ml/clustering/KMeans.scala | 318 ------- .../scala/spark/ml/clustering/KMeansModel.scala | 27 - .../scala/spark/ml/clustering/LocalKMeans.scala | 88 -- .../scala/spark/ml/optimization/Gradient.scala | 33 - .../spark/ml/optimization/GradientDescent.scala | 62 -- .../main/scala/spark/ml/optimization/Updater.scala | 27 - .../main/scala/spark/ml/recommendation/ALS.scala | 387 -------- .../recommendation/MatrixFactorizationModel.scala | 23 - .../spark/ml/regression/LogisticRegression.scala | 158 ---- .../regression/LogisticRegressionGenerator.scala | 41 - .../scala/spark/ml/regression/Regression.scala | 21 - .../spark/ml/regression/RidgeRegression.scala | 183 ---- .../ml/regression/RidgeRegressionGenerator.scala | 55 -- ml/src/main/scala/spark/ml/util/MLUtils.scala | 95 -- ml/src/test/resources/log4j.properties | 11 - .../scala/spark/ml/clustering/KMeansSuite.scala | 150 --- .../scala/spark/ml/recommendation/ALSSuite.scala | 80 -- .../ml/regression/LogisticRegressionSuite.scala | 57 -- .../spark/ml/regression/RidgeRegressionSuite.scala | 47 - mllib/data/als/test.data | 16 + mllib/data/lr-data/random.data | 1000 ++++++++++++++++++++ mllib/data/ridge-data/lpsa.data | 67 ++ .../main/scala/spark/ml/clustering/KMeans.scala | 319 +++++++ .../scala/spark/ml/clustering/KMeansModel.scala | 27 + .../scala/spark/ml/clustering/LocalKMeans.scala | 88 ++ .../scala/spark/ml/optimization/Gradient.scala | 33 + .../spark/ml/optimization/GradientDescent.scala | 62 ++ .../main/scala/spark/ml/optimization/Updater.scala | 27 + .../main/scala/spark/ml/recommendation/ALS.scala | 387 ++++++++ .../recommendation/MatrixFactorizationModel.scala | 23 + .../spark/ml/regression/LogisticRegression.scala | 158 ++++ .../regression/LogisticRegressionGenerator.scala | 41 + .../scala/spark/ml/regression/Regression.scala | 21 + .../spark/ml/regression/RidgeRegression.scala | 183 ++++ .../ml/regression/RidgeRegressionGenerator.scala | 55 ++ mllib/src/main/scala/spark/ml/util/MLUtils.scala | 95 ++ mllib/src/test/resources/log4j.properties | 11 + .../scala/spark/ml/clustering/KMeansSuite.scala | 150 +++ .../scala/spark/ml/recommendation/ALSSuite.scala | 80 ++ .../ml/regression/LogisticRegressionSuite.scala | 57 ++ .../spark/ml/regression/RidgeRegressionSuite.scala | 47 + project/SparkBuild.scala | 8 +- 47 files changed, 2955 insertions(+), 2950 deletions(-) delete mode 100644 ml/data/als/test.data delete mode 100755 ml/data/lr-data/random.data delete mode 100644 ml/data/ridge-data/lpsa.data delete mode 100644 ml/src/main/scala/spark/ml/clustering/KMeans.scala delete mode 100644 ml/src/main/scala/spark/ml/clustering/KMeansModel.scala delete mode 100644 ml/src/main/scala/spark/ml/clustering/LocalKMeans.scala delete mode 100644 ml/src/main/scala/spark/ml/optimization/Gradient.scala delete mode 100644 ml/src/main/scala/spark/ml/optimization/GradientDescent.scala delete mode 100644 ml/src/main/scala/spark/ml/optimization/Updater.scala delete mode 100644 ml/src/main/scala/spark/ml/recommendation/ALS.scala delete mode 100644 ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala delete mode 100644 ml/src/main/scala/spark/ml/regression/LogisticRegression.scala delete mode 100644 ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala delete mode 100644 ml/src/main/scala/spark/ml/regression/Regression.scala delete mode 100644 ml/src/main/scala/spark/ml/regression/RidgeRegression.scala delete mode 100644 ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala delete mode 100644 ml/src/main/scala/spark/ml/util/MLUtils.scala delete mode 100644 ml/src/test/resources/log4j.properties delete mode 100644 ml/src/test/scala/spark/ml/clustering/KMeansSuite.scala delete mode 100644 ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala delete mode 100644 ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala delete mode 100644 ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala create mode 100644 mllib/data/als/test.data create mode 100755 mllib/data/lr-data/random.data create mode 100644 mllib/data/ridge-data/lpsa.data create mode 100644 mllib/src/main/scala/spark/ml/clustering/KMeans.scala create mode 100644 mllib/src/main/scala/spark/ml/clustering/KMeansModel.scala create mode 100644 mllib/src/main/scala/spark/ml/clustering/LocalKMeans.scala create mode 100644 mllib/src/main/scala/spark/ml/optimization/Gradient.scala create mode 100644 mllib/src/main/scala/spark/ml/optimization/GradientDescent.scala create mode 100644 mllib/src/main/scala/spark/ml/optimization/Updater.scala create mode 100644 mllib/src/main/scala/spark/ml/recommendation/ALS.scala create mode 100644 mllib/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala create mode 100644 mllib/src/main/scala/spark/ml/regression/LogisticRegression.scala create mode 100644 mllib/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala create mode 100644 mllib/src/main/scala/spark/ml/regression/Regression.scala create mode 100644 mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala create mode 100644 mllib/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala create mode 100644 mllib/src/main/scala/spark/ml/util/MLUtils.scala create mode 100644 mllib/src/test/resources/log4j.properties create mode 100644 mllib/src/test/scala/spark/ml/clustering/KMeansSuite.scala create mode 100644 mllib/src/test/scala/spark/ml/recommendation/ALSSuite.scala create mode 100644 mllib/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala create mode 100644 mllib/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 6e7efbd334..44826f339c 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -15,6 +15,7 @@ set CORE_DIR=%FWDIR%core set REPL_DIR=%FWDIR%repl set EXAMPLES_DIR=%FWDIR%examples set BAGEL_DIR=%FWDIR%bagel +set MLLIB_DIR=%FWDIR%mllib set STREAMING_DIR=%FWDIR%streaming set PYSPARK_DIR=%FWDIR%python @@ -29,6 +30,7 @@ set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\bundles\* set CLASSPATH=%CLASSPATH%;%FWDIR%repl\lib\* set CLASSPATH=%CLASSPATH%;%FWDIR%python\lib\* set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes +set CLASSPATH=%CLASSPATH%;%MLLIB_DIR%\target\scala-%SCALA_VERSION%\classes rem Add hadoop conf dir - else FileSystem.*, etc fail rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 3a78880290..75c58d1181 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -18,6 +18,7 @@ REPL_DIR="$FWDIR/repl" REPL_BIN_DIR="$FWDIR/repl-bin" EXAMPLES_DIR="$FWDIR/examples" BAGEL_DIR="$FWDIR/bagel" +MLLIB_DIR="$FWDIR/mllib" STREAMING_DIR="$FWDIR/streaming" PYSPARK_DIR="$FWDIR/python" @@ -49,6 +50,7 @@ if [ -e $REPL_BIN_DIR/target ]; then CLASSPATH+=":$EXAMPLES_JAR" fi CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH="$CLASSPATH:$MLLIB_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do CLASSPATH="$CLASSPATH:$jar" done diff --git a/ml/data/als/test.data b/ml/data/als/test.data deleted file mode 100644 index e476cc23e0..0000000000 --- a/ml/data/als/test.data +++ /dev/null @@ -1,16 +0,0 @@ -1,1,5.0 -1,2,1.0 -1,3,5.0 -1,4,1.0 -2,1,5.0 -2,2,1.0 -2,3,5.0 -2,4,1.0 -3,1,1.0 -3,2,5.0 -3,3,1.0 -3,4,5.0 -4,1,1.0 -4,2,5.0 -4,3,1.0 -4,4,5.0 diff --git a/ml/data/lr-data/random.data b/ml/data/lr-data/random.data deleted file mode 100755 index 29bcb8acba..0000000000 --- a/ml/data/lr-data/random.data +++ /dev/null @@ -1,1000 +0,0 @@ -0.0,-0.19138793197590276 0.7834675900121327 -1.0,3.712420417753061 3.55967640829891 -0.0,-0.3173743619974614 0.9034702789806682 -1.0,4.759494447180777 3.407011867344781 -0.0,-0.7078607074437426 -0.7866705652344417 -1.0,2.6708084832010215 2.5322909406378016 -0.0,-0.07553885038446313 -0.1297104483563081 -1.0,2.759487072285262 2.474689814713741 -0.0,-2.2199161547238107 0.7543109438660762 -1.0,1.922617509832946 1.9412373902594937 -0.0,0.8140942462004225 1.883920822277784 -1.0,1.7649295902120172 3.8195077526061363 -0.0,-1.1173052428096684 -1.468964723960145 -1.0,1.8733449544967458 2.913026590975709 -0.0,-0.11212965215910947 1.068087981775071 -1.0,2.3368459971730227 5.453870208593922 -0.0,-1.2802488543364463 -0.47218504171867676 -1.0,4.1917343620336895 3.5602286778418355 -0.0,0.5995976502137177 -0.797374550890321 -1.0,3.721592294428238 4.824418090974808 -0.0,-0.0721649164244053 -1.3952880192542576 -1.0,3.609764030146346 3.4730043476891277 -0.0,-1.5078269860498976 -2.6460421495665987 -1.0,1.8510254911824193 1.6748364225650059 -0.0,1.021485727769095 -0.14476425336866738 -1.0,4.10105000223134 2.3772502437548493 -0.0,2.6132710211418675 -1.061646527586342 -1.0,2.6444875273854653 4.043302750329545 -0.0,1.115723715938777 0.38401588153403887 -1.0,2.045759949164019 3.156447533448806 -0.0,-1.0543022640565405 -0.6820337845705753 -1.0,3.535337069948117 3.8121122972294965 -0.0,0.9427529503486505 -0.25123516319259886 -1.0,3.9611643301316795 3.3144121016644443 -0.0,-0.15013188927817916 0.8178862482229886 -1.0,3.200504584029051 2.3088398886136057 -0.0,0.819731993393585 -0.47386644109886344 -1.0,3.283317566020217 3.4828146842654513 -0.0,-2.3283941193793303 -0.6148925379529 -1.0,3.901670215294089 3.6356776610143324 -0.0,-0.28635769830042973 0.049586437072917544 -1.0,3.1114746381043927 3.6314805300338775 -0.0,-1.3085536069757229 0.11172767926766304 -1.0,3.3676979357140744 4.689661419564771 -0.0,-1.5820787210442733 1.3226576351191428 -1.0,2.5957586701668207 3.0648240201825923 -0.0,-2.116823743560968 0.272822309954307 -1.0,3.31672509500716 3.870172182480263 -0.0,0.09751166932653511 0.6469052579904877 -1.0,2.0609623373451305 3.9496181906908694 -0.0,0.5238217321419351 -1.2424816480725946 -1.0,3.5731384504449717 5.293293512805712 -0.0,-0.8507917425723299 -1.2243124053200718 -1.0,3.3060954421001867 3.1337045819604565 -0.0,1.5066706426420082 0.04176666807070882 -1.0,4.197316426430547 2.327643377792433 -0.0,-1.8068158696573955 -1.6380836149377855 -1.0,3.568239793850545 3.561688791420822 -0.0,0.4705756905309871 1.1991675114038487 -1.0,4.85003762884306 4.253420553408024 -0.0,0.7595792932847568 0.014062431397674205 -1.0,1.6984862661221896 1.7746925013882613 -0.0,0.1132294255888917 -0.09228036942051128 -1.0,3.766092539171029 2.765647342841482 -0.0,1.053401788561791 -1.0588667339849278 -1.0,2.780021685872393 3.239478188786074 -0.0,0.4042022490052266 1.0982210323828034 -1.0,2.4939569547402063 2.4615506964861273 -0.0,0.4469359967563411 0.3880418183993791 -1.0,2.7943749030887486 3.742182807141721 -0.0,-0.4418685162293727 0.802180923066725 -1.0,3.711213212127241 4.620177703831104 -0.0,0.10737314976605918 -1.5716142960765325 -1.0,4.0522289913808365 3.77562942835957 -0.0,1.4798827061781141 1.1638601205648005 -1.0,3.6758023575825547 3.115500589955362 -0.0,-1.803338141681238 -0.639996207387159 -1.0,2.044667029270621 3.04922768663927 -0.0,-0.06067427095346295 1.394611410740688 -1.0,4.626495834477846 2.995800202291488 -0.0,-0.2770274350630315 0.4521526506693692 -1.0,3.130857841268635 3.76858860814448 -0.0,2.163400739017478 -1.303601716798734 -1.0,2.9131896969824367 3.4288919990054167 -0.0,-0.7145108501670207 1.4189762494365543 -1.0,3.535768896041034 1.4894011726406373 -0.0,1.605614523747256 0.29974289519139824 -1.0,2.413678734728178 2.1826316767457183 -0.0,-0.8821932593373774 0.26432786248412726 -1.0,2.0878695933047116 3.5277388966365177 -0.0,-1.107001191509183 0.38421647065699477 -1.0,2.6462094774496454 2.273786785429519 -0.0,1.0712046043765102 -1.1889735666835115 -1.0,3.7458483094910666 1.3868020542832566 -0.0,-0.8403883736429167 -0.7163969561320671 -1.0,3.3359151000342195 3.2382001552279576 -0.0,0.13309387098922537 0.938761191821517 -1.0,2.083439571838502 3.2204948086228944 -0.0,1.3030219848568272 0.5976630914634896 -1.0,2.7602376200551317 2.200505791897739 -0.0,-0.9458633178207942 0.0490955863627428 -1.0,3.7998466026531883 1.9291683955712686 -0.0,-1.327236501803235 0.06915643957270164 -1.0,3.4740573335685925 2.1080735512507114 -0.0,0.8627688253416859 -1.961802291046532 -1.0,3.5108780392869776 3.9854745964798326 -0.0,-0.69537574439301 0.2436269580373554 -1.0,2.920286302932126 4.704192389485899 -0.0,-2.031190954684878 -0.7843052045579578 -1.0,1.6768848711259499 1.345658047606076 -0.0,0.9234894202027507 -0.38179572928866495 -1.0,3.1710339307651334 4.129874876536583 -0.0,-2.5086697007630376 -0.2638692986795807 -1.0,2.079400422215581 3.124756711992435 -0.0,-0.1388012859869782 0.3698243463601514 -1.0,2.665728164475424 4.574860576068532 -0.0,0.11967116650891912 -0.8792117975750646 -1.0,3.042630437105455 2.7245525508413677 -0.0,0.6078023848042808 -0.7977233104047035 -1.0,3.3340709038589638 4.962729210819017 -0.0,0.6373101353982795 1.1335021278327686 -1.0,3.3821397455119446 4.349379573895378 -0.0,-0.9140176931412027 -0.03428220013900756 -1.0,4.579963977595727 3.8322809335521484 -0.0,-0.43958506434874983 0.21259366700539037 -1.0,2.644701808902675 3.945416465403505 -0.0,-1.119921743746522 -0.2089105317801997 -1.0,2.5480553203091922 3.123344220515146 -0.0,0.8723990414181355 1.11150972420879 -1.0,4.479600967837827 2.8645066949820057 -0.0,-0.003869320481891422 0.24756134775982133 -1.0,3.237294368758498 4.642548547098718 -0.0,0.34643329685515545 0.029869480691029456 -1.0,2.6324740490008893 1.2577448307260846 -0.0,-0.4416403319035849 -1.4597062027342758 -1.0,1.764049052224297 3.649850384544675 -0.0,0.6779287737716254 -1.9489876700506967 -1.0,1.4286669812409405 2.4906452014102416 -0.0,-1.2271599940693638 0.9869686407012563 -1.0,3.6244117441765993 2.36879554315985 -0.0,-0.11422653411940642 0.4741905017884626 -1.0,3.6192153991840694 2.149436181779614 -0.0,0.45425900443207484 -1.357987041493406 -1.0,4.312295702128074 3.7596991900930252 -0.0,-0.35153502234686884 -0.6297451691082592 -1.0,3.4901363450669476 2.0630236379093243 -0.0,-1.5343533005821828 -0.23745688647461852 -1.0,4.775056734905926 5.291243824646301 -0.0,-1.032123659747431 0.8458711875294105 -1.0,2.3091889606097844 3.3688150059111215 -0.0,0.7854236849909306 0.6742463927844289 -1.0,3.284779531346899 2.855746734955609 -0.0,0.380579394855332 -1.2378905330462027 -1.0,2.540193014555953 3.245568950444961 -0.0,-0.5491810448400926 -2.3179482776107894 -1.0,3.481785462949587 1.8870182253717969 -0.0,-0.06833732101790825 2.178923334945784 -1.0,1.1663083809702222 1.8919272314310458 -0.0,-0.7801536433937879 -1.4185984368350903 -1.0,1.457713814592066 3.0323739348144048 -0.0,-0.16377716798970973 0.09678021896691058 -1.0,2.2294515799173094 1.6179126855486068 -0.0,-0.5845552895984718 -0.8095679531228397 -1.0,2.024328902209618 2.4660315284543888 -0.0,0.2037503424802764 1.5767438723426828 -1.0,3.5058983262252643 3.292836693091364 -0.0,-1.4004772080893082 0.6150928060180622 -1.0,4.610936499146778 3.3674445809820313 -0.0,-0.7325641160695897 -3.0469742419403225 -1.0,2.6778956983269926 4.049681967443553 -0.0,-0.3375932473421461 -0.32976087151423067 -1.0,3.975838378562512 1.2032482992228626 -0.0,-1.6622711226380826 -0.6954676646542216 -1.0,3.1601568512397256 2.7472491112914357 -0.0,0.6739969973916968 1.3608866192945286 -1.0,3.097978499063888 3.88429576456391 -0.0,-0.16445244300279913 0.631410854999902 -1.0,4.244875698991619 3.0464568222900477 -0.0,0.1749522197766453 -0.3295077792829936 -1.0,4.158913950688044 1.1836177376726964 -0.0,-1.8286320279969996 -0.6355826362111864 -1.0,2.4795264391445326 0.8073937061906746 -0.0,-0.5095499320702017 -0.8451757050184052 -1.0,3.6489546081475206 2.7405880916534957 -0.0,-0.11733097334574003 0.020300758125140466 -1.0,1.9034123919197892 4.036941742254072 -0.0,-0.4678304671259669 -0.7653895561277071 -1.0,2.555027220737054 4.205906511993216 -0.0,0.1952150967011765 1.2402178923240337 -1.0,3.532371144429582 2.395018092924601 -0.0,1.4682834110821084 2.2292327929025078 -1.0,2.1160331256749663 3.7157102308564824 -0.0,1.3973790173654674 -1.1902799121683607 -1.0,3.4775573554170616 3.0459058509488557 -0.0,-2.215337088722839 0.7693588032777773 -1.0,2.3298220860458976 1.5924630285528396 -0.0,1.260641664088144 1.5474089692944746 -1.0,4.460878990061944 2.595950219349794 -0.0,-1.8214944389802914 -1.9733205363211535 -1.0,4.41874870213851 2.4975116019313264 -0.0,1.2037921250123007 -0.7057578432831773 -1.0,3.042628088030598 3.7366256492570136 -0.0,-0.02609770715133313 -0.01975791007372346 -1.0,1.123824442324706 3.5115607224884466 -0.0,0.3466005704292144 -1.206858960323042 -1.0,3.044152779557358 2.4308738719304266 -0.0,-0.8292396838183249 -0.5768591341562801 -1.0,2.9898679252543325 3.3291086316901484 -0.0,0.6033357093153775 0.18738779274832332 -1.0,3.2777482224094916 2.2676548172839714 -0.0,-0.7104360487845565 -1.0365712508175688 -1.0,2.617802272534323 1.887796671556582 -0.0,-0.21008998836798706 -2.4424443035468957 -1.0,3.9387085143031317 2.368798316318223 -0.0,-0.65027380204969 0.4757828709083824 -1.0,1.6786020855223545 1.62019388696364 -0.0,0.40325101156361803 0.26629562725726075 -1.0,2.4614637796912167 2.778406744842399 -0.0,-0.4327374795655596 0.5643009301153851 -1.0,2.6419358755663103 2.1911675067034206 -0.0,-0.06058610052148417 0.6118154934715632 -1.0,4.134485645832481 4.214482766162727 -0.0,-2.091472947105952 -0.21279450874188077 -1.0,3.7664041746453503 0.5848083052756543 -0.0,0.20187441248519114 0.7310035835212488 -1.0,3.6821251396696817 1.2016937526237272 -0.0,0.16248871053987612 -0.8547163523143474 -1.0,3.1725037691095834 3.051265058839004 -0.0,-1.7466975308858639 -0.048497170816597705 -1.0,4.296665913992498 4.432036327276331 -0.0,-0.49371042139965376 -1.3162216335880739 -1.0,3.0767376272412292 2.4082404056282467 -0.0,0.6517145281009619 -0.15229289422910688 -1.0,3.8556129079007406 4.932746403550176 -0.0,2.467072616559744 -0.6570760874457315 -1.0,3.8722558954619446 2.398547361219584 -0.0,-0.996362973160808 -0.24663573264285635 -1.0,2.058960472055059 0.09020868936476445 -0.0,1.1921444033047794 -1.2205820383864918 -1.0,3.499255855340612 4.26015377680707 -0.0,0.46495431359796363 -0.3535071804767937 -1.0,3.2772715993311534 1.8496849599545144 -0.0,0.9200766227075026 1.0153595739730128 -1.0,3.7395665378166516 4.161859093428991 -0.0,-1.3445731221950805 0.3711182438638966 -1.0,1.974184816991473 2.3758202020218637 -0.0,0.25747673028745044 1.4898729695115611 -1.0,3.643667737073963 2.5171980898063024 -0.0,-0.7491175934837044 1.807998586131331 -1.0,3.024294668483263 2.745713910567566 -0.0,-2.9902104324990075 0.48847563269083094 -1.0,2.693457241550706 4.067192099378729 -0.0,1.0010822910854564 1.065617155304199 -1.0,2.6231328305267576 3.2530925652040796 -0.0,-1.569524799794976 0.10080365850268516 -1.0,5.543177898986999 3.149276748958176 -0.0,-0.2697035609845456 -0.3834981890675749 -1.0,5.5737716796876935 3.134627621089238 -0.0,0.16848836970122472 1.7680681560270155 -1.0,2.984578320659214 3.8081853301923743 -0.0,2.00864307305994 -1.1769936806590435 -1.0,2.4301644281026538 1.5357007015355957 -0.0,-1.251515087462618 -1.0023388301407077 -1.0,2.7783106123714036 3.4753675099443138 -0.0,1.2067779830446301 -1.1138369735803868 -1.0,2.660559526103853 0.9246419639107195 -0.0,-0.2120078291751072 0.553871125085326 -1.0,3.2961674182984613 4.1840551114889655 -0.0,-1.7407002661640898 -0.13494920714243758 -1.0,2.61652747199719 2.606431158365525 -0.0,0.1810536358726569 -0.7041543708042312 -1.0,0.6618977487425206 4.43976232230529 -0.0,-1.1056190552516114 -0.26273698119076755 -1.0,3.245745718364984 0.9585399121419127 -0.0,0.451245033031027 0.3966692171364385 -1.0,0.7000962854359294 2.5787278270774685 -0.0,-0.20657738352563298 -0.3054434424581368 -1.0,2.194893094322135 1.2265276851138993 -0.0,1.6478689673866447 -1.2217538409516264 -1.0,2.6520153534620268 4.253943157694819 -0.0,-1.091459682813003 -1.5933476790183565 -1.0,2.381978388803204 2.5725801073346375 -0.0,-1.7089448316753346 -0.40058783295112843 -1.0,4.692976595302646 2.293610804758882 -0.0,-0.8154594160076379 0.9100123432125261 -1.0,1.8893957859271135 2.365552941116367 -0.0,1.4750445045587657 -0.5730495722105764 -1.0,4.627946484342315 4.01023129091373 -0.0,-0.5740578222548407 -0.9010801407945085 -1.0,1.1844352711236998 1.0077910117111921 -0.0,-1.1904557430938465 -0.972229300373332 -1.0,1.9514043869587852 2.6603232743467817 -0.0,-0.11744191317950421 1.8160954524210857 -1.0,2.796337014232012 3.45131164191957 -0.0,1.1908754571951825 1.37388641966138 -1.0,3.1347230127964805 3.4874636513372774 -0.0,1.4279445191621287 0.4142573535049987 -1.0,3.2845746999649457 2.942571828876143 -0.0,1.0418078095097314 -0.515727237947711 -1.0,3.0672407807876674 3.593602465858237 -0.0,0.1070041194341431 0.013584199138111364 -1.0,2.831124413123504 2.5083468687281196 -0.0,1.9088191143015583 1.1943157723052062 -1.0,2.888463730373365 3.8588231186101716 -0.0,0.3344825700647222 1.4902421889158837 -1.0,5.1805240354926285 2.347000348613805 -0.0,-0.14736761539184529 -1.3764336595247777 -1.0,4.945788020165247 4.520764535128319 -0.0,0.48089579766964224 -1.0406729486881927 -1.0,3.115699146536788 3.0271206455481905 -0.0,0.8816867514268375 -0.7885530518936628 -1.0,3.293642905051253 4.129500570671647 -0.0,0.021019117419869213 -1.0983625263034136 -1.0,3.4712873315273884 2.8896550248710255 -0.0,1.336463967380889 0.1782538924176004 -1.0,2.9674559623039674 2.1702990000666977 -0.0,-0.9137873001694705 -1.6488427315604255 -1.0,2.425720985355789 3.336546225859983 -0.0,-2.3622279944776245 0.33443034793657744 -1.0,3.557057454549674 0.9654984504665607 -0.0,0.4924227412613347 0.8572441753897001 -1.0,2.903599258175698 1.9821387894597133 -0.0,-0.562864152759892 -1.41025535274598 -1.0,2.621542267864135 3.0896861639721602 -0.0,-0.9659016052287058 1.8601390770202668 -1.0,2.73394050343452 1.5908844566159697 -0.0,0.316736908826005 0.2857224419323005 -1.0,2.3312567009140532 5.596694984859762 -0.0,0.3137619371424862 -0.1840942808000176 -1.0,3.857644883242267 1.7425846536145542 -0.0,-0.10204795362718587 3.253153279848385 -1.0,1.991635750012152 3.0091345292604816 -0.0,0.6187841242310289 0.9589700354301842 -1.0,2.9773010080735895 3.723750625441197 -0.0,-0.8890787476930039 0.6057780620635984 -1.0,3.2341068438464773 4.238588226643048 -0.0,-0.6100941277292691 -1.5125630779121992 -1.0,3.378840902739636 2.0705801293719017 -0.0,1.9736225258875286 1.725383750563661 -1.0,1.8874237286900284 3.9061132751393997 -0.0,-0.0823939289302894 1.8958431169469556 -1.0,1.5927855001333566 4.6310125064091965 -0.0,0.3112044157520983 -1.7878471816057036 -1.0,4.34881513764263 3.4693940014863784 -0.0,1.052103622850019 -0.16912252356217902 -1.0,3.167179956507673 2.8792495587252507 -0.0,0.16791453003538387 -0.8546142448164881 -1.0,3.0538805073215953 3.4494667407676842 -0.0,-0.9500475678227512 0.06998146933806365 -1.0,3.8909913837847467 2.6813428719208763 -0.0,-0.09976816220585052 -1.4875944011133129 -1.0,3.1791447205478742 4.424991854067018 -0.0,1.0999643223476656 -1.1200747827607145 -1.0,5.222367041159025 1.2015274537211948 -0.0,-0.2848179798736651 0.401703345435371 -1.0,3.92690552314874 0.5307127426832543 -0.0,-0.6771410319499919 -0.5806616553853885 -1.0,3.611779415106116 3.3322298911093533 -0.0,-1.359189339369671 -0.03773529290863042 -1.0,4.696002594470123 1.4346348756461187 -0.0,-1.0094856636150293 0.19687532044013809 -1.0,3.2169383066148383 3.2307201581236473 -0.0,0.7836015359045666 0.2941037782687062 -1.0,3.7317041306588012 3.7985843457251107 -0.0,-0.3693168101963429 1.4513472421644549 -1.0,4.398703283685875 2.654636797434109 -0.0,0.02043081741683321 0.20805199015337653 -1.0,2.324187503797731 3.8819865944906566 -0.0,1.671377007435211 1.3731572027338659 -1.0,4.534630721644852 1.1543799480085444 -0.0,-0.3253127279932509 -0.8285225286171498 -1.0,3.993821155042294 0.7056403589045206 -0.0,1.194500226045371 0.638917136862092 -1.0,2.72148063695256 3.858678264350294 -0.0,-0.1905653672336637 0.8969404368665279 -1.0,1.9587911397509248 3.937696894952624 -0.0,-1.1358853052995896 1.4443151501322575 -1.0,3.7551091652428026 2.475478572543473 -0.0,-0.9167034706173607 -1.7549316646340103 -1.0,1.4669571532496661 3.2025879996118567 -0.0,-0.9673112226998997 0.13104324478779786 -1.0,5.129589009385082 2.962228456981596 -0.0,-1.038791699676283 0.3394661925580474 -1.0,4.0067362767396055 3.7808733451013863 -0.0,0.4607763000001474 0.3165842402170894 -1.0,3.470781763864157 3.1917117382789906 -0.0,-1.0759836593672722 2.1677955321765423 -1.0,1.8061608083541592 2.1368201192592524 -0.0,0.18913968729195288 -0.6832055159990379 -1.0,2.222086435460701 2.462434683952491 -0.0,1.1697195016246194 -0.6482703204844716 -1.0,0.9469729137532825 2.564223951962673 -0.0,-0.2596612587018774 1.3675954564898984 -1.0,3.3498722540414603 2.8411678301395655 -0.0,0.15549061976540607 -0.8795816620250406 -1.0,3.2166810907529517 3.3909740833940147 -0.0,-0.27777898312342497 1.5708467895548373 -1.0,3.5590852623593734 3.022687446035052 -0.0,0.8854804450462548 -0.1674059547432505 -1.0,5.592380230543062 2.046846128948299 -0.0,-0.38403645419139704 -0.6879614453050698 -1.0,1.2059037878354082 3.1373448113023263 -0.0,-0.9332349591768346 0.3271191223126651 -1.0,2.6941262027196444 2.0016455336591275 -0.0,1.985628476449888 -1.720937514961405 -1.0,1.52678578836386 3.6524268651279113 -0.0,0.14930924959259012 0.3549736192569231 -1.0,2.5081810800507904 4.502494324423253 -0.0,1.3659157029970181 -1.4064298168920828 -1.0,2.8947698041280185 3.871692848909248 -0.0,-0.19002791703482588 0.8099829390725909 -1.0,3.0481549176670555 4.05245395484312 -0.0,-0.014729952199541938 0.43445426055411474 -1.0,3.0874888030440486 3.89317889717026 -0.0,0.9521743475193137 0.16292125350371375 -1.0,3.0564028575123805 3.150394468127784 -0.0,-2.5565867181635724 1.1693524400747453 -1.0,3.963399476624186 2.655863627219969 -0.0,2.0594134768376584 1.4326082874689938 -1.0,3.9415985004601524 4.816989711315565 -0.0,0.4986273362656531 -0.30506819506279537 -1.0,2.7697598834307633 2.0292290332215512 -0.0,-0.4716043983943112 1.4692631198715722 -1.0,3.4127279940145883 3.078218915501194 -0.0,-0.28649487641740207 -0.8009455078808752 -1.0,2.645854233845017 4.028461076417125 -0.0,-1.2333241385253426 -0.2850384355482007 -1.0,2.4938754741404976 1.3466482769013481 -0.0,0.6872021385233428 -0.5159203960430369 -1.0,3.136974388668967 1.69291587793452 -0.0,0.9532239280401443 2.619265789851879 -1.0,2.570576389986536 2.548658346643033 -0.0,-1.030037965987706 0.2814883160676786 -1.0,2.510605023939257 2.3227098241155213 -0.0,2.4171507836629256 1.245606490445435 -1.0,3.5520681299250985 0.7442734445298673 -0.0,1.1940577980770877 1.6319950123919318 -1.0,2.708933998825159 2.118496371335553 -0.0,0.26808250222082186 2.5727974909556437 -1.0,3.221534693193204 3.073316472650363 -0.0,-0.6915734756410544 0.25168141600713434 -1.0,1.839319878312068 1.765565689559382 -0.0,1.708990562782385 1.1196517028520787 -1.0,2.1942131633492643 3.733776318231434 -0.0,1.4884941762679373 -0.5221400677305167 -1.0,2.425026062564176 4.814343944240822 -0.0,-1.3572570451352999 0.04542725800519613 -1.0,3.211869589232063 0.01498355271713292 -0.0,1.6170759581287553 0.7420944718274473 -1.0,1.8096883146020295 1.2063063122336204 -0.0,0.8326608996906895 -0.9760063002065638 -1.0,3.60415819299222 3.905143144181063 -0.0,0.9709971797789466 -1.0644382680658016 -1.0,2.8104103693138778 3.5792951568581017 -0.0,-1.021059644329913 -0.25967578007654707 -1.0,2.4020556940935216 3.8705560506781826 -0.0,-2.704107564850001 -0.14300257306795375 -1.0,3.7681081908063643 2.5433599278958297 -0.0,-0.537043950598385 0.8892208622861 -1.0,3.894301374710518 2.76168141850308 -0.0,-0.8416385593366815 1.3377079857054535 -1.0,1.4560861866861152 1.9464951398785584 -0.0,0.8974462212548237 -0.9027814165394935 -1.0,2.848274393366227 4.089266410865265 -0.0,-1.9874388443190703 -2.0515326123686 -1.0,1.7443330286532606 5.182730816947559 -0.0,1.9345124573698136 0.15482916596109797 -1.0,3.730890742221753 3.4571088485293173 -0.0,-0.7591467032951466 0.7817400181511722 -1.0,1.9612060838774241 1.7874104906670758 -0.0,0.04241602781710118 1.7624663777014242 -1.0,2.983106574446788 2.057794179835603 -0.0,-2.2675373876565272 0.1810247094230928 -1.0,1.8242036739605434 3.2897838599534053 -0.0,0.42135250345103276 0.9201551657148959 -1.0,2.3324158301116547 3.2735600739611406 -0.0,-2.503382611181759 -0.604428052499623 -1.0,2.1068571110070753 1.3987709205712464 -0.0,-0.25006447102137164 1.1597904649452788 -1.0,3.6610503210650105 2.389802330720335 -0.0,0.6655774387829471 -0.7657689612002381 -1.0,3.85820287126228 5.653287382126853 -0.0,0.08244241317513575 0.4755361735454262 -1.0,3.6029514045048234 3.0483730792265247 -0.0,1.0276000901424318 -0.569237094330588 -1.0,2.484863163042475 3.4464671311141046 -0.0,0.24588867824456415 -0.7355421671684942 -1.0,2.8757627634577396 1.3730139621444188 -0.0,0.911649033206053 -1.0562220913143838 -1.0,0.6701966948829261 3.8815519088585195 -0.0,1.0649444423673609 0.5738944212075908 -1.0,3.1272553354329955 5.18450239514651 -0.0,-1.8305691156390467 -1.2811179644895232 -1.0,4.326027257587544 1.9589219729995737 -0.0,-0.2278417247639679 -0.6436775444106994 -1.0,3.9854139754166136 2.8662622299102947 -0.0,-0.33177487577648573 0.7122237484053809 -1.0,2.7631237758865255 2.490470927953921 -0.0,-0.2989203275224733 -0.9063254275476191 -1.0,2.7739570950234254 3.333596743208583 -0.0,-0.12025132003053318 -1.2251715775331837 -1.0,3.9028268386113307 2.580334438085556 -0.0,0.3114518803226873 0.35489645702286177 -1.0,2.8765994073916112 4.251640702192294 -0.0,-3.0895947568085367 -1.0526550179589378 -1.0,3.5182345295490216 2.764855512391279 -0.0,0.5749621254042305 0.7148834016467635 -1.0,4.039448299164001 2.377396087740471 -0.0,1.7077800661629936 -0.23711282974122355 -1.0,2.883211311171089 3.5259606315833287 -0.0,-1.0304518163976537 -0.16271910447066004 -1.0,3.8284470175501504 1.0841759781704199 -0.0,-1.3620621426919217 0.8678141368192274 -1.0,3.831976508070298 2.3592788803510505 -0.0,0.8398199934902235 0.8458121179021545 -1.0,2.166979759191688 4.408250411844058 -0.0,-1.2009412161006234 -0.04486968047943732 -1.0,3.0041897020427517 1.67577082931885 -0.0,-1.0550850035108499 2.6114061208535673 -1.0,1.46399823823424 3.6863318429400627 -0.0,-0.439942118867861 0.8107733517611471 -1.0,2.799907981207793 3.1021389011201244 -0.0,0.40512996190803663 -0.2720769110918539 -1.0,2.936414720731187 2.6121553148876706 -0.0,0.7864503163458285 0.879685137879171 -1.0,3.497848931993103 3.93953696354328 -0.0,1.0898800025299487 -0.3780987477521812 -1.0,3.0737866861658834 3.8281246288654067 -0.0,1.0100369320198321 -0.36412797089680377 -1.0,4.977156552398557 1.9361263628969327 -0.0,1.1948682006514484 -1.0421380659408503 -1.0,2.3707352395183743 3.319087891488442 -0.0,0.14662871945444525 -1.125277513770441 -1.0,4.18636170602371 5.079790109963499 -0.0,0.5213830491310841 2.5489667538554355 -1.0,3.456121838657517 2.9777488007628823 -0.0,1.3942157902546204 -0.7392170745991694 -1.0,4.027857416272539 2.5520251242493615 -0.0,0.6677437543225546 -0.7054702957392922 -1.0,2.419993627501343 3.147115729790262 -0.0,-1.1891285195785104 0.7121837556662985 -1.0,2.6768950566988114 2.746092902448666 -0.0,-0.5581632736462642 -0.8475377022167101 -1.0,2.2877649074222144 3.360822129377224 -0.0,0.12427410923130733 -0.029877611579596446 -1.0,2.1363649823278976 2.040672619624904 -0.0,0.164296403698455 -0.7853340225962958 -1.0,2.2867454265483063 2.920796736914219 -0.0,0.030938689766481568 0.02840531713718885 -1.0,4.935402862397514 4.984097800264938 -0.0,-0.49323021214001667 -0.009344009957387383 -1.0,2.2590589178865788 2.784700488476081 -0.0,-1.7996451721642797 -0.08927843209025701 -1.0,2.7189425454136047 3.366984002518318 -0.0,-0.4732503966611213 2.41667617281343 -1.0,1.914172722581019 2.723688261246487 -0.0,0.6854209215843875 -0.6321377274037409 -1.0,4.7025333481932705 2.6561807763401646 -0.0,0.016511529980536163 -0.4064291762993186 -1.0,1.3841179371371182 3.367159685928979 -0.0,-0.525665902025766 0.3189849885462113 -1.0,2.1237941386456276 3.4141040859263914 -0.0,-1.3977733609952327 1.6180332199555512 -1.0,3.3282228318571496 2.9879449742002184 -0.0,-1.3911999737510374 -0.47876736354905697 -1.0,3.071461319022103 3.902142645231827 -0.0,-1.4616870328596612 0.4234223737141411 -1.0,3.3069543201402576 1.3522887907099401 -0.0,0.1771175002160632 0.7092577154896049 -1.0,2.561517669553921 3.2663130772229185 -0.0,0.8635080818806004 1.7578935533355913 -1.0,3.3054989034355793 3.4205399612822633 -0.0,-0.5525474134214131 -0.008874526853035592 -1.0,5.024607965706471 3.377256085775693 -0.0,0.6499316691799448 0.7636813929956143 -1.0,1.7211648540475015 3.7290596058136307 -0.0,-0.4312096678787339 0.4723353140241522 -1.0,1.6269397815780402 1.9613109767814954 -0.0,0.06589250830042476 0.5659627954925366 -1.0,1.4141705667382305 2.9411215895612255 -0.0,-0.30655047441372724 1.134312621267185 -1.0,4.079371134159225 3.7127217011979767 -0.0,-0.11148410319718746 1.504423362990177 -1.0,3.21908765035085 1.5284527951297098 -0.0,0.38879874604519066 -0.7718569898512835 -1.0,3.0387686435299197 1.9571679686339727 -0.0,0.0432538958325193 -0.609046739618082 -1.0,3.858513576900389 2.3343789318227595 -0.0,-1.594606569379673 2.0291869081775498 -1.0,4.418575803606943 3.634284954659144 -0.0,-1.5657043498774568 0.48528442006547645 -1.0,3.7474369990653518 2.417108621170513 -0.0,-0.4087178618516316 -0.5585629524971241 -1.0,2.8830052178069345 2.714807180476644 -0.0,1.0200529614238536 1.633454495011907 -1.0,2.161101444560085 2.722233198993495 -0.0,0.8905571055499505 0.3531260808046299 -1.0,1.5770402091220281 2.5197577954902615 -0.0,0.19603489193696402 0.4391781215510938 -1.0,3.285302297900197 2.5981032583297274 -0.0,-1.7728311957227578 2.226646036588897 -1.0,2.212402423781055 2.994783519362575 -0.0,-0.26351331835428804 0.6197161896115081 -1.0,2.5101464936050144 2.747453537535198 -0.0,1.083443472210967 -0.7471502465676395 -1.0,2.618022142084275 3.201094589808021 -0.0,-0.10243507468644107 -1.5307780048431203 -1.0,2.0479014235932986 2.7174445598757764 -0.0,-0.2530316183327909 1.5105959457792464 -1.0,2.616239369128394 3.1011058356715644 -0.0,2.0703487677159997 -1.23039689097027 -1.0,2.00559575849234 3.088170264353322 -0.0,0.751453701775929 -0.34079600956200146 -1.0,2.6436129383324625 0.6934715851263205 -0.0,0.4735774669250165 0.24981500600111478 -1.0,3.614102521076285 3.297655445774221 -0.0,-0.8397190394129946 2.0791729859494583 -1.0,2.5800847823336372 2.312770726398467 -0.0,0.9528690775719402 -4.054641847252764 -1.0,1.6631425491523402 4.465488566725185 -0.0,-0.40442215938144854 2.1662912065078923 -1.0,3.2025444402071472 0.954639816329502 -0.0,0.8484611241529962 -0.6531501762867838 -1.0,2.907155165379039 4.494838051538261 -0.0,1.1473298350419248 -0.7604213061923158 -1.0,4.406872541176625 2.616395889868952 -0.0,-1.0643453307576694 0.32269083514118757 -1.0,3.4229771635424653 5.404174358063928 -0.0,0.8223012341648268 -2.0705983787489455 -1.0,0.6519219290294926 3.317297519573949 -0.0,0.6661739745821234 0.21368601256080724 -1.0,2.8092516816651187 2.9407143882873363 -0.0,-2.0396349059310626 0.6660958962860263 -1.0,1.621401319049101 2.120514741629026 -0.0,-0.6673242389540511 -1.033336539766657 -1.0,2.4729967381312257 2.0622671692969314 -0.0,0.318696287733599 0.7696143248064906 -1.0,-0.3310542190127661 2.503572170101248 -0.0,-0.024545405442632163 1.2826535279165514 -1.0,2.08361065329982 1.7709137020843035 -0.0,-0.03325908838419148 2.127731976717063 -1.0,0.8920712229737089 2.267227052639782 -0.0,2.4226620796703706 -1.5422597801969735 -1.0,2.6125707261695665 4.136941962252239 -0.0,0.710000430684373 -0.2365544035810329 -1.0,3.587983407259662 2.371118916918134 -0.0,1.548716105657387 2.6039797648647527 -1.0,2.288647833469394 2.8514285941696564 -0.0,0.5407956769257948 -1.4250712589214616 -1.0,3.9999271279969157 4.647262641336589 -0.0,0.46916438504363506 -0.16114805677977867 -1.0,3.9351714928555133 3.017851089635014 -0.0,-0.24683125971847 0.8686956304798523 -1.0,2.445900548419883 2.601998949302925 -0.0,0.9708272515136681 0.9540365110832763 -1.0,2.0889493306284472 1.670700190658552 -0.0,0.7573519355244429 -0.6731075400854291 -1.0,2.9938559890272676 0.5796453404844417 -0.0,-0.42350233780111274 0.1072223004754211 -1.0,3.22502989165533 3.2744724666391045 -0.0,-0.051171179793716125 0.035749085667007977 -1.0,4.256076524642883 3.956646576238979 -0.0,0.44715068158575316 -0.10904823199444005 -1.0,3.754239074295241 2.4862504435534283 -0.0,-0.12025734941101636 0.6682754649328633 -1.0,2.9673795614648815 3.6207880514009263 -0.0,-2.250093626462795 -0.49148713538228506 -1.0,1.7335315087131171 4.234455598757855 -0.0,-0.5145677322324603 -1.8872464244504652 -1.0,3.1524408905920547 2.534903833671654 -0.0,1.4188237424906527 -1.987300018397619 -1.0,3.025903676999244 2.1652631630581847 -0.0,0.5008343534015861 0.28011601768758965 -1.0,2.0039218613662197 2.3639397631018015 -0.0,1.342528231824729 1.0036076495884643 -1.0,3.3281244751369985 2.4251038991267277 -0.0,-0.38845861664115766 -1.5147629282596704 -1.0,2.613448357242925 4.463712912575443 -0.0,-0.19439583983218703 0.676381234314577 -1.0,1.0400516553104269 2.3981508685333424 -0.0,0.9469554018478826 -0.08144910777086176 -1.0,3.179705969662961 3.768848690124549 -0.0,0.39855441813668835 -1.6301847736954416 -1.0,2.1915941615815226 2.7947789889097763 -0.0,1.6023287643577222 0.05432794979410767 -1.0,1.5758610206949497 3.8709473262823777 -0.0,-1.3109119301269387 -0.8645189055395048 -1.0,3.715865055565244 1.9360512196442488 -0.0,-0.2073998491467907 -1.178882579876182 -1.0,2.565062666629786 2.3121370465462494 -0.0,-0.41397768670851737 -0.6674761320605563 -1.0,2.941938460212705 3.537877403937825 -0.0,0.5954231185191001 1.6839554319972647 -1.0,4.591360208911688 1.4381368838271187 -0.0,-1.3221878199013057 0.786799353955043 -1.0,0.6498018470693379 2.2143413646510095 -0.0,0.5346452265922554 0.45599002729248733 -1.0,2.668100742914233 2.679883986650412 -0.0,-0.22428284967184606 -1.0003823373608314 -1.0,4.233871998643562 3.3423521548333897 -0.0,0.7800144346305873 1.6512542456242612 -1.0,3.3192955924982677 4.664828345688715 -0.0,-0.9059493298933676 -0.42207747354389447 -1.0,3.1776956110847916 1.1393123509452483 -0.0,-0.5246202787832872 1.0246845701853746 -1.0,4.732113325540828 1.29018271893586 -0.0,0.9863596225434407 0.7506968948666005 -1.0,2.911409852038849 2.626474556246977 -0.0,0.8545346747310709 -2.1711133879380955 -1.0,2.476689592134109 4.03136160709651 -0.0,0.43108249592457043 0.4589971218864913 -1.0,3.2333287857145825 2.188137362144206 -0.0,1.4405649581445525 0.4131214094941824 -1.0,2.0631468420251093 3.807898318807702 -0.0,0.43964401099781425 0.6669437158150616 -1.0,2.165843657939062 4.109647016182597 -0.0,-0.9735452695016392 -0.6172105570335473 -1.0,3.169794653766589 3.2721053734106 -0.0,1.3129166037688875 -1.2040138532590103 -1.0,2.211361701514339 1.025981622029549 -0.0,0.3653350359702278 0.5229315457444437 -1.0,3.372206428302252 4.163685355869495 -0.0,-0.8690030167652726 0.3226849491596335 -1.0,4.188509026227427 2.1137749377457076 -0.0,2.2174789916979933 0.8249932442083762 -1.0,3.9224824525785706 2.9436443006575925 -0.0,0.1370905200148926 -0.043320354739616776 -1.0,3.1118662077850807 1.4983207834379917 -0.0,-0.5304073850344787 -0.4219778391981189 -1.0,1.2153552376808336 3.4749521622043438 -0.0,-2.545970043914331 -0.5480647959096547 -1.0,1.8097968872175412 4.733523163055134 -0.0,-0.5599306916727819 0.4648015112295201 -1.0,3.0242901796172204 4.354893518146392 -0.0,-0.49175893973189483 1.8635231981223406 -1.0,3.923889822736733 4.199324033436554 -0.0,0.32931083529824645 -1.2038529291812745 -1.0,2.8430570026355904 3.2581768028655214 -0.0,0.08015643729775149 -0.5281238499521005 -1.0,1.0251176552841985 2.452443183841665 -0.0,-1.4000614002792062 -0.4723026702712555 -1.0,4.642753244692533 3.5777684251625153 -0.0,-0.9732069449126244 -0.7507666182081589 -1.0,2.284811103731081 2.6226837934175817 -0.0,1.4938320459354653 1.2271703303402608 -1.0,2.5217907633717935 1.9804499278889345 -0.0,0.9177851256816916 -1.196945923903535 -1.0,2.650515007788954 0.9818159554114416 -0.0,-0.4172435945582116 0.11930551874205601 -1.0,1.8203127944592765 3.3069324017397594 -0.0,0.08195935202288789 -0.2585763476071969 -1.0,2.14910426585678 4.146147361847687 -0.0,1.578290774885182 0.16149960053586573 -1.0,1.2607405323635168 2.940350340912184 -0.0,1.6722138822230346 -0.5454073192477626 -1.0,0.3769561517619793 4.029314828130509 -0.0,-0.012008811772440746 0.2577932550827986 -1.0,2.330909580388283 3.1650439747088024 -0.0,-1.4224384024201595 -0.6369918128076046 -1.0,3.451178380794735 2.7553545272536746 -0.0,-0.7913135079702314 -0.012217405089490006 -1.0,3.7918310740082424 3.3927876820084033 -0.0,0.41016650792928255 0.3521369094279198 -1.0,2.380867149491576 3.7533007228820754 -0.0,-0.2787273586680994 1.3553543015884186 -1.0,2.8933236071325226 1.7975563396445144 -0.0,-0.4868680345968448 0.058461169788172784 -1.0,3.484434144626577 3.5622013162506683 -0.0,1.171904838026115 0.1162839888503951 -1.0,1.8132727587691455 2.238018140780368 -0.0,0.8114997821213137 -1.712768034302675 -1.0,2.977061410695451 2.802894970831404 -0.0,1.7141760742336318 0.5672102391229309 -1.0,3.2929421353515185 3.3754831695793945 -0.0,-2.280170614413754 -0.4912881923146271 -1.0,4.182771547422101 3.5331418354105812 -0.0,-0.2544453921577854 0.4682744998445509 -1.0,1.9236524545763007 2.628837510538455 -0.0,0.6645491524745186 -2.398604366119661 -1.0,3.50840713613987 3.7182332137428955 -0.0,-1.4532823239751684 -0.9916580822162051 -1.0,2.769613688635247 4.72661442603805 -0.0,-1.090104082054257 0.486265921887567 -1.0,3.4900626627065003 3.03025323652533 -0.0,1.4518716691137106 -0.10218738652959546 -1.0,2.745034544461333 4.366809709694589 -0.0,-0.17197050309086373 0.13673125942508174 -1.0,2.4934379443680985 2.954734256628178 -0.0,0.14078971520128297 -0.5401300324197861 -1.0,3.640563349517043 5.163454382169049 -0.0,1.0264020194022627 -0.8738489740165843 -1.0,3.791458514669831 2.2038333093620834 -0.0,-3.075231830613813 2.04054404065675 -1.0,4.647422323558612 3.5220753128741427 -0.0,-0.6423734479152313 0.5403500050100541 -1.0,1.5985339514690007 2.73447434771563 -0.0,-0.04474684215568748 -0.21477212224970194 -1.0,2.6701891009654792 3.9776885659794505 -0.0,-0.4714276238216119 1.4235807729101415 -1.0,3.5551789183755806 2.7057825768035104 -0.0,1.108254774651522 0.8596053056731966 -1.0,3.0623366138774983 2.718494058918926 -0.0,-1.375827910513567 0.011994162356159788 -1.0,3.841407434840553 2.8434319292302304 -0.0,-0.7149712282755271 0.1811986378283469 -1.0,5.155524316715826 2.1468464150279747 -0.0,-0.06822014690491127 -0.15801546435311806 -1.0,3.4838423066641173 4.211572262022802 -0.0,1.455177312877137 -0.9388697017811595 -1.0,3.917344840727481 3.569507254920478 -0.0,-2.080636526173827 -1.2489913979804321 -1.0,4.904327940183608 3.4289745068714295 -0.0,-1.4744723958060084 0.2930577753686633 -1.0,2.810346752831796 2.4062885063635333 -0.0,-0.17365054648101302 -2.26263747840141 -1.0,4.077713960215311 3.841309768575811 -0.0,1.581178479362914 -0.9672846912018417 -1.0,4.516244757634386 2.9078781629204054 -0.0,-1.5890391289381882 -0.4092245513024253 -1.0,3.359480708344044 3.7375262649030123 -0.0,1.5675385032786122 0.9010632060589036 -1.0,3.8564874267647644 3.060660915266198 -0.0,-0.2482500870678099 0.29655946916337894 -1.0,3.1672692968701397 1.1973226392521306 -0.0,-1.4471523637168304 0.5370395414503478 -1.0,4.814859889188941 2.229750617440331 -0.0,0.2812295731325761 0.6044036116090106 -1.0,2.4884527354338903 1.4171627784171204 -0.0,1.173099753717184 0.7948729712563257 -1.0,1.5092479631180256 4.1412277875509105 -0.0,-1.1453508695714685 -0.15567849492271865 -1.0,1.9397046305500465 3.430755367623314 -0.0,-1.6689604208958047 -1.161942047896626 -1.0,4.287905082572467 2.643797664646416 -0.0,0.5691715436318573 -0.6013793142266736 -1.0,2.622904412483301 1.769830678112635 -0.0,-1.0627706066421603 -1.2962746926911266 -1.0,2.5818494635089886 2.9547836545958663 -0.0,-1.555832778500785 0.6050365213516793 -1.0,0.6877755924513469 3.0627330470806617 -0.0,-0.6945984937358738 -0.5355659085722678 -1.0,3.631758943383 2.6990914911890194 -0.0,-0.10204034384758799 1.2650405538373874 -1.0,2.8618200471403488 2.7676923144816237 -0.0,-1.2337428464512885 -0.7151041760567872 -1.0,3.5209869997316807 3.280763138579491 -0.0,0.3700095159793621 -0.8614396246939711 -1.0,2.698616090611572 3.2205340189872795 -0.0,-0.8069663812258417 -0.07956402748767083 -1.0,2.929873320056276 4.030067053746698 -0.0,-1.2316919288622938 1.245687935224532 -1.0,2.9285679560367055 2.9682906465530783 -0.0,-0.3965578686363537 1.1748126835359254 -1.0,4.002714110052464 4.370338584188975 -0.0,-0.6084107635744659 -0.6092872315132073 -1.0,3.293912876563504 3.5843332356258464 -0.0,-0.8145032742370918 1.4050967895930515 -1.0,1.991600071099763 2.343264260750465 -0.0,-0.9433799779882722 1.5943129187456013 -1.0,2.369037146473894 1.9827898318071764 -0.0,-0.26885731570182714 0.47421918725401946 -1.0,3.263006333756187 3.0441051541001443 -0.0,0.21785408377528742 0.5754303556190559 -1.0,2.941128899266118 1.240818619804987 -0.0,0.736142634408259 -1.3173589352849961 -1.0,3.2027184783050644 2.9218716893221766 -0.0,1.9216539101612737 -2.2400666381338694 -1.0,2.4823406743823426 3.429705681271458 -0.0,0.0666674809216063 -0.976496437708073 -1.0,3.206108328915537 2.0828009180110976 -0.0,-0.11582094814525531 2.5093876016868366 -1.0,2.5373176496966328 2.32926952602907 -0.0,-0.9237765727032562 0.9342845305943139 -1.0,2.5300867778672123 3.2754703213122753 -0.0,0.13837351460348038 0.2533025702882705 -1.0,4.556185356940701 0.7629684714626066 -0.0,-1.8251759895063635 0.6966019254550819 -1.0,4.905392053322123 4.111245902434462 -0.0,0.09886105139472441 1.4093224263552915 -1.0,2.0484713074013223 4.874632770975326 -0.0,-0.040609033066195156 -1.3446008307073973 -1.0,3.678642687565624 4.156505531118834 -0.0,0.052003196801406706 1.2239229001362555 -1.0,3.4376496474012876 2.417529764306501 -0.0,-0.09054032070414311 -1.7571173217955876 -1.0,3.230032966809188 3.5965216835420546 -0.0,0.9100014718072797 0.5615698517199065 -1.0,3.938728443662248 3.2945250621813273 -0.0,-0.9205165004286314 -0.01425448590777016 -1.0,1.907285344344031 3.8629943281683987 -0.0,-0.8160057252300347 -0.2757475590440447 -1.0,2.3076630082503926 3.2283118851645476 -0.0,1.3000520665928303 0.581203895654615 -1.0,3.8425274250736887 3.6133028383400414 -0.0,0.13694776598217193 -1.1659103408047182 -1.0,2.688548985689179 1.5486856086329917 -0.0,-0.14378057635986438 -1.4649914115754739 -1.0,3.923705106138171 3.8281415874634783 -0.0,1.3334544187579878 -0.048721556115349604 -1.0,3.320777445436592 2.947489296620178 -0.0,-0.36251547004650103 -0.2886015741883188 -1.0,3.2163584307843567 2.9285953038088373 -0.0,0.5437339741631225 -0.23459273264636704 -1.0,2.820666118654177 4.0305429519659395 -0.0,0.04808393980018175 0.42285718084497675 -1.0,1.4686721107589078 2.6605885841423067 -0.0,1.1873828480862414 0.5487600196906772 -1.0,3.425690422789916 4.252827757634791 -0.0,-0.7323210179394448 -0.9818194354330615 -1.0,3.018263609974841 2.914037267945018 -0.0,1.005159548514262 -0.5055899932767433 -1.0,4.566046579419102 5.545663797862058 -0.0,-0.7129346827436536 2.2938920919917742 -1.0,2.869336979055624 2.5688122980246684 -0.0,1.5201806096451054 -0.7414084378784415 -1.0,1.71558426191034 2.4576286538624794 -0.0,0.8090326808020629 0.26208059965589425 -1.0,3.0163716479573077 2.4747608384001056 -0.0,0.47627288733283857 1.3085076289292734 -1.0,3.3891272567835684 3.20832981462489 -0.0,1.0488767400026389 1.2318533170755142 -1.0,3.3428160616141853 2.5497426855885075 -0.0,-0.6411040361810151 -0.4290410178863531 -1.0,2.219119637941564 2.6621113083439254 -0.0,1.5621125506487947 0.7273124535333745 -1.0,3.1459765929197636 1.3663869759433418 -0.0,-0.05263982623034547 0.43675636434345644 -1.0,1.890191705836878 3.435071392429276 -0.0,0.28718983621307775 -2.438042507707637 -1.0,5.717207001359904 2.2303522388797035 -0.0,0.17636841934036573 -0.2202348356695646 -1.0,2.7426941364254294 3.9506423829670734 -0.0,-1.118995077703066 0.6062681312772151 -1.0,4.510963440028501 2.4497214672006575 -0.0,0.07601426739661686 1.4712413920907517 -1.0,2.472822799411239 4.045939967967948 -0.0,-2.2061186560242603 0.32560701091997957 -1.0,3.250675248798315 3.268273446922124 -0.0,-0.024542349115316425 1.5505593308513355 -1.0,2.5654508852779654 2.9476923150082874 -0.0,0.8070230851041806 1.0614288963806608 -1.0,4.0121013342203655 1.7608333223695753 -0.0,-0.6895596222836047 0.035498410809669464 -1.0,1.697905057706837 4.053746875797327 -0.0,-0.3311042917990167 -0.09180266122060314 -1.0,3.720796880080382 4.467214289132983 -0.0,-0.318673057944378 -3.1474317710285202 -1.0,4.809204233917482 4.55250051737848 -0.0,0.596445093094233 0.41780789823963405 -1.0,4.432965399675368 3.4638105151117617 -0.0,-0.10285141484897965 1.747950423830727 -1.0,2.1513849154027014 3.9020766404442933 -0.0,1.5988780419195843 -0.08753929889987294 -1.0,0.9867334105272594 3.017081919852008 -0.0,-1.4952194834476749 1.0187701527429442 -1.0,2.2468599817570376 2.5883807516977395 -0.0,-1.804930212071194 0.3519094744696904 -1.0,4.1524048686549975 2.39387437993355 -0.0,0.7077190974093445 0.5703893640810606 -1.0,3.551726989450847 2.4786821848615985 -0.0,1.866022101379231 0.23733176192158173 -1.0,2.636453843734601 3.2607059005922467 -0.0,1.0052825898444602 0.5988275134415102 -1.0,2.643754787324359 3.72363185525656 -0.0,-0.9925822461102075 0.060644514219670244 -1.0,3.8994350969658136 1.9246001662480055 -0.0,0.6513177047637154 0.04450296971216735 -1.0,2.4564101844841106 3.6785165656991596 -0.0,0.2606556093620563 -0.6172755504020078 -1.0,2.4170362032345674 0.8639272362396189 -0.0,-0.6416537078444019 1.8622433251026849 -1.0,2.0247632881021267 2.538336421666863 -0.0,-1.0177991501405648 -0.8522549981552515 -1.0,3.3426117902650185 3.1635532244875586 -0.0,-0.08963512689480763 1.4555128614393191 -1.0,3.7470117779591092 3.414476280017385 -0.0,0.7721815837750134 -0.17297061945116646 -1.0,3.823597567639877 4.2427688079492665 -0.0,-0.6905817293226868 0.5838402640342898 -1.0,3.005258204213709 2.7252310853631125 -0.0,0.963732273262942 -1.3950688358262504 -1.0,3.2803836447761934 3.448945851174787 -0.0,-0.11576488451784747 1.8796627145034757 -1.0,3.905782244273501 3.3853014175990412 -0.0,0.3786078767939069 0.4054987293824608 -1.0,4.251338642737948 3.2212804055347375 -0.0,1.785664685579919 -0.4528337660796719 -1.0,0.9522164714530392 4.648272724469027 -0.0,2.06805484281029 0.3211833348167774 -1.0,3.2063266406360875 3.20907719820361 -0.0,-0.18542396323311192 -0.4721814985954186 -1.0,1.2468417100913183 2.988063666542869 -0.0,-0.9089767150726245 0.049627884005341995 -1.0,3.570670591235201 1.812766580123238 -0.0,1.9973417232460495 -0.17709723581574177 -1.0,2.810527831677345 2.0292239826226717 -0.0,0.06390562956663569 0.9110683296487658 -1.0,4.449308253046676 2.5895593413305997 -0.0,-0.18596846882351442 1.2495641818989083 -1.0,2.1189215966743986 3.7928094437779283 diff --git a/ml/data/ridge-data/lpsa.data b/ml/data/ridge-data/lpsa.data deleted file mode 100644 index fdd16e36b4..0000000000 --- a/ml/data/ridge-data/lpsa.data +++ /dev/null @@ -1,67 +0,0 @@ --0.4307829,-1.63735562648104 -2.00621178480549 -1.86242597251066 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 --0.1625189,-1.98898046126935 -0.722008756122123 -0.787896192088153 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 --0.1625189,-1.57881887548545 -2.1887840293994 1.36116336875686 -1.02470580167082 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.155348103855541 --0.1625189,-2.16691708463163 -0.807993896938655 -0.787896192088153 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 -0.3715636,-0.507874475300631 -0.458834049396776 -0.250631301876899 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 -0.7654678,-2.03612849966376 -0.933954647105133 -1.86242597251066 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 -0.8544153,-0.557312518810673 -0.208756571683607 -0.787896192088153 0.990146852537193 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 -1.2669476,-0.929360463147704 -0.0578991819441687 0.152317365781542 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 -1.2669476,-2.28833047634983 -0.0706369432557794 -0.116315079324086 0.80409888772376 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 -1.2669476,0.223498042876113 -1.41471935455355 -0.116315079324086 -1.02470580167082 -0.522940888712441 -0.29928234305568 0.342627053981254 0.199211097885341 -1.3480731,0.107785900236813 -1.47221551299731 0.420949810887169 -1.02470580167082 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.687186906466865 -1.446919,0.162180092313795 -1.32557369901905 0.286633588334355 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 -1.4701758,-1.49795329918548 -0.263601072284232 0.823898478545609 0.788388310173035 -0.522940888712441 -0.29928234305568 0.342627053981254 0.199211097885341 -1.4929041,0.796247055396743 0.0476559407005752 0.286633588334355 -1.02470580167082 -0.522940888712441 0.394013435896129 -1.04215728919298 -0.864466507337306 -1.5581446,-1.62233848461465 -0.843294091975396 -3.07127197548598 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 -1.5993876,-0.990720665490831 0.458513517212311 0.823898478545609 1.07379746308195 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 -1.6389967,-0.171901281967138 -0.489197399065355 -0.65357996953534 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 -1.6956156,-1.60758252338831 -0.590700340358265 -0.65357996953534 -0.619561070667254 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 -1.7137979,0.366273918511144 -0.414014962912583 -0.116315079324086 0.232904453212813 -0.522940888712441 0.971228997418125 0.342627053981254 1.26288870310799 -1.8000583,-0.710307384579833 0.211731938156277 0.152317365781542 -1.02470580167082 -0.522940888712441 -0.442797990776478 0.342627053981254 1.61744790484887 -1.8484548,-0.262791728113881 -1.16708345615721 0.420949810887169 0.0846342590816532 -0.522940888712441 0.163172393491611 0.342627053981254 1.97200710658975 -1.8946169,0.899043117369237 -0.590700340358265 0.152317365781542 -1.02470580167082 -0.522940888712441 1.28643254437683 -1.04215728919298 -0.864466507337306 -1.9242487,-0.903451690500615 1.07659722048274 0.152317365781542 1.28380453408541 -0.522940888712441 -0.442797990776478 -1.04215728919298 -0.864466507337306 -2.008214,-0.0633337899773081 -1.38088970920094 0.958214701098423 0.80409888772376 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 -2.0476928,-1.15393789990757 -0.961853075398404 -0.116315079324086 -1.02470580167082 -0.522940888712441 -0.442797990776478 -1.04215728919298 -0.864466507337306 -2.1575593,0.0620203721138446 0.0657973885499142 1.22684714620405 -0.468824786336838 -0.522940888712441 1.31421001659859 1.72741139715549 -0.332627704725983 -2.1916535,-0.75731027755674 -2.92717970468456 0.018001143228728 -1.02470580167082 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.332627704725983 -2.2137539,1.11226993252773 1.06484916245061 0.555266033439982 0.877691038550889 1.89254797819741 1.43890404648442 0.342627053981254 0.376490698755783 -2.2772673,-0.468768642850639 -1.43754788774533 -1.05652863719378 0.576050411655607 -0.522940888712441 0.0120483832567209 0.342627053981254 -0.687186906466865 -2.2975726,-0.618884859896728 -1.1366360750781 -0.519263746982526 -1.02470580167082 -0.522940888712441 -0.863171185425945 3.11219574032972 1.97200710658975 -2.3272777,-0.651431999123483 0.55329161145762 -0.250631301876899 1.11210019001038 -0.522940888712441 -0.179808625688859 -1.04215728919298 -0.864466507337306 -2.5217206,0.115499102435224 -0.512233676577595 0.286633588334355 1.13650173283446 -0.522940888712441 -0.179808625688859 0.342627053981254 -0.155348103855541 -2.5533438,0.266341329949937 -0.551137885443386 -0.384947524429713 0.354857790686005 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.332627704725983 -2.5687881,1.16902610257751 0.855491905752846 2.03274448152093 1.22628985326088 1.89254797819741 2.02833774827712 3.11219574032972 2.68112551007152 -2.6567569,-0.218972367124187 0.851192298581141 0.555266033439982 -1.02470580167082 -0.522940888712441 -0.863171185425945 0.342627053981254 0.908329501367106 -2.677591,0.263121415733908 1.4142681068416 0.018001143228728 1.35980653053822 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 -2.7180005,-0.0704736333296423 1.52000996595417 0.286633588334355 1.39364261119802 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.332627704725983 -2.7942279,-0.751957286017338 0.316843561689933 -1.99674219506348 0.911736065044475 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 -2.8063861,-0.685277652430997 1.28214038482516 0.823898478545609 0.232904453212813 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.155348103855541 -2.8124102,-0.244991501432929 0.51882005949686 -0.384947524429713 0.823246560137838 -0.522940888712441 -0.863171185425945 0.342627053981254 0.553770299626224 -2.8419982,-0.75731027755674 2.09041984898851 1.22684714620405 1.53428167116843 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 -2.8535925,1.20962937075363 -0.242882661178889 1.09253092365124 -1.02470580167082 -0.522940888712441 1.24263233939889 3.11219574032972 2.50384590920108 -2.9204698,0.570886990493502 0.58243883987948 0.555266033439982 1.16006887775962 -0.522940888712441 1.07357183940747 0.342627053981254 1.61744790484887 -2.9626924,0.719758684343624 0.984970304132004 1.09253092365124 1.52137230773457 -0.522940888712441 -0.179808625688859 0.342627053981254 -0.509907305596424 -2.9626924,-1.52406140158064 1.81975700990333 0.689582255992796 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 -2.9729753,-0.132431544081234 2.68769877553723 1.09253092365124 1.53428167116843 -0.522940888712441 -0.442797990776478 0.342627053981254 -0.687186906466865 -3.0130809,0.436161292804989 -0.0834447307428255 -0.519263746982526 -1.02470580167082 1.89254797819741 1.07357183940747 0.342627053981254 1.26288870310799 -3.0373539,-0.161195191984091 -0.671900359186746 1.7641120364153 1.13650173283446 -0.522940888712441 -0.863171185425945 0.342627053981254 0.0219314970149 -3.2752562,1.39927182372944 0.513852869452676 0.689582255992796 -1.02470580167082 1.89254797819741 1.49394503405693 0.342627053981254 -0.155348103855541 -3.3375474,1.51967002306341 -0.852203755696565 0.555266033439982 -0.104527297798983 1.89254797819741 1.85927724828569 0.342627053981254 0.908329501367106 -3.3928291,0.560725834706224 1.87867703391426 1.09253092365124 1.39364261119802 -0.522940888712441 0.486423065822545 0.342627053981254 1.26288870310799 -3.4355988,1.00765532502814 1.69426310090641 1.89842825896812 1.53428167116843 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.509907305596424 -3.4578927,1.10152996153577 -0.10927271844907 0.689582255992796 -1.02470580167082 1.89254797819741 1.97630171771485 0.342627053981254 1.61744790484887 -3.5160131,0.100001934217311 -1.30380956369388 0.286633588334355 0.316555063757567 -0.522940888712441 0.28786643052924 0.342627053981254 0.553770299626224 -3.5307626,0.987291634724086 -0.36279314978779 -0.922212414640967 0.232904453212813 -0.522940888712441 1.79270085261407 0.342627053981254 1.26288870310799 -3.5652984,1.07158528137575 0.606453149641961 1.7641120364153 -0.432854616994416 1.89254797819741 0.528504607720369 0.342627053981254 0.199211097885341 -3.5876769,0.180156323255198 0.188987436375017 -0.519263746982526 1.09956763075594 -0.522940888712441 0.708239632330506 0.342627053981254 0.199211097885341 -3.6309855,1.65687973755377 -0.256675483533719 0.018001143228728 -1.02470580167082 1.89254797819741 1.79270085261407 0.342627053981254 1.26288870310799 -3.6800909,0.5720085322365 0.239854450210939 -0.787896192088153 1.0605418233138 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 -3.7123518,0.323806133438225 -0.606717660886078 -0.250631301876899 -1.02470580167082 1.89254797819741 0.342907418101747 0.342627053981254 0.199211097885341 -3.9843437,1.23668206715898 2.54220539083611 0.152317365781542 -1.02470580167082 1.89254797819741 1.89037692416194 0.342627053981254 1.26288870310799 -3.993603,0.180156323255198 0.154448192444669 1.62979581386249 0.576050411655607 1.89254797819741 0.708239632330506 0.342627053981254 1.79472750571931 -4.029806,1.60906277046565 1.10378605019827 0.555266033439982 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 -4.1295508,1.0036214996026 0.113496885050331 -0.384947524429713 0.860016436332751 1.89254797819741 -0.863171185425945 0.342627053981254 -0.332627704725983 -4.3851468,1.25591974271076 0.577607033774471 0.555266033439982 -1.02470580167082 1.89254797819741 1.07357183940747 0.342627053981254 1.26288870310799 -4.6844434,2.09650591351268 0.625488598331018 -2.66832330782754 -1.02470580167082 1.89254797819741 1.67954222367555 0.342627053981254 0.553770299626224 -5.477509,1.30028987435881 0.338383613253713 0.555266033439982 1.00481276295349 1.89254797819741 1.24263233939889 0.342627053981254 1.97200710658975 diff --git a/ml/src/main/scala/spark/ml/clustering/KMeans.scala b/ml/src/main/scala/spark/ml/clustering/KMeans.scala deleted file mode 100644 index d35f942c01..0000000000 --- a/ml/src/main/scala/spark/ml/clustering/KMeans.scala +++ /dev/null @@ -1,318 +0,0 @@ -package spark.ml.clustering - -import scala.collection.mutable.ArrayBuffer -import scala.util.Random - -import spark.{SparkContext, RDD} -import spark.SparkContext._ -import spark.Logging -import spark.ml.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 train(data: RDD[Array[Double]]): KMeansModel = { - // TODO: check whether data is persistent; this needs RDD.storageLevel to be publicly readable - - val sc = data.sparkContext - - var 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) { - for ((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()) - Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k)) - } - - /** - * 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) - 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) - .train(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[ml] 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[ml] 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 sc = new SparkContext(master, "KMeans") - val data = sc.textFile(inputFile).map(line => line.split(' ').map(_.toDouble)) - val model = KMeans.train(data, k, iters) - val cost = model.computeCost(data) - println("Cluster centers:") - for (c <- model.clusterCenters) { - println(" " + c.mkString(" ")) - } - println("Cost: " + cost) - System.exit(0) - } -} diff --git a/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala b/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala deleted file mode 100644 index 8244ccc55b..0000000000 --- a/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala +++ /dev/null @@ -1,27 +0,0 @@ -package spark.ml.clustering - -import spark.RDD -import spark.SparkContext._ -import spark.ml.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/ml/src/main/scala/spark/ml/clustering/LocalKMeans.scala b/ml/src/main/scala/spark/ml/clustering/LocalKMeans.scala deleted file mode 100644 index 03129ef552..0000000000 --- a/ml/src/main/scala/spark/ml/clustering/LocalKMeans.scala +++ /dev/null @@ -1,88 +0,0 @@ -package spark.ml.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[ml] 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/ml/src/main/scala/spark/ml/optimization/Gradient.scala b/ml/src/main/scala/spark/ml/optimization/Gradient.scala deleted file mode 100644 index 6d062ebddf..0000000000 --- a/ml/src/main/scala/spark/ml/optimization/Gradient.scala +++ /dev/null @@ -1,33 +0,0 @@ -package spark.ml.optimization - -import org.jblas.DoubleMatrix - -abstract class Gradient extends Serializable { - /** - * Compute the gradient for a given row of data. - * - * @param data - One row of data. Row matrix of size 1xn where n is the number of features. - * @param label - Label for this data item. - * @param weights - Column matrix containing weights for every feature. - */ - def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): - (DoubleMatrix, Double) -} - -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) - } -} diff --git a/ml/src/main/scala/spark/ml/optimization/GradientDescent.scala b/ml/src/main/scala/spark/ml/optimization/GradientDescent.scala deleted file mode 100644 index d959ebf71c..0000000000 --- a/ml/src/main/scala/spark/ml/optimization/GradientDescent.scala +++ /dev/null @@ -1,62 +0,0 @@ -package spark.ml.optimization - -import spark.{Logging, RDD, SparkContext} -import spark.SparkContext._ - -import org.jblas.DoubleMatrix - -import scala.collection.mutable.ArrayBuffer - - -object GradientDescent { - - /** - * Run gradient descent in parallel using mini batches. - * Based on Matlab code written by John Duchi. - * - * @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 numIters - number of iterations that SGD should be run. - * @param miniBatchFraction - fraction of the input data set that should be used for - * one iteration of SGD. Default value 1.0. - * - * @return weights - Column matrix containing weights for every feature. - * @return lossHistory - Array containing the loss computed for every iteration. - */ - def runMiniBatchSGD( - data: RDD[(Double, Array[Double])], - gradient: Gradient, - updater: Updater, - stepSize: Double, - numIters: Int, - miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { - - val lossHistory = new ArrayBuffer[Double](numIters) - - val nfeatures: Int = data.take(1)(0)._2.length - val nexamples: Long = data.count() - val miniBatchSize = nexamples * miniBatchFraction - - // Initialize weights as a column matrix - var weights = DoubleMatrix.ones(nfeatures) - var reg_val = 0.0 - - for (i <- 1 to numIters) { - val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42+i).map { - case (y, features) => - val featuresRow = new DoubleMatrix(features.length, 1, features:_*) - val (grad, loss) = gradient.compute(featuresRow, y, weights) - (grad, loss) - }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) - - lossHistory.append(lossSum / miniBatchSize + reg_val) - val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i) - weights = update._1 - reg_val = update._2 - } - - (weights, lossHistory.toArray) - } -} diff --git a/ml/src/main/scala/spark/ml/optimization/Updater.scala b/ml/src/main/scala/spark/ml/optimization/Updater.scala deleted file mode 100644 index dfc7bf2025..0000000000 --- a/ml/src/main/scala/spark/ml/optimization/Updater.scala +++ /dev/null @@ -1,27 +0,0 @@ -package spark.ml.optimization - -import org.jblas.DoubleMatrix - -abstract class Updater extends Serializable { - /** - * Compute an updated value for weights given the gradient, stepSize and iteration number. - * - * @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 - * - * @return weightsNew - Column matrix containing updated weights - * @return reg_val - regularization value - */ - def compute(weightsOlds: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): - (DoubleMatrix, Double) -} - -class SimpleUpdater extends Updater { - override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, - stepSize: Double, iter: Int): (DoubleMatrix, Double) = { - val normGradient = gradient.mul(stepSize / math.sqrt(iter)) - (weightsOld.sub(normGradient), 0) - } -} diff --git a/ml/src/main/scala/spark/ml/recommendation/ALS.scala b/ml/src/main/scala/spark/ml/recommendation/ALS.scala deleted file mode 100644 index 8d5c16847a..0000000000 --- a/ml/src/main/scala/spark/ml/recommendation/ALS.scala +++ /dev/null @@ -1,387 +0,0 @@ -package spark.ml.recommendation - -import scala.collection.mutable.{ArrayBuffer, BitSet} -import scala.util.Random - -import spark.{HashPartitioner, Partitioner, SparkContext, RDD} -import spark.storage.StorageLevel -import spark.SparkContext._ - -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])]]) - - -/** - * 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 parmeters on an input RDD of (user, product, rating) triples. - * Returns a MatrixFactorizationModel with feature vectors for each user and product. - */ - def train(ratings: RDD[(Int, Int, Double)]): MatrixFactorizationModel = { - val numBlocks = if (this.numBlocks == -1) { - math.max(ratings.context.defaultParallelism, ratings.partitions.size) - } else { - this.numBlocks - } - - val partitioner = new HashPartitioner(numBlocks) - - val ratingsByUserBlock = ratings.map{ case (u, p, r) => (u % numBlocks, (u, p, r)) } - val ratingsByProductBlock = ratings.map{ case (u, p, r) => (p % numBlocks, (p, u, r)) } - - val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock) - val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) - - // Initialize user and product factors randomly - val seed = new Random().nextInt() - var users = userOutLinks.mapValues(_.elementIds.map(u => randomFactor(rank, seed ^ u))) - var products = productOutLinks.mapValues(_.elementIds.map(p => randomFactor(rank, seed ^ ~p))) - - 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[(Int, Int, Double)]): OutLinkBlock = { - val userIds = ratings.map(_._1).distinct.sorted - val numUsers = userIds.length - val userIdToPos = userIds.zipWithIndex.toMap - val shouldSend = Array.fill(numUsers)(new BitSet(numBlocks)) - for ((u, p, r) <- ratings) { - shouldSend(userIdToPos(u))(p % 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[(Int, Int, Double)]): InLinkBlock = { - val userIds = ratings.map(_._1).distinct.sorted - val numUsers = userIds.length - val userIdToPos = userIds.zipWithIndex.toMap - val ratingsForBlock = new Array[Array[(Array[Int], Array[Double])]](numBlocks) - for (productBlock <- 0 until numBlocks) { - val ratingsInBlock = ratings.filter(t => t._2 % numBlocks == productBlock) - val ratingsByProduct = ratingsInBlock.groupBy(_._2) // (p, Seq[(u, p, r)]) - .toArray - .sortBy(_._1) - .map{case (p, rs) => (rs.map(t => userIdToPos(t._1)), rs.map(_._3))} - ratingsForBlock(productBlock) = ratingsByProduct - } - 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, (Int, Int, Double))]) - : (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 - Iterator((blockId, (makeInLinkBlock(numBlocks, ratings), makeOutLinkBlock(numBlocks, ratings)))) - }, true) - links.persist(StorageLevel.MEMORY_AND_DISK) - (links.mapValues(_._1), links.mapValues(_._2)) - } - - /** - * Make a random factor vector with the given seed. - * TODO: Initialize things using mapPartitionsWithIndex to make it faster? - */ - private def randomFactor(rank: Int, seed: Int): Array[Double] = { - val rand = new Random(seed) - 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[(Int, Int, Double)], - rank: Int, - iterations: Int, - lambda: Double, - blocks: Int) - : MatrixFactorizationModel = - { - new ALS(blocks, rank, iterations, lambda).train(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[(Int, Int, Double)], 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[(Int, Int, Double)], rank: Int, iterations: Int) - : MatrixFactorizationModel = - { - train(ratings, rank, iterations, 0.01, -1) - } - - def main(args: Array[String]) { - if (args.length != 5) { - 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 sc = new SparkContext(master, "ALS") - val ratings = sc.textFile(ratingsFile).map { line => - val fields = line.split(',') - (fields(0).toInt, fields(1).toInt, fields(2).toDouble) - } - val model = ALS.train(ratings, rank, iters) - 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/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala b/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala deleted file mode 100644 index cfdf2ba523..0000000000 --- a/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala +++ /dev/null @@ -1,23 +0,0 @@ -package spark.ml.recommendation - -import spark.RDD -import spark.SparkContext._ - -import org.jblas._ - -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/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala b/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala deleted file mode 100644 index 3c471ab652..0000000000 --- a/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala +++ /dev/null @@ -1,158 +0,0 @@ -package spark.ml.regression - -import spark.{Logging, RDD, SparkContext} -import spark.ml.optimization._ -import spark.ml.util.MLUtils - -import org.jblas.DoubleMatrix - -/** - * Logistic Regression using Stochastic Gradient Descent. - * Based on Matlab code written by John Duchi. - */ -class LogisticRegressionModel( - val weights: DoubleMatrix, - val intercept: Double, - val losses: Array[Double]) extends RegressionModel { - - override def predict(testData: spark.RDD[Array[Double]]) = { - testData.map { x => - val margin = new DoubleMatrix(1, x.length, x:_*).mmul(this.weights).get(0) + this.intercept - 1.0/ (1.0 + math.exp(margin * -1)) - } - } - - override def predict(testData: Array[Double]): Double = { - val dataMat = new DoubleMatrix(1, testData.length, testData:_*) - val margin = dataMat.mmul(this.weights).get(0) + this.intercept - 1.0/ (1.0 + math.exp(margin * -1)) - } -} - -class LogisticRegression private (var stepSize: Double, var miniBatchFraction: Double, - var numIters: Int) - extends Logging { - - /** - * Construct a LogisticRegression object with default parameters - */ - def this() = this(1.0, 1.0, 100) - - /** - * Set the step size per-iteration of SGD. Default 1.0. - */ - def setStepSize(step: Double) = { - this.stepSize = step - this - } - - /** - * Set fraction of data to be used for each SGD iteration. Default 1.0. - */ - def setMiniBatchFraction(fraction: Double) = { - this.miniBatchFraction = fraction - this - } - - /** - * Set the number of iterations for SGD. Default 100. - */ - def setNumIterations(iters: Int) = { - this.numIters = iters - this - } - - def train(input: RDD[(Double, Array[Double])]): LogisticRegressionModel = { - // Add a extra variable consisting of all 1.0's for the intercept. - val data = input.map { case (y, features) => - (y, Array(1.0, features:_*)) - } - - val (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, miniBatchFraction) - - val weightsScaled = weights.getRange(1, weights.length) - val intercept = weights.get(0) - - val model = new LogisticRegressionModel(weightsScaled, intercept, losses) - - logInfo("Final model weights " + model.weights) - logInfo("Final model intercept " + model.intercept) - logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) - model - } -} - -/** - * Top-level methods for calling Logistic Regression. - */ -object LogisticRegression { - - /** - * 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. - * - * @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[(Double, Array[Double])], - numIterations: Int, - stepSize: Double, - miniBatchFraction: Double) - : LogisticRegressionModel = - { - new LogisticRegression(stepSize, miniBatchFraction, numIterations).train(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. - * - * @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[(Double, Array[Double])], - 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. - * - * @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[(Double, Array[Double])], - 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.loadData(sc, args(1)) - val model = LogisticRegression.train(data, args(3).toInt, args(2).toDouble) - - sc.stop() - } -} diff --git a/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala b/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala deleted file mode 100644 index 6d37aad047..0000000000 --- a/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala +++ /dev/null @@ -1,41 +0,0 @@ -package spark.ml.regression - -import scala.util.Random - -import org.jblas.DoubleMatrix - -import spark.{RDD, SparkContext} -import spark.ml.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.saveData(data, outputPath) - sc.stop() - } -} diff --git a/ml/src/main/scala/spark/ml/regression/Regression.scala b/ml/src/main/scala/spark/ml/regression/Regression.scala deleted file mode 100644 index 4a20f513b7..0000000000 --- a/ml/src/main/scala/spark/ml/regression/Regression.scala +++ /dev/null @@ -1,21 +0,0 @@ -package spark.ml.regression - -import spark.RDD - -trait RegressionModel { - /** - * 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/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala b/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala deleted file mode 100644 index dae224144e..0000000000 --- a/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala +++ /dev/null @@ -1,183 +0,0 @@ -package spark.ml.regression - -import spark.{Logging, RDD, SparkContext} -import spark.SparkContext._ -import spark.ml.util.MLUtils - -import org.jblas.DoubleMatrix -import org.jblas.Solve - -/** - * Ridge Regression from Joseph Gonzalez's implementation in MLBase - */ -class RidgeRegressionModel( - val weights: DoubleMatrix, - val intercept: Double, - val lambdaOpt: Double, - val lambdas: List[(Double, Double, DoubleMatrix)]) - extends RegressionModel { - - override def predict(testData: RDD[Array[Double]]): RDD[Double] = { - testData.map { x => - (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept - } - } - - override def predict(testData: Array[Double]): Double = { - (new DoubleMatrix(1, testData.length, testData:_*).mmul(this.weights)).get(0) + this.intercept - } -} - -class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) - extends Logging { - - def this() = this(0.0, 100.0) - - /** - * Set the lower bound on binary search for lambda's. Default is 0. - */ - def setLowLambda(low: Double) = { - this.lambdaLow = low - this - } - - /** - * Set the upper bound on binary search for lambda's. Default is 100.0. - */ - def setHighLambda(hi: Double) = { - this.lambdaHigh = hi - this - } - - def train(input: RDD[(Double, Array[Double])]): RidgeRegressionModel = { - val nfeatures: Int = input.take(1)(0)._2.length - val nexamples: Long = input.count() - - val (yMean, xColMean, xColSd) = MLUtils.computeStats(input, nfeatures, nexamples) - - val data = input.map { case(y, features) => - val yNormalized = y - yMean - val featuresMat = new DoubleMatrix(nfeatures, 1, features:_*) - val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd) - (yNormalized, featuresNormalized.toArray) - } - - // Compute XtX - Size of XtX is nfeatures by nfeatures - val XtX: DoubleMatrix = data.map { case (y, features) => - val x = new DoubleMatrix(1, features.length, features:_*) - x.transpose().mmul(x) - }.reduce(_.addi(_)) - - // Compute Xt*y - Size of Xty is nfeatures by 1 - val Xty: DoubleMatrix = data.map { case (y, features) => - new DoubleMatrix(features.length, 1, features:_*).mul(y) - }.reduce(_.addi(_)) - - // Define a function to compute the leave one out cross validation error - // for a single example - def crossValidate(lambda: Double): (Double, Double, DoubleMatrix) = { - // Compute the MLE ridge regression parameter value - - // Ridge Regression parameter = inv(XtX + \lambda*I) * Xty - val XtXlambda = DoubleMatrix.eye(nfeatures).muli(lambda).addi(XtX) - val w = Solve.solveSymmetric(XtXlambda, Xty) - - val invXtX = Solve.solveSymmetric(XtXlambda, DoubleMatrix.eye(nfeatures)) - - // compute the generalized cross validation score - val cverror = data.map { - case (y, features) => - val x = new DoubleMatrix(features.length, 1, features:_*) - val yhat = w.transpose().mmul(x).get(0) - val H_ii = x.transpose().mmul(invXtX).mmul(x).get(0) - val residual = (y - yhat) / (1.0 - H_ii) - residual * residual - }.reduce(_ + _) / nexamples - - (lambda, cverror, w) - } - - // Binary search for the best assignment to lambda. - def binSearch(low: Double, high: Double): List[(Double, Double, DoubleMatrix)] = { - val mid = (high - low) / 2 + low - val lowValue = crossValidate((mid - low) / 2 + low) - val highValue = crossValidate((high - mid) / 2 + mid) - val (newLow, newHigh) = if (lowValue._2 < highValue._2) { - (low, mid + (high-low)/4) - } else { - (mid - (high-low)/4, high) - } - if (newHigh - newLow > 1.0E-7) { - // :: is list prepend in Scala. - lowValue :: highValue :: binSearch(newLow, newHigh) - } else { - List(lowValue, highValue) - } - } - - // Actually compute the best lambda - val lambdas = binSearch(lambdaLow, lambdaHigh).sortBy(_._1) - - // Find the best parameter set by taking the lowest cverror. - val (lambdaOpt, cverror, weights) = lambdas.reduce((a, b) => if (a._2 < b._2) a else b) - - // Return the model which contains the solution - val weightsScaled = weights.div(xColSd) - val intercept = yMean - (weights.transpose().mmul(xColMean.div(xColSd)).get(0)) - val model = new RidgeRegressionModel(weightsScaled, intercept, lambdaOpt, lambdas) - - logInfo("RidgeRegression: optimal lambda " + model.lambdaOpt) - logInfo("RidgeRegression: optimal weights " + model.weights) - logInfo("RidgeRegression: optimal intercept " + model.intercept) - logInfo("RidgeRegression: cross-validation error " + cverror) - - model - } -} -/** - * Top-level methods for calling Ridge Regression. - */ -object RidgeRegression { - - /** - * Train a ridge regression model given an RDD of (response, features) pairs. - * We use the closed form solution to compute the cross-validation score for - * a given lambda. The optimal lambda is computed by performing binary search - * between the provided bounds of lambda. - * - * @param input RDD of (response, array of features) pairs. - * @param lambdaLow lower bound used in binary search for lambda - * @param lambdaHigh upper bound used in binary search for lambda - */ - def train( - input: RDD[(Double, Array[Double])], - lambdaLow: Double, - lambdaHigh: Double) - : RidgeRegressionModel = - { - new RidgeRegression(lambdaLow, lambdaHigh).train(input) - } - - /** - * Train a ridge regression model given an RDD of (response, features) pairs. - * We use the closed form solution to compute the cross-validation score for - * a given lambda. The optimal lambda is computed by performing binary search - * between lambda values of 0 and 100. - * - * @param input RDD of (response, array of features) pairs. - */ - def train(input: RDD[(Double, Array[Double])]) : RidgeRegressionModel = { - train(input, 0.0, 100.0) - } - - def main(args: Array[String]) { - if (args.length != 2) { - println("Usage: RidgeRegression ") - System.exit(1) - } - val sc = new SparkContext(args(0), "RidgeRegression") - val data = MLUtils.loadData(sc, args(1)) - val model = RidgeRegression.train(data, 0, 1000) - sc.stop() - } -} diff --git a/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala b/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala deleted file mode 100644 index 75854fe1de..0000000000 --- a/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala +++ /dev/null @@ -1,55 +0,0 @@ -package spark.ml.regression - -import scala.util.Random - -import org.jblas.DoubleMatrix - -import spark.{RDD, SparkContext} -import spark.ml.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.saveData(data, outputPath) - sc.stop() - } -} diff --git a/ml/src/main/scala/spark/ml/util/MLUtils.scala b/ml/src/main/scala/spark/ml/util/MLUtils.scala deleted file mode 100644 index 6efa7c81ad..0000000000 --- a/ml/src/main/scala/spark/ml/util/MLUtils.scala +++ /dev/null @@ -1,95 +0,0 @@ -package spark.ml.util - -import spark.{RDD, SparkContext} -import spark.SparkContext._ - -import org.jblas.DoubleMatrix - -/** - * Helper methods to load and save data - * Data format: - * , ... - * where , are feature values in Double and is the corresponding label as Double. - */ -object MLUtils { - - /** - * @param sc SparkContext - * @param dir Directory to the input data files. - * @return An RDD of tuples. For each tuple, the first element is the label, and the second - * element represents the feature values (an array of Double). - */ - def loadData(sc: SparkContext, dir: String): RDD[(Double, Array[Double])] = { - sc.textFile(dir).map { line => - val parts = line.split(",") - val label = parts(0).toDouble - val features = parts(1).trim().split(" ").map(_.toDouble) - (label, features) - } - } - - def saveData(data: RDD[(Double, Array[Double])], dir: String) { - val dataStr = data.map(x => x._1 + "," + x._2.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[(Double, Array[Double])], nfeatures: Int, nexamples: Long): - (Double, DoubleMatrix, DoubleMatrix) = { - val yMean: Double = data.map { case (y, features) => y }.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 { case(y, features) => - val nCols = features.length - // Traverse over every column and emit (col, value, value^2) - Iterator.tabulate(nCols) { i => - (i, (features(i), features(i)*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/ml/src/test/resources/log4j.properties b/ml/src/test/resources/log4j.properties deleted file mode 100644 index 390c92763c..0000000000 --- a/ml/src/test/resources/log4j.properties +++ /dev/null @@ -1,11 +0,0 @@ -# Set everything to be logged to the file core/target/unit-tests.log -log4j.rootCategory=INFO, file -log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=false -log4j.appender.file.file=ml/target/unit-tests.log -log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n - -# Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN - diff --git a/ml/src/test/scala/spark/ml/clustering/KMeansSuite.scala b/ml/src/test/scala/spark/ml/clustering/KMeansSuite.scala deleted file mode 100644 index f3bd1d599f..0000000000 --- a/ml/src/test/scala/spark/ml/clustering/KMeansSuite.scala +++ /dev/null @@ -1,150 +0,0 @@ -package spark.ml.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 { - val sc = new SparkContext("local", "test") - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - val EPSILON = 1e-4 - - 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||") - 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||") - 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/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala b/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala deleted file mode 100644 index f3f56c4357..0000000000 --- a/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala +++ /dev/null @@ -1,80 +0,0 @@ -package spark.ml.recommendation - -import scala.util.Random - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import spark.SparkContext -import spark.SparkContext._ - -import org.jblas._ - - -class ALSSuite extends FunSuite with BeforeAndAfterAll { - val 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 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 (u, p, trueRatings.get(u, p)) - } - - 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/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala b/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala deleted file mode 100644 index 55f2c5c18e..0000000000 --- a/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala +++ /dev/null @@ -1,57 +0,0 @@ -package spark.ml.regression - -import scala.util.Random - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import spark.SparkContext -import spark.SparkContext._ - - -class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { - val sc = new SparkContext("local", "test") - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - // Test if we can correctly learn A, B where Y = logistic(A + B*X) - test("logistic regression") { - val nPoints = 10000 - val rnd = new Random(42) - - val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - - val A = 2.0 - val B = -1.5 - - // 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(100) - // y <- as.numeric(y > 0) - val y = (0 until nPoints).map { i => - val yVal = A + B * x1(i) + rLogis(i) - if (yVal > 0) 1.0 else 0.0 - } - - val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i)))).toArray - - val testRDD = sc.parallelize(testData, 2) - testRDD.cache() - val lr = new LogisticRegression().setStepSize(10.0) - .setNumIterations(20) - - val model = lr.train(testRDD) - - val weight0 = model.weights.get(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]") - } -} diff --git a/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala b/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala deleted file mode 100644 index aed5cbec24..0000000000 --- a/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala +++ /dev/null @@ -1,47 +0,0 @@ -package spark.ml.regression - -import scala.util.Random - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import spark.SparkContext -import spark.SparkContext._ - - -class RidgeRegressionSuite extends FunSuite with BeforeAndAfterAll { - val sc = new SparkContext("local", "test") - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - // Test if we can correctly learn Y = 3 + X1 + X2 when - // X1 and X2 are collinear. - test("multi-collinear variables") { - val rnd = new Random(43) - val x1 = Array.fill[Double](20)(rnd.nextGaussian()) - - // Pick a mean close to mean of x1 - val rnd1 = new Random(42) //new NormalDistribution(0.1, 0.01) - val x2 = Array.fill[Double](20)(0.1 + rnd1.nextGaussian() * 0.01) - - val xMat = (0 until 20).map(i => Array(x1(i), x2(i))).toArray - - val y = xMat.map(i => 3 + i(0) + i(1)) - val testData = (0 until 20).map(i => (y(i), xMat(i))).toArray - - val testRDD = sc.parallelize(testData, 2) - testRDD.cache() - val ridgeReg = new RidgeRegression().setLowLambda(0) - .setHighLambda(10) - - val model = ridgeReg.train(testRDD) - - assert(model.intercept >= 2.9 && model.intercept <= 3.1) - assert(model.weights.length === 2) - assert(model.weights.get(0) >= 0.9 && model.weights.get(0) <= 1.1) - assert(model.weights.get(1) >= 0.9 && model.weights.get(1) <= 1.1) - } -} diff --git a/mllib/data/als/test.data b/mllib/data/als/test.data new file mode 100644 index 0000000000..e476cc23e0 --- /dev/null +++ b/mllib/data/als/test.data @@ -0,0 +1,16 @@ +1,1,5.0 +1,2,1.0 +1,3,5.0 +1,4,1.0 +2,1,5.0 +2,2,1.0 +2,3,5.0 +2,4,1.0 +3,1,1.0 +3,2,5.0 +3,3,1.0 +3,4,5.0 +4,1,1.0 +4,2,5.0 +4,3,1.0 +4,4,5.0 diff --git a/mllib/data/lr-data/random.data b/mllib/data/lr-data/random.data new file mode 100755 index 0000000000..29bcb8acba --- /dev/null +++ b/mllib/data/lr-data/random.data @@ -0,0 +1,1000 @@ +0.0,-0.19138793197590276 0.7834675900121327 +1.0,3.712420417753061 3.55967640829891 +0.0,-0.3173743619974614 0.9034702789806682 +1.0,4.759494447180777 3.407011867344781 +0.0,-0.7078607074437426 -0.7866705652344417 +1.0,2.6708084832010215 2.5322909406378016 +0.0,-0.07553885038446313 -0.1297104483563081 +1.0,2.759487072285262 2.474689814713741 +0.0,-2.2199161547238107 0.7543109438660762 +1.0,1.922617509832946 1.9412373902594937 +0.0,0.8140942462004225 1.883920822277784 +1.0,1.7649295902120172 3.8195077526061363 +0.0,-1.1173052428096684 -1.468964723960145 +1.0,1.8733449544967458 2.913026590975709 +0.0,-0.11212965215910947 1.068087981775071 +1.0,2.3368459971730227 5.453870208593922 +0.0,-1.2802488543364463 -0.47218504171867676 +1.0,4.1917343620336895 3.5602286778418355 +0.0,0.5995976502137177 -0.797374550890321 +1.0,3.721592294428238 4.824418090974808 +0.0,-0.0721649164244053 -1.3952880192542576 +1.0,3.609764030146346 3.4730043476891277 +0.0,-1.5078269860498976 -2.6460421495665987 +1.0,1.8510254911824193 1.6748364225650059 +0.0,1.021485727769095 -0.14476425336866738 +1.0,4.10105000223134 2.3772502437548493 +0.0,2.6132710211418675 -1.061646527586342 +1.0,2.6444875273854653 4.043302750329545 +0.0,1.115723715938777 0.38401588153403887 +1.0,2.045759949164019 3.156447533448806 +0.0,-1.0543022640565405 -0.6820337845705753 +1.0,3.535337069948117 3.8121122972294965 +0.0,0.9427529503486505 -0.25123516319259886 +1.0,3.9611643301316795 3.3144121016644443 +0.0,-0.15013188927817916 0.8178862482229886 +1.0,3.200504584029051 2.3088398886136057 +0.0,0.819731993393585 -0.47386644109886344 +1.0,3.283317566020217 3.4828146842654513 +0.0,-2.3283941193793303 -0.6148925379529 +1.0,3.901670215294089 3.6356776610143324 +0.0,-0.28635769830042973 0.049586437072917544 +1.0,3.1114746381043927 3.6314805300338775 +0.0,-1.3085536069757229 0.11172767926766304 +1.0,3.3676979357140744 4.689661419564771 +0.0,-1.5820787210442733 1.3226576351191428 +1.0,2.5957586701668207 3.0648240201825923 +0.0,-2.116823743560968 0.272822309954307 +1.0,3.31672509500716 3.870172182480263 +0.0,0.09751166932653511 0.6469052579904877 +1.0,2.0609623373451305 3.9496181906908694 +0.0,0.5238217321419351 -1.2424816480725946 +1.0,3.5731384504449717 5.293293512805712 +0.0,-0.8507917425723299 -1.2243124053200718 +1.0,3.3060954421001867 3.1337045819604565 +0.0,1.5066706426420082 0.04176666807070882 +1.0,4.197316426430547 2.327643377792433 +0.0,-1.8068158696573955 -1.6380836149377855 +1.0,3.568239793850545 3.561688791420822 +0.0,0.4705756905309871 1.1991675114038487 +1.0,4.85003762884306 4.253420553408024 +0.0,0.7595792932847568 0.014062431397674205 +1.0,1.6984862661221896 1.7746925013882613 +0.0,0.1132294255888917 -0.09228036942051128 +1.0,3.766092539171029 2.765647342841482 +0.0,1.053401788561791 -1.0588667339849278 +1.0,2.780021685872393 3.239478188786074 +0.0,0.4042022490052266 1.0982210323828034 +1.0,2.4939569547402063 2.4615506964861273 +0.0,0.4469359967563411 0.3880418183993791 +1.0,2.7943749030887486 3.742182807141721 +0.0,-0.4418685162293727 0.802180923066725 +1.0,3.711213212127241 4.620177703831104 +0.0,0.10737314976605918 -1.5716142960765325 +1.0,4.0522289913808365 3.77562942835957 +0.0,1.4798827061781141 1.1638601205648005 +1.0,3.6758023575825547 3.115500589955362 +0.0,-1.803338141681238 -0.639996207387159 +1.0,2.044667029270621 3.04922768663927 +0.0,-0.06067427095346295 1.394611410740688 +1.0,4.626495834477846 2.995800202291488 +0.0,-0.2770274350630315 0.4521526506693692 +1.0,3.130857841268635 3.76858860814448 +0.0,2.163400739017478 -1.303601716798734 +1.0,2.9131896969824367 3.4288919990054167 +0.0,-0.7145108501670207 1.4189762494365543 +1.0,3.535768896041034 1.4894011726406373 +0.0,1.605614523747256 0.29974289519139824 +1.0,2.413678734728178 2.1826316767457183 +0.0,-0.8821932593373774 0.26432786248412726 +1.0,2.0878695933047116 3.5277388966365177 +0.0,-1.107001191509183 0.38421647065699477 +1.0,2.6462094774496454 2.273786785429519 +0.0,1.0712046043765102 -1.1889735666835115 +1.0,3.7458483094910666 1.3868020542832566 +0.0,-0.8403883736429167 -0.7163969561320671 +1.0,3.3359151000342195 3.2382001552279576 +0.0,0.13309387098922537 0.938761191821517 +1.0,2.083439571838502 3.2204948086228944 +0.0,1.3030219848568272 0.5976630914634896 +1.0,2.7602376200551317 2.200505791897739 +0.0,-0.9458633178207942 0.0490955863627428 +1.0,3.7998466026531883 1.9291683955712686 +0.0,-1.327236501803235 0.06915643957270164 +1.0,3.4740573335685925 2.1080735512507114 +0.0,0.8627688253416859 -1.961802291046532 +1.0,3.5108780392869776 3.9854745964798326 +0.0,-0.69537574439301 0.2436269580373554 +1.0,2.920286302932126 4.704192389485899 +0.0,-2.031190954684878 -0.7843052045579578 +1.0,1.6768848711259499 1.345658047606076 +0.0,0.9234894202027507 -0.38179572928866495 +1.0,3.1710339307651334 4.129874876536583 +0.0,-2.5086697007630376 -0.2638692986795807 +1.0,2.079400422215581 3.124756711992435 +0.0,-0.1388012859869782 0.3698243463601514 +1.0,2.665728164475424 4.574860576068532 +0.0,0.11967116650891912 -0.8792117975750646 +1.0,3.042630437105455 2.7245525508413677 +0.0,0.6078023848042808 -0.7977233104047035 +1.0,3.3340709038589638 4.962729210819017 +0.0,0.6373101353982795 1.1335021278327686 +1.0,3.3821397455119446 4.349379573895378 +0.0,-0.9140176931412027 -0.03428220013900756 +1.0,4.579963977595727 3.8322809335521484 +0.0,-0.43958506434874983 0.21259366700539037 +1.0,2.644701808902675 3.945416465403505 +0.0,-1.119921743746522 -0.2089105317801997 +1.0,2.5480553203091922 3.123344220515146 +0.0,0.8723990414181355 1.11150972420879 +1.0,4.479600967837827 2.8645066949820057 +0.0,-0.003869320481891422 0.24756134775982133 +1.0,3.237294368758498 4.642548547098718 +0.0,0.34643329685515545 0.029869480691029456 +1.0,2.6324740490008893 1.2577448307260846 +0.0,-0.4416403319035849 -1.4597062027342758 +1.0,1.764049052224297 3.649850384544675 +0.0,0.6779287737716254 -1.9489876700506967 +1.0,1.4286669812409405 2.4906452014102416 +0.0,-1.2271599940693638 0.9869686407012563 +1.0,3.6244117441765993 2.36879554315985 +0.0,-0.11422653411940642 0.4741905017884626 +1.0,3.6192153991840694 2.149436181779614 +0.0,0.45425900443207484 -1.357987041493406 +1.0,4.312295702128074 3.7596991900930252 +0.0,-0.35153502234686884 -0.6297451691082592 +1.0,3.4901363450669476 2.0630236379093243 +0.0,-1.5343533005821828 -0.23745688647461852 +1.0,4.775056734905926 5.291243824646301 +0.0,-1.032123659747431 0.8458711875294105 +1.0,2.3091889606097844 3.3688150059111215 +0.0,0.7854236849909306 0.6742463927844289 +1.0,3.284779531346899 2.855746734955609 +0.0,0.380579394855332 -1.2378905330462027 +1.0,2.540193014555953 3.245568950444961 +0.0,-0.5491810448400926 -2.3179482776107894 +1.0,3.481785462949587 1.8870182253717969 +0.0,-0.06833732101790825 2.178923334945784 +1.0,1.1663083809702222 1.8919272314310458 +0.0,-0.7801536433937879 -1.4185984368350903 +1.0,1.457713814592066 3.0323739348144048 +0.0,-0.16377716798970973 0.09678021896691058 +1.0,2.2294515799173094 1.6179126855486068 +0.0,-0.5845552895984718 -0.8095679531228397 +1.0,2.024328902209618 2.4660315284543888 +0.0,0.2037503424802764 1.5767438723426828 +1.0,3.5058983262252643 3.292836693091364 +0.0,-1.4004772080893082 0.6150928060180622 +1.0,4.610936499146778 3.3674445809820313 +0.0,-0.7325641160695897 -3.0469742419403225 +1.0,2.6778956983269926 4.049681967443553 +0.0,-0.3375932473421461 -0.32976087151423067 +1.0,3.975838378562512 1.2032482992228626 +0.0,-1.6622711226380826 -0.6954676646542216 +1.0,3.1601568512397256 2.7472491112914357 +0.0,0.6739969973916968 1.3608866192945286 +1.0,3.097978499063888 3.88429576456391 +0.0,-0.16445244300279913 0.631410854999902 +1.0,4.244875698991619 3.0464568222900477 +0.0,0.1749522197766453 -0.3295077792829936 +1.0,4.158913950688044 1.1836177376726964 +0.0,-1.8286320279969996 -0.6355826362111864 +1.0,2.4795264391445326 0.8073937061906746 +0.0,-0.5095499320702017 -0.8451757050184052 +1.0,3.6489546081475206 2.7405880916534957 +0.0,-0.11733097334574003 0.020300758125140466 +1.0,1.9034123919197892 4.036941742254072 +0.0,-0.4678304671259669 -0.7653895561277071 +1.0,2.555027220737054 4.205906511993216 +0.0,0.1952150967011765 1.2402178923240337 +1.0,3.532371144429582 2.395018092924601 +0.0,1.4682834110821084 2.2292327929025078 +1.0,2.1160331256749663 3.7157102308564824 +0.0,1.3973790173654674 -1.1902799121683607 +1.0,3.4775573554170616 3.0459058509488557 +0.0,-2.215337088722839 0.7693588032777773 +1.0,2.3298220860458976 1.5924630285528396 +0.0,1.260641664088144 1.5474089692944746 +1.0,4.460878990061944 2.595950219349794 +0.0,-1.8214944389802914 -1.9733205363211535 +1.0,4.41874870213851 2.4975116019313264 +0.0,1.2037921250123007 -0.7057578432831773 +1.0,3.042628088030598 3.7366256492570136 +0.0,-0.02609770715133313 -0.01975791007372346 +1.0,1.123824442324706 3.5115607224884466 +0.0,0.3466005704292144 -1.206858960323042 +1.0,3.044152779557358 2.4308738719304266 +0.0,-0.8292396838183249 -0.5768591341562801 +1.0,2.9898679252543325 3.3291086316901484 +0.0,0.6033357093153775 0.18738779274832332 +1.0,3.2777482224094916 2.2676548172839714 +0.0,-0.7104360487845565 -1.0365712508175688 +1.0,2.617802272534323 1.887796671556582 +0.0,-0.21008998836798706 -2.4424443035468957 +1.0,3.9387085143031317 2.368798316318223 +0.0,-0.65027380204969 0.4757828709083824 +1.0,1.6786020855223545 1.62019388696364 +0.0,0.40325101156361803 0.26629562725726075 +1.0,2.4614637796912167 2.778406744842399 +0.0,-0.4327374795655596 0.5643009301153851 +1.0,2.6419358755663103 2.1911675067034206 +0.0,-0.06058610052148417 0.6118154934715632 +1.0,4.134485645832481 4.214482766162727 +0.0,-2.091472947105952 -0.21279450874188077 +1.0,3.7664041746453503 0.5848083052756543 +0.0,0.20187441248519114 0.7310035835212488 +1.0,3.6821251396696817 1.2016937526237272 +0.0,0.16248871053987612 -0.8547163523143474 +1.0,3.1725037691095834 3.051265058839004 +0.0,-1.7466975308858639 -0.048497170816597705 +1.0,4.296665913992498 4.432036327276331 +0.0,-0.49371042139965376 -1.3162216335880739 +1.0,3.0767376272412292 2.4082404056282467 +0.0,0.6517145281009619 -0.15229289422910688 +1.0,3.8556129079007406 4.932746403550176 +0.0,2.467072616559744 -0.6570760874457315 +1.0,3.8722558954619446 2.398547361219584 +0.0,-0.996362973160808 -0.24663573264285635 +1.0,2.058960472055059 0.09020868936476445 +0.0,1.1921444033047794 -1.2205820383864918 +1.0,3.499255855340612 4.26015377680707 +0.0,0.46495431359796363 -0.3535071804767937 +1.0,3.2772715993311534 1.8496849599545144 +0.0,0.9200766227075026 1.0153595739730128 +1.0,3.7395665378166516 4.161859093428991 +0.0,-1.3445731221950805 0.3711182438638966 +1.0,1.974184816991473 2.3758202020218637 +0.0,0.25747673028745044 1.4898729695115611 +1.0,3.643667737073963 2.5171980898063024 +0.0,-0.7491175934837044 1.807998586131331 +1.0,3.024294668483263 2.745713910567566 +0.0,-2.9902104324990075 0.48847563269083094 +1.0,2.693457241550706 4.067192099378729 +0.0,1.0010822910854564 1.065617155304199 +1.0,2.6231328305267576 3.2530925652040796 +0.0,-1.569524799794976 0.10080365850268516 +1.0,5.543177898986999 3.149276748958176 +0.0,-0.2697035609845456 -0.3834981890675749 +1.0,5.5737716796876935 3.134627621089238 +0.0,0.16848836970122472 1.7680681560270155 +1.0,2.984578320659214 3.8081853301923743 +0.0,2.00864307305994 -1.1769936806590435 +1.0,2.4301644281026538 1.5357007015355957 +0.0,-1.251515087462618 -1.0023388301407077 +1.0,2.7783106123714036 3.4753675099443138 +0.0,1.2067779830446301 -1.1138369735803868 +1.0,2.660559526103853 0.9246419639107195 +0.0,-0.2120078291751072 0.553871125085326 +1.0,3.2961674182984613 4.1840551114889655 +0.0,-1.7407002661640898 -0.13494920714243758 +1.0,2.61652747199719 2.606431158365525 +0.0,0.1810536358726569 -0.7041543708042312 +1.0,0.6618977487425206 4.43976232230529 +0.0,-1.1056190552516114 -0.26273698119076755 +1.0,3.245745718364984 0.9585399121419127 +0.0,0.451245033031027 0.3966692171364385 +1.0,0.7000962854359294 2.5787278270774685 +0.0,-0.20657738352563298 -0.3054434424581368 +1.0,2.194893094322135 1.2265276851138993 +0.0,1.6478689673866447 -1.2217538409516264 +1.0,2.6520153534620268 4.253943157694819 +0.0,-1.091459682813003 -1.5933476790183565 +1.0,2.381978388803204 2.5725801073346375 +0.0,-1.7089448316753346 -0.40058783295112843 +1.0,4.692976595302646 2.293610804758882 +0.0,-0.8154594160076379 0.9100123432125261 +1.0,1.8893957859271135 2.365552941116367 +0.0,1.4750445045587657 -0.5730495722105764 +1.0,4.627946484342315 4.01023129091373 +0.0,-0.5740578222548407 -0.9010801407945085 +1.0,1.1844352711236998 1.0077910117111921 +0.0,-1.1904557430938465 -0.972229300373332 +1.0,1.9514043869587852 2.6603232743467817 +0.0,-0.11744191317950421 1.8160954524210857 +1.0,2.796337014232012 3.45131164191957 +0.0,1.1908754571951825 1.37388641966138 +1.0,3.1347230127964805 3.4874636513372774 +0.0,1.4279445191621287 0.4142573535049987 +1.0,3.2845746999649457 2.942571828876143 +0.0,1.0418078095097314 -0.515727237947711 +1.0,3.0672407807876674 3.593602465858237 +0.0,0.1070041194341431 0.013584199138111364 +1.0,2.831124413123504 2.5083468687281196 +0.0,1.9088191143015583 1.1943157723052062 +1.0,2.888463730373365 3.8588231186101716 +0.0,0.3344825700647222 1.4902421889158837 +1.0,5.1805240354926285 2.347000348613805 +0.0,-0.14736761539184529 -1.3764336595247777 +1.0,4.945788020165247 4.520764535128319 +0.0,0.48089579766964224 -1.0406729486881927 +1.0,3.115699146536788 3.0271206455481905 +0.0,0.8816867514268375 -0.7885530518936628 +1.0,3.293642905051253 4.129500570671647 +0.0,0.021019117419869213 -1.0983625263034136 +1.0,3.4712873315273884 2.8896550248710255 +0.0,1.336463967380889 0.1782538924176004 +1.0,2.9674559623039674 2.1702990000666977 +0.0,-0.9137873001694705 -1.6488427315604255 +1.0,2.425720985355789 3.336546225859983 +0.0,-2.3622279944776245 0.33443034793657744 +1.0,3.557057454549674 0.9654984504665607 +0.0,0.4924227412613347 0.8572441753897001 +1.0,2.903599258175698 1.9821387894597133 +0.0,-0.562864152759892 -1.41025535274598 +1.0,2.621542267864135 3.0896861639721602 +0.0,-0.9659016052287058 1.8601390770202668 +1.0,2.73394050343452 1.5908844566159697 +0.0,0.316736908826005 0.2857224419323005 +1.0,2.3312567009140532 5.596694984859762 +0.0,0.3137619371424862 -0.1840942808000176 +1.0,3.857644883242267 1.7425846536145542 +0.0,-0.10204795362718587 3.253153279848385 +1.0,1.991635750012152 3.0091345292604816 +0.0,0.6187841242310289 0.9589700354301842 +1.0,2.9773010080735895 3.723750625441197 +0.0,-0.8890787476930039 0.6057780620635984 +1.0,3.2341068438464773 4.238588226643048 +0.0,-0.6100941277292691 -1.5125630779121992 +1.0,3.378840902739636 2.0705801293719017 +0.0,1.9736225258875286 1.725383750563661 +1.0,1.8874237286900284 3.9061132751393997 +0.0,-0.0823939289302894 1.8958431169469556 +1.0,1.5927855001333566 4.6310125064091965 +0.0,0.3112044157520983 -1.7878471816057036 +1.0,4.34881513764263 3.4693940014863784 +0.0,1.052103622850019 -0.16912252356217902 +1.0,3.167179956507673 2.8792495587252507 +0.0,0.16791453003538387 -0.8546142448164881 +1.0,3.0538805073215953 3.4494667407676842 +0.0,-0.9500475678227512 0.06998146933806365 +1.0,3.8909913837847467 2.6813428719208763 +0.0,-0.09976816220585052 -1.4875944011133129 +1.0,3.1791447205478742 4.424991854067018 +0.0,1.0999643223476656 -1.1200747827607145 +1.0,5.222367041159025 1.2015274537211948 +0.0,-0.2848179798736651 0.401703345435371 +1.0,3.92690552314874 0.5307127426832543 +0.0,-0.6771410319499919 -0.5806616553853885 +1.0,3.611779415106116 3.3322298911093533 +0.0,-1.359189339369671 -0.03773529290863042 +1.0,4.696002594470123 1.4346348756461187 +0.0,-1.0094856636150293 0.19687532044013809 +1.0,3.2169383066148383 3.2307201581236473 +0.0,0.7836015359045666 0.2941037782687062 +1.0,3.7317041306588012 3.7985843457251107 +0.0,-0.3693168101963429 1.4513472421644549 +1.0,4.398703283685875 2.654636797434109 +0.0,0.02043081741683321 0.20805199015337653 +1.0,2.324187503797731 3.8819865944906566 +0.0,1.671377007435211 1.3731572027338659 +1.0,4.534630721644852 1.1543799480085444 +0.0,-0.3253127279932509 -0.8285225286171498 +1.0,3.993821155042294 0.7056403589045206 +0.0,1.194500226045371 0.638917136862092 +1.0,2.72148063695256 3.858678264350294 +0.0,-0.1905653672336637 0.8969404368665279 +1.0,1.9587911397509248 3.937696894952624 +0.0,-1.1358853052995896 1.4443151501322575 +1.0,3.7551091652428026 2.475478572543473 +0.0,-0.9167034706173607 -1.7549316646340103 +1.0,1.4669571532496661 3.2025879996118567 +0.0,-0.9673112226998997 0.13104324478779786 +1.0,5.129589009385082 2.962228456981596 +0.0,-1.038791699676283 0.3394661925580474 +1.0,4.0067362767396055 3.7808733451013863 +0.0,0.4607763000001474 0.3165842402170894 +1.0,3.470781763864157 3.1917117382789906 +0.0,-1.0759836593672722 2.1677955321765423 +1.0,1.8061608083541592 2.1368201192592524 +0.0,0.18913968729195288 -0.6832055159990379 +1.0,2.222086435460701 2.462434683952491 +0.0,1.1697195016246194 -0.6482703204844716 +1.0,0.9469729137532825 2.564223951962673 +0.0,-0.2596612587018774 1.3675954564898984 +1.0,3.3498722540414603 2.8411678301395655 +0.0,0.15549061976540607 -0.8795816620250406 +1.0,3.2166810907529517 3.3909740833940147 +0.0,-0.27777898312342497 1.5708467895548373 +1.0,3.5590852623593734 3.022687446035052 +0.0,0.8854804450462548 -0.1674059547432505 +1.0,5.592380230543062 2.046846128948299 +0.0,-0.38403645419139704 -0.6879614453050698 +1.0,1.2059037878354082 3.1373448113023263 +0.0,-0.9332349591768346 0.3271191223126651 +1.0,2.6941262027196444 2.0016455336591275 +0.0,1.985628476449888 -1.720937514961405 +1.0,1.52678578836386 3.6524268651279113 +0.0,0.14930924959259012 0.3549736192569231 +1.0,2.5081810800507904 4.502494324423253 +0.0,1.3659157029970181 -1.4064298168920828 +1.0,2.8947698041280185 3.871692848909248 +0.0,-0.19002791703482588 0.8099829390725909 +1.0,3.0481549176670555 4.05245395484312 +0.0,-0.014729952199541938 0.43445426055411474 +1.0,3.0874888030440486 3.89317889717026 +0.0,0.9521743475193137 0.16292125350371375 +1.0,3.0564028575123805 3.150394468127784 +0.0,-2.5565867181635724 1.1693524400747453 +1.0,3.963399476624186 2.655863627219969 +0.0,2.0594134768376584 1.4326082874689938 +1.0,3.9415985004601524 4.816989711315565 +0.0,0.4986273362656531 -0.30506819506279537 +1.0,2.7697598834307633 2.0292290332215512 +0.0,-0.4716043983943112 1.4692631198715722 +1.0,3.4127279940145883 3.078218915501194 +0.0,-0.28649487641740207 -0.8009455078808752 +1.0,2.645854233845017 4.028461076417125 +0.0,-1.2333241385253426 -0.2850384355482007 +1.0,2.4938754741404976 1.3466482769013481 +0.0,0.6872021385233428 -0.5159203960430369 +1.0,3.136974388668967 1.69291587793452 +0.0,0.9532239280401443 2.619265789851879 +1.0,2.570576389986536 2.548658346643033 +0.0,-1.030037965987706 0.2814883160676786 +1.0,2.510605023939257 2.3227098241155213 +0.0,2.4171507836629256 1.245606490445435 +1.0,3.5520681299250985 0.7442734445298673 +0.0,1.1940577980770877 1.6319950123919318 +1.0,2.708933998825159 2.118496371335553 +0.0,0.26808250222082186 2.5727974909556437 +1.0,3.221534693193204 3.073316472650363 +0.0,-0.6915734756410544 0.25168141600713434 +1.0,1.839319878312068 1.765565689559382 +0.0,1.708990562782385 1.1196517028520787 +1.0,2.1942131633492643 3.733776318231434 +0.0,1.4884941762679373 -0.5221400677305167 +1.0,2.425026062564176 4.814343944240822 +0.0,-1.3572570451352999 0.04542725800519613 +1.0,3.211869589232063 0.01498355271713292 +0.0,1.6170759581287553 0.7420944718274473 +1.0,1.8096883146020295 1.2063063122336204 +0.0,0.8326608996906895 -0.9760063002065638 +1.0,3.60415819299222 3.905143144181063 +0.0,0.9709971797789466 -1.0644382680658016 +1.0,2.8104103693138778 3.5792951568581017 +0.0,-1.021059644329913 -0.25967578007654707 +1.0,2.4020556940935216 3.8705560506781826 +0.0,-2.704107564850001 -0.14300257306795375 +1.0,3.7681081908063643 2.5433599278958297 +0.0,-0.537043950598385 0.8892208622861 +1.0,3.894301374710518 2.76168141850308 +0.0,-0.8416385593366815 1.3377079857054535 +1.0,1.4560861866861152 1.9464951398785584 +0.0,0.8974462212548237 -0.9027814165394935 +1.0,2.848274393366227 4.089266410865265 +0.0,-1.9874388443190703 -2.0515326123686 +1.0,1.7443330286532606 5.182730816947559 +0.0,1.9345124573698136 0.15482916596109797 +1.0,3.730890742221753 3.4571088485293173 +0.0,-0.7591467032951466 0.7817400181511722 +1.0,1.9612060838774241 1.7874104906670758 +0.0,0.04241602781710118 1.7624663777014242 +1.0,2.983106574446788 2.057794179835603 +0.0,-2.2675373876565272 0.1810247094230928 +1.0,1.8242036739605434 3.2897838599534053 +0.0,0.42135250345103276 0.9201551657148959 +1.0,2.3324158301116547 3.2735600739611406 +0.0,-2.503382611181759 -0.604428052499623 +1.0,2.1068571110070753 1.3987709205712464 +0.0,-0.25006447102137164 1.1597904649452788 +1.0,3.6610503210650105 2.389802330720335 +0.0,0.6655774387829471 -0.7657689612002381 +1.0,3.85820287126228 5.653287382126853 +0.0,0.08244241317513575 0.4755361735454262 +1.0,3.6029514045048234 3.0483730792265247 +0.0,1.0276000901424318 -0.569237094330588 +1.0,2.484863163042475 3.4464671311141046 +0.0,0.24588867824456415 -0.7355421671684942 +1.0,2.8757627634577396 1.3730139621444188 +0.0,0.911649033206053 -1.0562220913143838 +1.0,0.6701966948829261 3.8815519088585195 +0.0,1.0649444423673609 0.5738944212075908 +1.0,3.1272553354329955 5.18450239514651 +0.0,-1.8305691156390467 -1.2811179644895232 +1.0,4.326027257587544 1.9589219729995737 +0.0,-0.2278417247639679 -0.6436775444106994 +1.0,3.9854139754166136 2.8662622299102947 +0.0,-0.33177487577648573 0.7122237484053809 +1.0,2.7631237758865255 2.490470927953921 +0.0,-0.2989203275224733 -0.9063254275476191 +1.0,2.7739570950234254 3.333596743208583 +0.0,-0.12025132003053318 -1.2251715775331837 +1.0,3.9028268386113307 2.580334438085556 +0.0,0.3114518803226873 0.35489645702286177 +1.0,2.8765994073916112 4.251640702192294 +0.0,-3.0895947568085367 -1.0526550179589378 +1.0,3.5182345295490216 2.764855512391279 +0.0,0.5749621254042305 0.7148834016467635 +1.0,4.039448299164001 2.377396087740471 +0.0,1.7077800661629936 -0.23711282974122355 +1.0,2.883211311171089 3.5259606315833287 +0.0,-1.0304518163976537 -0.16271910447066004 +1.0,3.8284470175501504 1.0841759781704199 +0.0,-1.3620621426919217 0.8678141368192274 +1.0,3.831976508070298 2.3592788803510505 +0.0,0.8398199934902235 0.8458121179021545 +1.0,2.166979759191688 4.408250411844058 +0.0,-1.2009412161006234 -0.04486968047943732 +1.0,3.0041897020427517 1.67577082931885 +0.0,-1.0550850035108499 2.6114061208535673 +1.0,1.46399823823424 3.6863318429400627 +0.0,-0.439942118867861 0.8107733517611471 +1.0,2.799907981207793 3.1021389011201244 +0.0,0.40512996190803663 -0.2720769110918539 +1.0,2.936414720731187 2.6121553148876706 +0.0,0.7864503163458285 0.879685137879171 +1.0,3.497848931993103 3.93953696354328 +0.0,1.0898800025299487 -0.3780987477521812 +1.0,3.0737866861658834 3.8281246288654067 +0.0,1.0100369320198321 -0.36412797089680377 +1.0,4.977156552398557 1.9361263628969327 +0.0,1.1948682006514484 -1.0421380659408503 +1.0,2.3707352395183743 3.319087891488442 +0.0,0.14662871945444525 -1.125277513770441 +1.0,4.18636170602371 5.079790109963499 +0.0,0.5213830491310841 2.5489667538554355 +1.0,3.456121838657517 2.9777488007628823 +0.0,1.3942157902546204 -0.7392170745991694 +1.0,4.027857416272539 2.5520251242493615 +0.0,0.6677437543225546 -0.7054702957392922 +1.0,2.419993627501343 3.147115729790262 +0.0,-1.1891285195785104 0.7121837556662985 +1.0,2.6768950566988114 2.746092902448666 +0.0,-0.5581632736462642 -0.8475377022167101 +1.0,2.2877649074222144 3.360822129377224 +0.0,0.12427410923130733 -0.029877611579596446 +1.0,2.1363649823278976 2.040672619624904 +0.0,0.164296403698455 -0.7853340225962958 +1.0,2.2867454265483063 2.920796736914219 +0.0,0.030938689766481568 0.02840531713718885 +1.0,4.935402862397514 4.984097800264938 +0.0,-0.49323021214001667 -0.009344009957387383 +1.0,2.2590589178865788 2.784700488476081 +0.0,-1.7996451721642797 -0.08927843209025701 +1.0,2.7189425454136047 3.366984002518318 +0.0,-0.4732503966611213 2.41667617281343 +1.0,1.914172722581019 2.723688261246487 +0.0,0.6854209215843875 -0.6321377274037409 +1.0,4.7025333481932705 2.6561807763401646 +0.0,0.016511529980536163 -0.4064291762993186 +1.0,1.3841179371371182 3.367159685928979 +0.0,-0.525665902025766 0.3189849885462113 +1.0,2.1237941386456276 3.4141040859263914 +0.0,-1.3977733609952327 1.6180332199555512 +1.0,3.3282228318571496 2.9879449742002184 +0.0,-1.3911999737510374 -0.47876736354905697 +1.0,3.071461319022103 3.902142645231827 +0.0,-1.4616870328596612 0.4234223737141411 +1.0,3.3069543201402576 1.3522887907099401 +0.0,0.1771175002160632 0.7092577154896049 +1.0,2.561517669553921 3.2663130772229185 +0.0,0.8635080818806004 1.7578935533355913 +1.0,3.3054989034355793 3.4205399612822633 +0.0,-0.5525474134214131 -0.008874526853035592 +1.0,5.024607965706471 3.377256085775693 +0.0,0.6499316691799448 0.7636813929956143 +1.0,1.7211648540475015 3.7290596058136307 +0.0,-0.4312096678787339 0.4723353140241522 +1.0,1.6269397815780402 1.9613109767814954 +0.0,0.06589250830042476 0.5659627954925366 +1.0,1.4141705667382305 2.9411215895612255 +0.0,-0.30655047441372724 1.134312621267185 +1.0,4.079371134159225 3.7127217011979767 +0.0,-0.11148410319718746 1.504423362990177 +1.0,3.21908765035085 1.5284527951297098 +0.0,0.38879874604519066 -0.7718569898512835 +1.0,3.0387686435299197 1.9571679686339727 +0.0,0.0432538958325193 -0.609046739618082 +1.0,3.858513576900389 2.3343789318227595 +0.0,-1.594606569379673 2.0291869081775498 +1.0,4.418575803606943 3.634284954659144 +0.0,-1.5657043498774568 0.48528442006547645 +1.0,3.7474369990653518 2.417108621170513 +0.0,-0.4087178618516316 -0.5585629524971241 +1.0,2.8830052178069345 2.714807180476644 +0.0,1.0200529614238536 1.633454495011907 +1.0,2.161101444560085 2.722233198993495 +0.0,0.8905571055499505 0.3531260808046299 +1.0,1.5770402091220281 2.5197577954902615 +0.0,0.19603489193696402 0.4391781215510938 +1.0,3.285302297900197 2.5981032583297274 +0.0,-1.7728311957227578 2.226646036588897 +1.0,2.212402423781055 2.994783519362575 +0.0,-0.26351331835428804 0.6197161896115081 +1.0,2.5101464936050144 2.747453537535198 +0.0,1.083443472210967 -0.7471502465676395 +1.0,2.618022142084275 3.201094589808021 +0.0,-0.10243507468644107 -1.5307780048431203 +1.0,2.0479014235932986 2.7174445598757764 +0.0,-0.2530316183327909 1.5105959457792464 +1.0,2.616239369128394 3.1011058356715644 +0.0,2.0703487677159997 -1.23039689097027 +1.0,2.00559575849234 3.088170264353322 +0.0,0.751453701775929 -0.34079600956200146 +1.0,2.6436129383324625 0.6934715851263205 +0.0,0.4735774669250165 0.24981500600111478 +1.0,3.614102521076285 3.297655445774221 +0.0,-0.8397190394129946 2.0791729859494583 +1.0,2.5800847823336372 2.312770726398467 +0.0,0.9528690775719402 -4.054641847252764 +1.0,1.6631425491523402 4.465488566725185 +0.0,-0.40442215938144854 2.1662912065078923 +1.0,3.2025444402071472 0.954639816329502 +0.0,0.8484611241529962 -0.6531501762867838 +1.0,2.907155165379039 4.494838051538261 +0.0,1.1473298350419248 -0.7604213061923158 +1.0,4.406872541176625 2.616395889868952 +0.0,-1.0643453307576694 0.32269083514118757 +1.0,3.4229771635424653 5.404174358063928 +0.0,0.8223012341648268 -2.0705983787489455 +1.0,0.6519219290294926 3.317297519573949 +0.0,0.6661739745821234 0.21368601256080724 +1.0,2.8092516816651187 2.9407143882873363 +0.0,-2.0396349059310626 0.6660958962860263 +1.0,1.621401319049101 2.120514741629026 +0.0,-0.6673242389540511 -1.033336539766657 +1.0,2.4729967381312257 2.0622671692969314 +0.0,0.318696287733599 0.7696143248064906 +1.0,-0.3310542190127661 2.503572170101248 +0.0,-0.024545405442632163 1.2826535279165514 +1.0,2.08361065329982 1.7709137020843035 +0.0,-0.03325908838419148 2.127731976717063 +1.0,0.8920712229737089 2.267227052639782 +0.0,2.4226620796703706 -1.5422597801969735 +1.0,2.6125707261695665 4.136941962252239 +0.0,0.710000430684373 -0.2365544035810329 +1.0,3.587983407259662 2.371118916918134 +0.0,1.548716105657387 2.6039797648647527 +1.0,2.288647833469394 2.8514285941696564 +0.0,0.5407956769257948 -1.4250712589214616 +1.0,3.9999271279969157 4.647262641336589 +0.0,0.46916438504363506 -0.16114805677977867 +1.0,3.9351714928555133 3.017851089635014 +0.0,-0.24683125971847 0.8686956304798523 +1.0,2.445900548419883 2.601998949302925 +0.0,0.9708272515136681 0.9540365110832763 +1.0,2.0889493306284472 1.670700190658552 +0.0,0.7573519355244429 -0.6731075400854291 +1.0,2.9938559890272676 0.5796453404844417 +0.0,-0.42350233780111274 0.1072223004754211 +1.0,3.22502989165533 3.2744724666391045 +0.0,-0.051171179793716125 0.035749085667007977 +1.0,4.256076524642883 3.956646576238979 +0.0,0.44715068158575316 -0.10904823199444005 +1.0,3.754239074295241 2.4862504435534283 +0.0,-0.12025734941101636 0.6682754649328633 +1.0,2.9673795614648815 3.6207880514009263 +0.0,-2.250093626462795 -0.49148713538228506 +1.0,1.7335315087131171 4.234455598757855 +0.0,-0.5145677322324603 -1.8872464244504652 +1.0,3.1524408905920547 2.534903833671654 +0.0,1.4188237424906527 -1.987300018397619 +1.0,3.025903676999244 2.1652631630581847 +0.0,0.5008343534015861 0.28011601768758965 +1.0,2.0039218613662197 2.3639397631018015 +0.0,1.342528231824729 1.0036076495884643 +1.0,3.3281244751369985 2.4251038991267277 +0.0,-0.38845861664115766 -1.5147629282596704 +1.0,2.613448357242925 4.463712912575443 +0.0,-0.19439583983218703 0.676381234314577 +1.0,1.0400516553104269 2.3981508685333424 +0.0,0.9469554018478826 -0.08144910777086176 +1.0,3.179705969662961 3.768848690124549 +0.0,0.39855441813668835 -1.6301847736954416 +1.0,2.1915941615815226 2.7947789889097763 +0.0,1.6023287643577222 0.05432794979410767 +1.0,1.5758610206949497 3.8709473262823777 +0.0,-1.3109119301269387 -0.8645189055395048 +1.0,3.715865055565244 1.9360512196442488 +0.0,-0.2073998491467907 -1.178882579876182 +1.0,2.565062666629786 2.3121370465462494 +0.0,-0.41397768670851737 -0.6674761320605563 +1.0,2.941938460212705 3.537877403937825 +0.0,0.5954231185191001 1.6839554319972647 +1.0,4.591360208911688 1.4381368838271187 +0.0,-1.3221878199013057 0.786799353955043 +1.0,0.6498018470693379 2.2143413646510095 +0.0,0.5346452265922554 0.45599002729248733 +1.0,2.668100742914233 2.679883986650412 +0.0,-0.22428284967184606 -1.0003823373608314 +1.0,4.233871998643562 3.3423521548333897 +0.0,0.7800144346305873 1.6512542456242612 +1.0,3.3192955924982677 4.664828345688715 +0.0,-0.9059493298933676 -0.42207747354389447 +1.0,3.1776956110847916 1.1393123509452483 +0.0,-0.5246202787832872 1.0246845701853746 +1.0,4.732113325540828 1.29018271893586 +0.0,0.9863596225434407 0.7506968948666005 +1.0,2.911409852038849 2.626474556246977 +0.0,0.8545346747310709 -2.1711133879380955 +1.0,2.476689592134109 4.03136160709651 +0.0,0.43108249592457043 0.4589971218864913 +1.0,3.2333287857145825 2.188137362144206 +0.0,1.4405649581445525 0.4131214094941824 +1.0,2.0631468420251093 3.807898318807702 +0.0,0.43964401099781425 0.6669437158150616 +1.0,2.165843657939062 4.109647016182597 +0.0,-0.9735452695016392 -0.6172105570335473 +1.0,3.169794653766589 3.2721053734106 +0.0,1.3129166037688875 -1.2040138532590103 +1.0,2.211361701514339 1.025981622029549 +0.0,0.3653350359702278 0.5229315457444437 +1.0,3.372206428302252 4.163685355869495 +0.0,-0.8690030167652726 0.3226849491596335 +1.0,4.188509026227427 2.1137749377457076 +0.0,2.2174789916979933 0.8249932442083762 +1.0,3.9224824525785706 2.9436443006575925 +0.0,0.1370905200148926 -0.043320354739616776 +1.0,3.1118662077850807 1.4983207834379917 +0.0,-0.5304073850344787 -0.4219778391981189 +1.0,1.2153552376808336 3.4749521622043438 +0.0,-2.545970043914331 -0.5480647959096547 +1.0,1.8097968872175412 4.733523163055134 +0.0,-0.5599306916727819 0.4648015112295201 +1.0,3.0242901796172204 4.354893518146392 +0.0,-0.49175893973189483 1.8635231981223406 +1.0,3.923889822736733 4.199324033436554 +0.0,0.32931083529824645 -1.2038529291812745 +1.0,2.8430570026355904 3.2581768028655214 +0.0,0.08015643729775149 -0.5281238499521005 +1.0,1.0251176552841985 2.452443183841665 +0.0,-1.4000614002792062 -0.4723026702712555 +1.0,4.642753244692533 3.5777684251625153 +0.0,-0.9732069449126244 -0.7507666182081589 +1.0,2.284811103731081 2.6226837934175817 +0.0,1.4938320459354653 1.2271703303402608 +1.0,2.5217907633717935 1.9804499278889345 +0.0,0.9177851256816916 -1.196945923903535 +1.0,2.650515007788954 0.9818159554114416 +0.0,-0.4172435945582116 0.11930551874205601 +1.0,1.8203127944592765 3.3069324017397594 +0.0,0.08195935202288789 -0.2585763476071969 +1.0,2.14910426585678 4.146147361847687 +0.0,1.578290774885182 0.16149960053586573 +1.0,1.2607405323635168 2.940350340912184 +0.0,1.6722138822230346 -0.5454073192477626 +1.0,0.3769561517619793 4.029314828130509 +0.0,-0.012008811772440746 0.2577932550827986 +1.0,2.330909580388283 3.1650439747088024 +0.0,-1.4224384024201595 -0.6369918128076046 +1.0,3.451178380794735 2.7553545272536746 +0.0,-0.7913135079702314 -0.012217405089490006 +1.0,3.7918310740082424 3.3927876820084033 +0.0,0.41016650792928255 0.3521369094279198 +1.0,2.380867149491576 3.7533007228820754 +0.0,-0.2787273586680994 1.3553543015884186 +1.0,2.8933236071325226 1.7975563396445144 +0.0,-0.4868680345968448 0.058461169788172784 +1.0,3.484434144626577 3.5622013162506683 +0.0,1.171904838026115 0.1162839888503951 +1.0,1.8132727587691455 2.238018140780368 +0.0,0.8114997821213137 -1.712768034302675 +1.0,2.977061410695451 2.802894970831404 +0.0,1.7141760742336318 0.5672102391229309 +1.0,3.2929421353515185 3.3754831695793945 +0.0,-2.280170614413754 -0.4912881923146271 +1.0,4.182771547422101 3.5331418354105812 +0.0,-0.2544453921577854 0.4682744998445509 +1.0,1.9236524545763007 2.628837510538455 +0.0,0.6645491524745186 -2.398604366119661 +1.0,3.50840713613987 3.7182332137428955 +0.0,-1.4532823239751684 -0.9916580822162051 +1.0,2.769613688635247 4.72661442603805 +0.0,-1.090104082054257 0.486265921887567 +1.0,3.4900626627065003 3.03025323652533 +0.0,1.4518716691137106 -0.10218738652959546 +1.0,2.745034544461333 4.366809709694589 +0.0,-0.17197050309086373 0.13673125942508174 +1.0,2.4934379443680985 2.954734256628178 +0.0,0.14078971520128297 -0.5401300324197861 +1.0,3.640563349517043 5.163454382169049 +0.0,1.0264020194022627 -0.8738489740165843 +1.0,3.791458514669831 2.2038333093620834 +0.0,-3.075231830613813 2.04054404065675 +1.0,4.647422323558612 3.5220753128741427 +0.0,-0.6423734479152313 0.5403500050100541 +1.0,1.5985339514690007 2.73447434771563 +0.0,-0.04474684215568748 -0.21477212224970194 +1.0,2.6701891009654792 3.9776885659794505 +0.0,-0.4714276238216119 1.4235807729101415 +1.0,3.5551789183755806 2.7057825768035104 +0.0,1.108254774651522 0.8596053056731966 +1.0,3.0623366138774983 2.718494058918926 +0.0,-1.375827910513567 0.011994162356159788 +1.0,3.841407434840553 2.8434319292302304 +0.0,-0.7149712282755271 0.1811986378283469 +1.0,5.155524316715826 2.1468464150279747 +0.0,-0.06822014690491127 -0.15801546435311806 +1.0,3.4838423066641173 4.211572262022802 +0.0,1.455177312877137 -0.9388697017811595 +1.0,3.917344840727481 3.569507254920478 +0.0,-2.080636526173827 -1.2489913979804321 +1.0,4.904327940183608 3.4289745068714295 +0.0,-1.4744723958060084 0.2930577753686633 +1.0,2.810346752831796 2.4062885063635333 +0.0,-0.17365054648101302 -2.26263747840141 +1.0,4.077713960215311 3.841309768575811 +0.0,1.581178479362914 -0.9672846912018417 +1.0,4.516244757634386 2.9078781629204054 +0.0,-1.5890391289381882 -0.4092245513024253 +1.0,3.359480708344044 3.7375262649030123 +0.0,1.5675385032786122 0.9010632060589036 +1.0,3.8564874267647644 3.060660915266198 +0.0,-0.2482500870678099 0.29655946916337894 +1.0,3.1672692968701397 1.1973226392521306 +0.0,-1.4471523637168304 0.5370395414503478 +1.0,4.814859889188941 2.229750617440331 +0.0,0.2812295731325761 0.6044036116090106 +1.0,2.4884527354338903 1.4171627784171204 +0.0,1.173099753717184 0.7948729712563257 +1.0,1.5092479631180256 4.1412277875509105 +0.0,-1.1453508695714685 -0.15567849492271865 +1.0,1.9397046305500465 3.430755367623314 +0.0,-1.6689604208958047 -1.161942047896626 +1.0,4.287905082572467 2.643797664646416 +0.0,0.5691715436318573 -0.6013793142266736 +1.0,2.622904412483301 1.769830678112635 +0.0,-1.0627706066421603 -1.2962746926911266 +1.0,2.5818494635089886 2.9547836545958663 +0.0,-1.555832778500785 0.6050365213516793 +1.0,0.6877755924513469 3.0627330470806617 +0.0,-0.6945984937358738 -0.5355659085722678 +1.0,3.631758943383 2.6990914911890194 +0.0,-0.10204034384758799 1.2650405538373874 +1.0,2.8618200471403488 2.7676923144816237 +0.0,-1.2337428464512885 -0.7151041760567872 +1.0,3.5209869997316807 3.280763138579491 +0.0,0.3700095159793621 -0.8614396246939711 +1.0,2.698616090611572 3.2205340189872795 +0.0,-0.8069663812258417 -0.07956402748767083 +1.0,2.929873320056276 4.030067053746698 +0.0,-1.2316919288622938 1.245687935224532 +1.0,2.9285679560367055 2.9682906465530783 +0.0,-0.3965578686363537 1.1748126835359254 +1.0,4.002714110052464 4.370338584188975 +0.0,-0.6084107635744659 -0.6092872315132073 +1.0,3.293912876563504 3.5843332356258464 +0.0,-0.8145032742370918 1.4050967895930515 +1.0,1.991600071099763 2.343264260750465 +0.0,-0.9433799779882722 1.5943129187456013 +1.0,2.369037146473894 1.9827898318071764 +0.0,-0.26885731570182714 0.47421918725401946 +1.0,3.263006333756187 3.0441051541001443 +0.0,0.21785408377528742 0.5754303556190559 +1.0,2.941128899266118 1.240818619804987 +0.0,0.736142634408259 -1.3173589352849961 +1.0,3.2027184783050644 2.9218716893221766 +0.0,1.9216539101612737 -2.2400666381338694 +1.0,2.4823406743823426 3.429705681271458 +0.0,0.0666674809216063 -0.976496437708073 +1.0,3.206108328915537 2.0828009180110976 +0.0,-0.11582094814525531 2.5093876016868366 +1.0,2.5373176496966328 2.32926952602907 +0.0,-0.9237765727032562 0.9342845305943139 +1.0,2.5300867778672123 3.2754703213122753 +0.0,0.13837351460348038 0.2533025702882705 +1.0,4.556185356940701 0.7629684714626066 +0.0,-1.8251759895063635 0.6966019254550819 +1.0,4.905392053322123 4.111245902434462 +0.0,0.09886105139472441 1.4093224263552915 +1.0,2.0484713074013223 4.874632770975326 +0.0,-0.040609033066195156 -1.3446008307073973 +1.0,3.678642687565624 4.156505531118834 +0.0,0.052003196801406706 1.2239229001362555 +1.0,3.4376496474012876 2.417529764306501 +0.0,-0.09054032070414311 -1.7571173217955876 +1.0,3.230032966809188 3.5965216835420546 +0.0,0.9100014718072797 0.5615698517199065 +1.0,3.938728443662248 3.2945250621813273 +0.0,-0.9205165004286314 -0.01425448590777016 +1.0,1.907285344344031 3.8629943281683987 +0.0,-0.8160057252300347 -0.2757475590440447 +1.0,2.3076630082503926 3.2283118851645476 +0.0,1.3000520665928303 0.581203895654615 +1.0,3.8425274250736887 3.6133028383400414 +0.0,0.13694776598217193 -1.1659103408047182 +1.0,2.688548985689179 1.5486856086329917 +0.0,-0.14378057635986438 -1.4649914115754739 +1.0,3.923705106138171 3.8281415874634783 +0.0,1.3334544187579878 -0.048721556115349604 +1.0,3.320777445436592 2.947489296620178 +0.0,-0.36251547004650103 -0.2886015741883188 +1.0,3.2163584307843567 2.9285953038088373 +0.0,0.5437339741631225 -0.23459273264636704 +1.0,2.820666118654177 4.0305429519659395 +0.0,0.04808393980018175 0.42285718084497675 +1.0,1.4686721107589078 2.6605885841423067 +0.0,1.1873828480862414 0.5487600196906772 +1.0,3.425690422789916 4.252827757634791 +0.0,-0.7323210179394448 -0.9818194354330615 +1.0,3.018263609974841 2.914037267945018 +0.0,1.005159548514262 -0.5055899932767433 +1.0,4.566046579419102 5.545663797862058 +0.0,-0.7129346827436536 2.2938920919917742 +1.0,2.869336979055624 2.5688122980246684 +0.0,1.5201806096451054 -0.7414084378784415 +1.0,1.71558426191034 2.4576286538624794 +0.0,0.8090326808020629 0.26208059965589425 +1.0,3.0163716479573077 2.4747608384001056 +0.0,0.47627288733283857 1.3085076289292734 +1.0,3.3891272567835684 3.20832981462489 +0.0,1.0488767400026389 1.2318533170755142 +1.0,3.3428160616141853 2.5497426855885075 +0.0,-0.6411040361810151 -0.4290410178863531 +1.0,2.219119637941564 2.6621113083439254 +0.0,1.5621125506487947 0.7273124535333745 +1.0,3.1459765929197636 1.3663869759433418 +0.0,-0.05263982623034547 0.43675636434345644 +1.0,1.890191705836878 3.435071392429276 +0.0,0.28718983621307775 -2.438042507707637 +1.0,5.717207001359904 2.2303522388797035 +0.0,0.17636841934036573 -0.2202348356695646 +1.0,2.7426941364254294 3.9506423829670734 +0.0,-1.118995077703066 0.6062681312772151 +1.0,4.510963440028501 2.4497214672006575 +0.0,0.07601426739661686 1.4712413920907517 +1.0,2.472822799411239 4.045939967967948 +0.0,-2.2061186560242603 0.32560701091997957 +1.0,3.250675248798315 3.268273446922124 +0.0,-0.024542349115316425 1.5505593308513355 +1.0,2.5654508852779654 2.9476923150082874 +0.0,0.8070230851041806 1.0614288963806608 +1.0,4.0121013342203655 1.7608333223695753 +0.0,-0.6895596222836047 0.035498410809669464 +1.0,1.697905057706837 4.053746875797327 +0.0,-0.3311042917990167 -0.09180266122060314 +1.0,3.720796880080382 4.467214289132983 +0.0,-0.318673057944378 -3.1474317710285202 +1.0,4.809204233917482 4.55250051737848 +0.0,0.596445093094233 0.41780789823963405 +1.0,4.432965399675368 3.4638105151117617 +0.0,-0.10285141484897965 1.747950423830727 +1.0,2.1513849154027014 3.9020766404442933 +0.0,1.5988780419195843 -0.08753929889987294 +1.0,0.9867334105272594 3.017081919852008 +0.0,-1.4952194834476749 1.0187701527429442 +1.0,2.2468599817570376 2.5883807516977395 +0.0,-1.804930212071194 0.3519094744696904 +1.0,4.1524048686549975 2.39387437993355 +0.0,0.7077190974093445 0.5703893640810606 +1.0,3.551726989450847 2.4786821848615985 +0.0,1.866022101379231 0.23733176192158173 +1.0,2.636453843734601 3.2607059005922467 +0.0,1.0052825898444602 0.5988275134415102 +1.0,2.643754787324359 3.72363185525656 +0.0,-0.9925822461102075 0.060644514219670244 +1.0,3.8994350969658136 1.9246001662480055 +0.0,0.6513177047637154 0.04450296971216735 +1.0,2.4564101844841106 3.6785165656991596 +0.0,0.2606556093620563 -0.6172755504020078 +1.0,2.4170362032345674 0.8639272362396189 +0.0,-0.6416537078444019 1.8622433251026849 +1.0,2.0247632881021267 2.538336421666863 +0.0,-1.0177991501405648 -0.8522549981552515 +1.0,3.3426117902650185 3.1635532244875586 +0.0,-0.08963512689480763 1.4555128614393191 +1.0,3.7470117779591092 3.414476280017385 +0.0,0.7721815837750134 -0.17297061945116646 +1.0,3.823597567639877 4.2427688079492665 +0.0,-0.6905817293226868 0.5838402640342898 +1.0,3.005258204213709 2.7252310853631125 +0.0,0.963732273262942 -1.3950688358262504 +1.0,3.2803836447761934 3.448945851174787 +0.0,-0.11576488451784747 1.8796627145034757 +1.0,3.905782244273501 3.3853014175990412 +0.0,0.3786078767939069 0.4054987293824608 +1.0,4.251338642737948 3.2212804055347375 +0.0,1.785664685579919 -0.4528337660796719 +1.0,0.9522164714530392 4.648272724469027 +0.0,2.06805484281029 0.3211833348167774 +1.0,3.2063266406360875 3.20907719820361 +0.0,-0.18542396323311192 -0.4721814985954186 +1.0,1.2468417100913183 2.988063666542869 +0.0,-0.9089767150726245 0.049627884005341995 +1.0,3.570670591235201 1.812766580123238 +0.0,1.9973417232460495 -0.17709723581574177 +1.0,2.810527831677345 2.0292239826226717 +0.0,0.06390562956663569 0.9110683296487658 +1.0,4.449308253046676 2.5895593413305997 +0.0,-0.18596846882351442 1.2495641818989083 +1.0,2.1189215966743986 3.7928094437779283 diff --git a/mllib/data/ridge-data/lpsa.data b/mllib/data/ridge-data/lpsa.data new file mode 100644 index 0000000000..fdd16e36b4 --- /dev/null +++ b/mllib/data/ridge-data/lpsa.data @@ -0,0 +1,67 @@ +-0.4307829,-1.63735562648104 -2.00621178480549 -1.86242597251066 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +-0.1625189,-1.98898046126935 -0.722008756122123 -0.787896192088153 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +-0.1625189,-1.57881887548545 -2.1887840293994 1.36116336875686 -1.02470580167082 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.155348103855541 +-0.1625189,-2.16691708463163 -0.807993896938655 -0.787896192088153 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +0.3715636,-0.507874475300631 -0.458834049396776 -0.250631301876899 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +0.7654678,-2.03612849966376 -0.933954647105133 -1.86242597251066 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +0.8544153,-0.557312518810673 -0.208756571683607 -0.787896192088153 0.990146852537193 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.2669476,-0.929360463147704 -0.0578991819441687 0.152317365781542 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.2669476,-2.28833047634983 -0.0706369432557794 -0.116315079324086 0.80409888772376 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.2669476,0.223498042876113 -1.41471935455355 -0.116315079324086 -1.02470580167082 -0.522940888712441 -0.29928234305568 0.342627053981254 0.199211097885341 +1.3480731,0.107785900236813 -1.47221551299731 0.420949810887169 -1.02470580167082 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.687186906466865 +1.446919,0.162180092313795 -1.32557369901905 0.286633588334355 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.4701758,-1.49795329918548 -0.263601072284232 0.823898478545609 0.788388310173035 -0.522940888712441 -0.29928234305568 0.342627053981254 0.199211097885341 +1.4929041,0.796247055396743 0.0476559407005752 0.286633588334355 -1.02470580167082 -0.522940888712441 0.394013435896129 -1.04215728919298 -0.864466507337306 +1.5581446,-1.62233848461465 -0.843294091975396 -3.07127197548598 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.5993876,-0.990720665490831 0.458513517212311 0.823898478545609 1.07379746308195 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.6389967,-0.171901281967138 -0.489197399065355 -0.65357996953534 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.6956156,-1.60758252338831 -0.590700340358265 -0.65357996953534 -0.619561070667254 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +1.7137979,0.366273918511144 -0.414014962912583 -0.116315079324086 0.232904453212813 -0.522940888712441 0.971228997418125 0.342627053981254 1.26288870310799 +1.8000583,-0.710307384579833 0.211731938156277 0.152317365781542 -1.02470580167082 -0.522940888712441 -0.442797990776478 0.342627053981254 1.61744790484887 +1.8484548,-0.262791728113881 -1.16708345615721 0.420949810887169 0.0846342590816532 -0.522940888712441 0.163172393491611 0.342627053981254 1.97200710658975 +1.8946169,0.899043117369237 -0.590700340358265 0.152317365781542 -1.02470580167082 -0.522940888712441 1.28643254437683 -1.04215728919298 -0.864466507337306 +1.9242487,-0.903451690500615 1.07659722048274 0.152317365781542 1.28380453408541 -0.522940888712441 -0.442797990776478 -1.04215728919298 -0.864466507337306 +2.008214,-0.0633337899773081 -1.38088970920094 0.958214701098423 0.80409888772376 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +2.0476928,-1.15393789990757 -0.961853075398404 -0.116315079324086 -1.02470580167082 -0.522940888712441 -0.442797990776478 -1.04215728919298 -0.864466507337306 +2.1575593,0.0620203721138446 0.0657973885499142 1.22684714620405 -0.468824786336838 -0.522940888712441 1.31421001659859 1.72741139715549 -0.332627704725983 +2.1916535,-0.75731027755674 -2.92717970468456 0.018001143228728 -1.02470580167082 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.332627704725983 +2.2137539,1.11226993252773 1.06484916245061 0.555266033439982 0.877691038550889 1.89254797819741 1.43890404648442 0.342627053981254 0.376490698755783 +2.2772673,-0.468768642850639 -1.43754788774533 -1.05652863719378 0.576050411655607 -0.522940888712441 0.0120483832567209 0.342627053981254 -0.687186906466865 +2.2975726,-0.618884859896728 -1.1366360750781 -0.519263746982526 -1.02470580167082 -0.522940888712441 -0.863171185425945 3.11219574032972 1.97200710658975 +2.3272777,-0.651431999123483 0.55329161145762 -0.250631301876899 1.11210019001038 -0.522940888712441 -0.179808625688859 -1.04215728919298 -0.864466507337306 +2.5217206,0.115499102435224 -0.512233676577595 0.286633588334355 1.13650173283446 -0.522940888712441 -0.179808625688859 0.342627053981254 -0.155348103855541 +2.5533438,0.266341329949937 -0.551137885443386 -0.384947524429713 0.354857790686005 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.332627704725983 +2.5687881,1.16902610257751 0.855491905752846 2.03274448152093 1.22628985326088 1.89254797819741 2.02833774827712 3.11219574032972 2.68112551007152 +2.6567569,-0.218972367124187 0.851192298581141 0.555266033439982 -1.02470580167082 -0.522940888712441 -0.863171185425945 0.342627053981254 0.908329501367106 +2.677591,0.263121415733908 1.4142681068416 0.018001143228728 1.35980653053822 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +2.7180005,-0.0704736333296423 1.52000996595417 0.286633588334355 1.39364261119802 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.332627704725983 +2.7942279,-0.751957286017338 0.316843561689933 -1.99674219506348 0.911736065044475 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +2.8063861,-0.685277652430997 1.28214038482516 0.823898478545609 0.232904453212813 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.155348103855541 +2.8124102,-0.244991501432929 0.51882005949686 -0.384947524429713 0.823246560137838 -0.522940888712441 -0.863171185425945 0.342627053981254 0.553770299626224 +2.8419982,-0.75731027755674 2.09041984898851 1.22684714620405 1.53428167116843 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +2.8535925,1.20962937075363 -0.242882661178889 1.09253092365124 -1.02470580167082 -0.522940888712441 1.24263233939889 3.11219574032972 2.50384590920108 +2.9204698,0.570886990493502 0.58243883987948 0.555266033439982 1.16006887775962 -0.522940888712441 1.07357183940747 0.342627053981254 1.61744790484887 +2.9626924,0.719758684343624 0.984970304132004 1.09253092365124 1.52137230773457 -0.522940888712441 -0.179808625688859 0.342627053981254 -0.509907305596424 +2.9626924,-1.52406140158064 1.81975700990333 0.689582255992796 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +2.9729753,-0.132431544081234 2.68769877553723 1.09253092365124 1.53428167116843 -0.522940888712441 -0.442797990776478 0.342627053981254 -0.687186906466865 +3.0130809,0.436161292804989 -0.0834447307428255 -0.519263746982526 -1.02470580167082 1.89254797819741 1.07357183940747 0.342627053981254 1.26288870310799 +3.0373539,-0.161195191984091 -0.671900359186746 1.7641120364153 1.13650173283446 -0.522940888712441 -0.863171185425945 0.342627053981254 0.0219314970149 +3.2752562,1.39927182372944 0.513852869452676 0.689582255992796 -1.02470580167082 1.89254797819741 1.49394503405693 0.342627053981254 -0.155348103855541 +3.3375474,1.51967002306341 -0.852203755696565 0.555266033439982 -0.104527297798983 1.89254797819741 1.85927724828569 0.342627053981254 0.908329501367106 +3.3928291,0.560725834706224 1.87867703391426 1.09253092365124 1.39364261119802 -0.522940888712441 0.486423065822545 0.342627053981254 1.26288870310799 +3.4355988,1.00765532502814 1.69426310090641 1.89842825896812 1.53428167116843 -0.522940888712441 -0.863171185425945 0.342627053981254 -0.509907305596424 +3.4578927,1.10152996153577 -0.10927271844907 0.689582255992796 -1.02470580167082 1.89254797819741 1.97630171771485 0.342627053981254 1.61744790484887 +3.5160131,0.100001934217311 -1.30380956369388 0.286633588334355 0.316555063757567 -0.522940888712441 0.28786643052924 0.342627053981254 0.553770299626224 +3.5307626,0.987291634724086 -0.36279314978779 -0.922212414640967 0.232904453212813 -0.522940888712441 1.79270085261407 0.342627053981254 1.26288870310799 +3.5652984,1.07158528137575 0.606453149641961 1.7641120364153 -0.432854616994416 1.89254797819741 0.528504607720369 0.342627053981254 0.199211097885341 +3.5876769,0.180156323255198 0.188987436375017 -0.519263746982526 1.09956763075594 -0.522940888712441 0.708239632330506 0.342627053981254 0.199211097885341 +3.6309855,1.65687973755377 -0.256675483533719 0.018001143228728 -1.02470580167082 1.89254797819741 1.79270085261407 0.342627053981254 1.26288870310799 +3.6800909,0.5720085322365 0.239854450210939 -0.787896192088153 1.0605418233138 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +3.7123518,0.323806133438225 -0.606717660886078 -0.250631301876899 -1.02470580167082 1.89254797819741 0.342907418101747 0.342627053981254 0.199211097885341 +3.9843437,1.23668206715898 2.54220539083611 0.152317365781542 -1.02470580167082 1.89254797819741 1.89037692416194 0.342627053981254 1.26288870310799 +3.993603,0.180156323255198 0.154448192444669 1.62979581386249 0.576050411655607 1.89254797819741 0.708239632330506 0.342627053981254 1.79472750571931 +4.029806,1.60906277046565 1.10378605019827 0.555266033439982 -1.02470580167082 -0.522940888712441 -0.863171185425945 -1.04215728919298 -0.864466507337306 +4.1295508,1.0036214996026 0.113496885050331 -0.384947524429713 0.860016436332751 1.89254797819741 -0.863171185425945 0.342627053981254 -0.332627704725983 +4.3851468,1.25591974271076 0.577607033774471 0.555266033439982 -1.02470580167082 1.89254797819741 1.07357183940747 0.342627053981254 1.26288870310799 +4.6844434,2.09650591351268 0.625488598331018 -2.66832330782754 -1.02470580167082 1.89254797819741 1.67954222367555 0.342627053981254 0.553770299626224 +5.477509,1.30028987435881 0.338383613253713 0.555266033439982 1.00481276295349 1.89254797819741 1.24263233939889 0.342627053981254 1.97200710658975 diff --git a/mllib/src/main/scala/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/spark/ml/clustering/KMeans.scala new file mode 100644 index 0000000000..6d78f926c2 --- /dev/null +++ b/mllib/src/main/scala/spark/ml/clustering/KMeans.scala @@ -0,0 +1,319 @@ +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 train(data: RDD[Array[Double]]): KMeansModel = { + // TODO: check whether data is persistent; this needs RDD.storageLevel to be publicly readable + + val sc = data.sparkContext + + var 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) { + for ((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()) + Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k)) + } + + /** + * 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) + 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) + .train(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 sc = new SparkContext(master, "KMeans") + val data = sc.textFile(inputFile).map(line => line.split(' ').map(_.toDouble)) + val model = KMeans.train(data, k, iters) + 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/ml/clustering/KMeansModel.scala b/mllib/src/main/scala/spark/ml/clustering/KMeansModel.scala new file mode 100644 index 0000000000..4fd0646160 --- /dev/null +++ b/mllib/src/main/scala/spark/ml/clustering/KMeansModel.scala @@ -0,0 +1,27 @@ +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/ml/clustering/LocalKMeans.scala b/mllib/src/main/scala/spark/ml/clustering/LocalKMeans.scala new file mode 100644 index 0000000000..e12b3be251 --- /dev/null +++ b/mllib/src/main/scala/spark/ml/clustering/LocalKMeans.scala @@ -0,0 +1,88 @@ +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/ml/optimization/Gradient.scala b/mllib/src/main/scala/spark/ml/optimization/Gradient.scala new file mode 100644 index 0000000000..90b0999a5e --- /dev/null +++ b/mllib/src/main/scala/spark/ml/optimization/Gradient.scala @@ -0,0 +1,33 @@ +package spark.mllib.optimization + +import org.jblas.DoubleMatrix + +abstract class Gradient extends Serializable { + /** + * Compute the gradient for a given row of data. + * + * @param data - One row of data. Row matrix of size 1xn where n is the number of features. + * @param label - Label for this data item. + * @param weights - Column matrix containing weights for every feature. + */ + def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): + (DoubleMatrix, Double) +} + +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) + } +} diff --git a/mllib/src/main/scala/spark/ml/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/ml/optimization/GradientDescent.scala new file mode 100644 index 0000000000..eff853f379 --- /dev/null +++ b/mllib/src/main/scala/spark/ml/optimization/GradientDescent.scala @@ -0,0 +1,62 @@ +package spark.mllib.optimization + +import spark.{Logging, RDD, SparkContext} +import spark.SparkContext._ + +import org.jblas.DoubleMatrix + +import scala.collection.mutable.ArrayBuffer + + +object GradientDescent { + + /** + * Run gradient descent in parallel using mini batches. + * Based on Matlab code written by John Duchi. + * + * @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 numIters - number of iterations that SGD should be run. + * @param miniBatchFraction - fraction of the input data set that should be used for + * one iteration of SGD. Default value 1.0. + * + * @return weights - Column matrix containing weights for every feature. + * @return lossHistory - Array containing the loss computed for every iteration. + */ + def runMiniBatchSGD( + data: RDD[(Double, Array[Double])], + gradient: Gradient, + updater: Updater, + stepSize: Double, + numIters: Int, + miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { + + val lossHistory = new ArrayBuffer[Double](numIters) + + val nfeatures: Int = data.take(1)(0)._2.length + val nexamples: Long = data.count() + val miniBatchSize = nexamples * miniBatchFraction + + // Initialize weights as a column matrix + var weights = DoubleMatrix.ones(nfeatures) + var reg_val = 0.0 + + for (i <- 1 to numIters) { + val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42+i).map { + case (y, features) => + val featuresRow = new DoubleMatrix(features.length, 1, features:_*) + val (grad, loss) = gradient.compute(featuresRow, y, weights) + (grad, loss) + }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) + + lossHistory.append(lossSum / miniBatchSize + reg_val) + val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i) + weights = update._1 + reg_val = update._2 + } + + (weights, lossHistory.toArray) + } +} diff --git a/mllib/src/main/scala/spark/ml/optimization/Updater.scala b/mllib/src/main/scala/spark/ml/optimization/Updater.scala new file mode 100644 index 0000000000..ea80bfcbfd --- /dev/null +++ b/mllib/src/main/scala/spark/ml/optimization/Updater.scala @@ -0,0 +1,27 @@ +package spark.mllib.optimization + +import org.jblas.DoubleMatrix + +abstract class Updater extends Serializable { + /** + * Compute an updated value for weights given the gradient, stepSize and iteration number. + * + * @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 + * + * @return weightsNew - Column matrix containing updated weights + * @return reg_val - regularization value + */ + def compute(weightsOlds: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): + (DoubleMatrix, Double) +} + +class SimpleUpdater extends Updater { + override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, + stepSize: Double, iter: Int): (DoubleMatrix, Double) = { + val normGradient = gradient.mul(stepSize / math.sqrt(iter)) + (weightsOld.sub(normGradient), 0) + } +} diff --git a/mllib/src/main/scala/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/spark/ml/recommendation/ALS.scala new file mode 100644 index 0000000000..0c6fa6f741 --- /dev/null +++ b/mllib/src/main/scala/spark/ml/recommendation/ALS.scala @@ -0,0 +1,387 @@ +package spark.mllib.recommendation + +import scala.collection.mutable.{ArrayBuffer, BitSet} +import scala.util.Random + +import spark.{HashPartitioner, Partitioner, SparkContext, RDD} +import spark.storage.StorageLevel +import spark.SparkContext._ + +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])]]) + + +/** + * 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 parmeters on an input RDD of (user, product, rating) triples. + * Returns a MatrixFactorizationModel with feature vectors for each user and product. + */ + def train(ratings: RDD[(Int, Int, Double)]): MatrixFactorizationModel = { + val numBlocks = if (this.numBlocks == -1) { + math.max(ratings.context.defaultParallelism, ratings.partitions.size) + } else { + this.numBlocks + } + + val partitioner = new HashPartitioner(numBlocks) + + val ratingsByUserBlock = ratings.map{ case (u, p, r) => (u % numBlocks, (u, p, r)) } + val ratingsByProductBlock = ratings.map{ case (u, p, r) => (p % numBlocks, (p, u, r)) } + + val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock) + val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) + + // Initialize user and product factors randomly + val seed = new Random().nextInt() + var users = userOutLinks.mapValues(_.elementIds.map(u => randomFactor(rank, seed ^ u))) + var products = productOutLinks.mapValues(_.elementIds.map(p => randomFactor(rank, seed ^ ~p))) + + 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[(Int, Int, Double)]): OutLinkBlock = { + val userIds = ratings.map(_._1).distinct.sorted + val numUsers = userIds.length + val userIdToPos = userIds.zipWithIndex.toMap + val shouldSend = Array.fill(numUsers)(new BitSet(numBlocks)) + for ((u, p, r) <- ratings) { + shouldSend(userIdToPos(u))(p % 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[(Int, Int, Double)]): InLinkBlock = { + val userIds = ratings.map(_._1).distinct.sorted + val numUsers = userIds.length + val userIdToPos = userIds.zipWithIndex.toMap + val ratingsForBlock = new Array[Array[(Array[Int], Array[Double])]](numBlocks) + for (productBlock <- 0 until numBlocks) { + val ratingsInBlock = ratings.filter(t => t._2 % numBlocks == productBlock) + val ratingsByProduct = ratingsInBlock.groupBy(_._2) // (p, Seq[(u, p, r)]) + .toArray + .sortBy(_._1) + .map{case (p, rs) => (rs.map(t => userIdToPos(t._1)), rs.map(_._3))} + ratingsForBlock(productBlock) = ratingsByProduct + } + 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, (Int, Int, Double))]) + : (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 + Iterator((blockId, (makeInLinkBlock(numBlocks, ratings), makeOutLinkBlock(numBlocks, ratings)))) + }, true) + links.persist(StorageLevel.MEMORY_AND_DISK) + (links.mapValues(_._1), links.mapValues(_._2)) + } + + /** + * Make a random factor vector with the given seed. + * TODO: Initialize things using mapPartitionsWithIndex to make it faster? + */ + private def randomFactor(rank: Int, seed: Int): Array[Double] = { + val rand = new Random(seed) + 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[(Int, Int, Double)], + rank: Int, + iterations: Int, + lambda: Double, + blocks: Int) + : MatrixFactorizationModel = + { + new ALS(blocks, rank, iterations, lambda).train(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[(Int, Int, Double)], 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[(Int, Int, Double)], rank: Int, iterations: Int) + : MatrixFactorizationModel = + { + train(ratings, rank, iterations, 0.01, -1) + } + + def main(args: Array[String]) { + if (args.length != 5) { + 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 sc = new SparkContext(master, "ALS") + val ratings = sc.textFile(ratingsFile).map { line => + val fields = line.split(',') + (fields(0).toInt, fields(1).toInt, fields(2).toDouble) + } + val model = ALS.train(ratings, rank, iters) + 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/ml/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala new file mode 100644 index 0000000000..fb812a6dbe --- /dev/null +++ b/mllib/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala @@ -0,0 +1,23 @@ +package spark.mllib.recommendation + +import spark.RDD +import spark.SparkContext._ + +import org.jblas._ + +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/ml/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/ml/regression/LogisticRegression.scala new file mode 100644 index 0000000000..448ab9dce9 --- /dev/null +++ b/mllib/src/main/scala/spark/ml/regression/LogisticRegression.scala @@ -0,0 +1,158 @@ +package spark.mllib.regression + +import spark.{Logging, RDD, SparkContext} +import spark.mllib.optimization._ +import spark.mllib.util.MLUtils + +import org.jblas.DoubleMatrix + +/** + * Logistic Regression using Stochastic Gradient Descent. + * Based on Matlab code written by John Duchi. + */ +class LogisticRegressionModel( + val weights: DoubleMatrix, + val intercept: Double, + val losses: Array[Double]) extends RegressionModel { + + override def predict(testData: spark.RDD[Array[Double]]) = { + testData.map { x => + val margin = new DoubleMatrix(1, x.length, x:_*).mmul(this.weights).get(0) + this.intercept + 1.0/ (1.0 + math.exp(margin * -1)) + } + } + + override def predict(testData: Array[Double]): Double = { + val dataMat = new DoubleMatrix(1, testData.length, testData:_*) + val margin = dataMat.mmul(this.weights).get(0) + this.intercept + 1.0/ (1.0 + math.exp(margin * -1)) + } +} + +class LogisticRegression private (var stepSize: Double, var miniBatchFraction: Double, + var numIters: Int) + extends Logging { + + /** + * Construct a LogisticRegression object with default parameters + */ + def this() = this(1.0, 1.0, 100) + + /** + * Set the step size per-iteration of SGD. Default 1.0. + */ + def setStepSize(step: Double) = { + this.stepSize = step + this + } + + /** + * Set fraction of data to be used for each SGD iteration. Default 1.0. + */ + def setMiniBatchFraction(fraction: Double) = { + this.miniBatchFraction = fraction + this + } + + /** + * Set the number of iterations for SGD. Default 100. + */ + def setNumIterations(iters: Int) = { + this.numIters = iters + this + } + + def train(input: RDD[(Double, Array[Double])]): LogisticRegressionModel = { + // Add a extra variable consisting of all 1.0's for the intercept. + val data = input.map { case (y, features) => + (y, Array(1.0, features:_*)) + } + + val (weights, losses) = GradientDescent.runMiniBatchSGD( + data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, miniBatchFraction) + + val weightsScaled = weights.getRange(1, weights.length) + val intercept = weights.get(0) + + val model = new LogisticRegressionModel(weightsScaled, intercept, losses) + + logInfo("Final model weights " + model.weights) + logInfo("Final model intercept " + model.intercept) + logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) + model + } +} + +/** + * Top-level methods for calling Logistic Regression. + */ +object LogisticRegression { + + /** + * 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. + * + * @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[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double) + : LogisticRegressionModel = + { + new LogisticRegression(stepSize, miniBatchFraction, numIterations).train(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. + * + * @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[(Double, Array[Double])], + 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. + * + * @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[(Double, Array[Double])], + 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.loadData(sc, args(1)) + val model = LogisticRegression.train(data, args(3).toInt, args(2).toDouble) + + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala new file mode 100644 index 0000000000..9f6abab70b --- /dev/null +++ b/mllib/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala @@ -0,0 +1,41 @@ +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.saveData(data, outputPath) + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/ml/regression/Regression.scala b/mllib/src/main/scala/spark/ml/regression/Regression.scala new file mode 100644 index 0000000000..f79974c191 --- /dev/null +++ b/mllib/src/main/scala/spark/ml/regression/Regression.scala @@ -0,0 +1,21 @@ +package spark.mllib.regression + +import spark.RDD + +trait RegressionModel { + /** + * 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/ml/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala new file mode 100644 index 0000000000..2d07c77141 --- /dev/null +++ b/mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala @@ -0,0 +1,183 @@ +package spark.mllib.regression + +import spark.{Logging, RDD, SparkContext} +import spark.SparkContext._ +import spark.mllib.util.MLUtils + +import org.jblas.DoubleMatrix +import org.jblas.Solve + +/** + * Ridge Regression from Joseph Gonzalez's implementation in MLBase + */ +class RidgeRegressionModel( + val weights: DoubleMatrix, + val intercept: Double, + val lambdaOpt: Double, + val lambdas: List[(Double, Double, DoubleMatrix)]) + extends RegressionModel { + + override def predict(testData: RDD[Array[Double]]): RDD[Double] = { + testData.map { x => + (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept + } + } + + override def predict(testData: Array[Double]): Double = { + (new DoubleMatrix(1, testData.length, testData:_*).mmul(this.weights)).get(0) + this.intercept + } +} + +class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) + extends Logging { + + def this() = this(0.0, 100.0) + + /** + * Set the lower bound on binary search for lambda's. Default is 0. + */ + def setLowLambda(low: Double) = { + this.lambdaLow = low + this + } + + /** + * Set the upper bound on binary search for lambda's. Default is 100.0. + */ + def setHighLambda(hi: Double) = { + this.lambdaHigh = hi + this + } + + def train(input: RDD[(Double, Array[Double])]): RidgeRegressionModel = { + val nfeatures: Int = input.take(1)(0)._2.length + val nexamples: Long = input.count() + + val (yMean, xColMean, xColSd) = MLUtils.computeStats(input, nfeatures, nexamples) + + val data = input.map { case(y, features) => + val yNormalized = y - yMean + val featuresMat = new DoubleMatrix(nfeatures, 1, features:_*) + val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd) + (yNormalized, featuresNormalized.toArray) + } + + // Compute XtX - Size of XtX is nfeatures by nfeatures + val XtX: DoubleMatrix = data.map { case (y, features) => + val x = new DoubleMatrix(1, features.length, features:_*) + x.transpose().mmul(x) + }.reduce(_.addi(_)) + + // Compute Xt*y - Size of Xty is nfeatures by 1 + val Xty: DoubleMatrix = data.map { case (y, features) => + new DoubleMatrix(features.length, 1, features:_*).mul(y) + }.reduce(_.addi(_)) + + // Define a function to compute the leave one out cross validation error + // for a single example + def crossValidate(lambda: Double): (Double, Double, DoubleMatrix) = { + // Compute the MLE ridge regression parameter value + + // Ridge Regression parameter = inv(XtX + \lambda*I) * Xty + val XtXlambda = DoubleMatrix.eye(nfeatures).muli(lambda).addi(XtX) + val w = Solve.solveSymmetric(XtXlambda, Xty) + + val invXtX = Solve.solveSymmetric(XtXlambda, DoubleMatrix.eye(nfeatures)) + + // compute the generalized cross validation score + val cverror = data.map { + case (y, features) => + val x = new DoubleMatrix(features.length, 1, features:_*) + val yhat = w.transpose().mmul(x).get(0) + val H_ii = x.transpose().mmul(invXtX).mmul(x).get(0) + val residual = (y - yhat) / (1.0 - H_ii) + residual * residual + }.reduce(_ + _) / nexamples + + (lambda, cverror, w) + } + + // Binary search for the best assignment to lambda. + def binSearch(low: Double, high: Double): List[(Double, Double, DoubleMatrix)] = { + val mid = (high - low) / 2 + low + val lowValue = crossValidate((mid - low) / 2 + low) + val highValue = crossValidate((high - mid) / 2 + mid) + val (newLow, newHigh) = if (lowValue._2 < highValue._2) { + (low, mid + (high-low)/4) + } else { + (mid - (high-low)/4, high) + } + if (newHigh - newLow > 1.0E-7) { + // :: is list prepend in Scala. + lowValue :: highValue :: binSearch(newLow, newHigh) + } else { + List(lowValue, highValue) + } + } + + // Actually compute the best lambda + val lambdas = binSearch(lambdaLow, lambdaHigh).sortBy(_._1) + + // Find the best parameter set by taking the lowest cverror. + val (lambdaOpt, cverror, weights) = lambdas.reduce((a, b) => if (a._2 < b._2) a else b) + + // Return the model which contains the solution + val weightsScaled = weights.div(xColSd) + val intercept = yMean - (weights.transpose().mmul(xColMean.div(xColSd)).get(0)) + val model = new RidgeRegressionModel(weightsScaled, intercept, lambdaOpt, lambdas) + + logInfo("RidgeRegression: optimal lambda " + model.lambdaOpt) + logInfo("RidgeRegression: optimal weights " + model.weights) + logInfo("RidgeRegression: optimal intercept " + model.intercept) + logInfo("RidgeRegression: cross-validation error " + cverror) + + model + } +} +/** + * Top-level methods for calling Ridge Regression. + */ +object RidgeRegression { + + /** + * Train a ridge regression model given an RDD of (response, features) pairs. + * We use the closed form solution to compute the cross-validation score for + * a given lambda. The optimal lambda is computed by performing binary search + * between the provided bounds of lambda. + * + * @param input RDD of (response, array of features) pairs. + * @param lambdaLow lower bound used in binary search for lambda + * @param lambdaHigh upper bound used in binary search for lambda + */ + def train( + input: RDD[(Double, Array[Double])], + lambdaLow: Double, + lambdaHigh: Double) + : RidgeRegressionModel = + { + new RidgeRegression(lambdaLow, lambdaHigh).train(input) + } + + /** + * Train a ridge regression model given an RDD of (response, features) pairs. + * We use the closed form solution to compute the cross-validation score for + * a given lambda. The optimal lambda is computed by performing binary search + * between lambda values of 0 and 100. + * + * @param input RDD of (response, array of features) pairs. + */ + def train(input: RDD[(Double, Array[Double])]) : RidgeRegressionModel = { + train(input, 0.0, 100.0) + } + + def main(args: Array[String]) { + if (args.length != 2) { + println("Usage: RidgeRegression ") + System.exit(1) + } + val sc = new SparkContext(args(0), "RidgeRegression") + val data = MLUtils.loadData(sc, args(1)) + val model = RidgeRegression.train(data, 0, 1000) + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala new file mode 100644 index 0000000000..c9ac4a8b07 --- /dev/null +++ b/mllib/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala @@ -0,0 +1,55 @@ +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.saveData(data, outputPath) + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/ml/util/MLUtils.scala b/mllib/src/main/scala/spark/ml/util/MLUtils.scala new file mode 100644 index 0000000000..0a4a037c71 --- /dev/null +++ b/mllib/src/main/scala/spark/ml/util/MLUtils.scala @@ -0,0 +1,95 @@ +package spark.mllib.util + +import spark.{RDD, SparkContext} +import spark.SparkContext._ + +import org.jblas.DoubleMatrix + +/** + * Helper methods to load and save data + * Data format: + * , ... + * where , are feature values in Double and is the corresponding label as Double. + */ +object MLUtils { + + /** + * @param sc SparkContext + * @param dir Directory to the input data files. + * @return An RDD of tuples. For each tuple, the first element is the label, and the second + * element represents the feature values (an array of Double). + */ + def loadData(sc: SparkContext, dir: String): RDD[(Double, Array[Double])] = { + sc.textFile(dir).map { line => + val parts = line.split(",") + val label = parts(0).toDouble + val features = parts(1).trim().split(" ").map(_.toDouble) + (label, features) + } + } + + def saveData(data: RDD[(Double, Array[Double])], dir: String) { + val dataStr = data.map(x => x._1 + "," + x._2.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[(Double, Array[Double])], nfeatures: Int, nexamples: Long): + (Double, DoubleMatrix, DoubleMatrix) = { + val yMean: Double = data.map { case (y, features) => y }.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 { case(y, features) => + val nCols = features.length + // Traverse over every column and emit (col, value, value^2) + Iterator.tabulate(nCols) { i => + (i, (features(i), features(i)*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/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties new file mode 100644 index 0000000000..390c92763c --- /dev/null +++ b/mllib/src/test/resources/log4j.properties @@ -0,0 +1,11 @@ +# Set everything to be logged to the file core/target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=ml/target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN + diff --git a/mllib/src/test/scala/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/spark/ml/clustering/KMeansSuite.scala new file mode 100644 index 0000000000..ae7cf57c42 --- /dev/null +++ b/mllib/src/test/scala/spark/ml/clustering/KMeansSuite.scala @@ -0,0 +1,150 @@ +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 { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + val EPSILON = 1e-4 + + 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||") + 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||") + 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/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/spark/ml/recommendation/ALSSuite.scala new file mode 100644 index 0000000000..2ada9ae76b --- /dev/null +++ b/mllib/src/test/scala/spark/ml/recommendation/ALSSuite.scala @@ -0,0 +1,80 @@ +package spark.mllib.recommendation + +import scala.util.Random + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import spark.SparkContext +import spark.SparkContext._ + +import org.jblas._ + + +class ALSSuite extends FunSuite with BeforeAndAfterAll { + val 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 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 (u, p, trueRatings.get(u, p)) + } + + 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/ml/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala new file mode 100644 index 0000000000..04d3400cb4 --- /dev/null +++ b/mllib/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala @@ -0,0 +1,57 @@ +package spark.mllib.regression + +import scala.util.Random + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import spark.SparkContext +import spark.SparkContext._ + + +class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + // Test if we can correctly learn A, B where Y = logistic(A + B*X) + test("logistic regression") { + val nPoints = 10000 + val rnd = new Random(42) + + val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + + val A = 2.0 + val B = -1.5 + + // 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(100) + // y <- as.numeric(y > 0) + val y = (0 until nPoints).map { i => + val yVal = A + B * x1(i) + rLogis(i) + if (yVal > 0) 1.0 else 0.0 + } + + val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i)))).toArray + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + val lr = new LogisticRegression().setStepSize(10.0) + .setNumIterations(20) + + val model = lr.train(testRDD) + + val weight0 = model.weights.get(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]") + } +} diff --git a/mllib/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala new file mode 100644 index 0000000000..df41dbbdff --- /dev/null +++ b/mllib/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala @@ -0,0 +1,47 @@ +package spark.mllib.regression + +import scala.util.Random + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import spark.SparkContext +import spark.SparkContext._ + + +class RidgeRegressionSuite extends FunSuite with BeforeAndAfterAll { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + // Test if we can correctly learn Y = 3 + X1 + X2 when + // X1 and X2 are collinear. + test("multi-collinear variables") { + val rnd = new Random(43) + val x1 = Array.fill[Double](20)(rnd.nextGaussian()) + + // Pick a mean close to mean of x1 + val rnd1 = new Random(42) //new NormalDistribution(0.1, 0.01) + val x2 = Array.fill[Double](20)(0.1 + rnd1.nextGaussian() * 0.01) + + val xMat = (0 until 20).map(i => Array(x1(i), x2(i))).toArray + + val y = xMat.map(i => 3 + i(0) + i(1)) + val testData = (0 until 20).map(i => (y(i), xMat(i))).toArray + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + val ridgeReg = new RidgeRegression().setLowLambda(0) + .setHighLambda(10) + + val model = ridgeReg.train(testRDD) + + assert(model.intercept >= 2.9 && model.intercept <= 3.1) + assert(model.weights.length === 2) + assert(model.weights.get(0) >= 0.9 && model.weights.get(0) <= 1.1) + assert(model.weights.get(1) >= 0.9 && model.weights.get(1) <= 1.1) + } +} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5dbb5d4a65..c487f34d4a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -25,7 +25,7 @@ object SparkBuild extends Build { //val HADOOP_MAJOR_VERSION = "2" //val HADOOP_YARN = true - lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming, ml) + lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming, mllib) lazy val core = Project("core", file("core"), settings = coreSettings) @@ -37,7 +37,7 @@ object SparkBuild extends Build { lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn (core) - lazy val ml = Project("ml", file("ml"), settings = mlSettings) dependsOn (core) + lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn (core) // A configuration to set an alternative publishLocalConfiguration lazy val MavenCompile = config("m2r") extend(Compile) @@ -221,8 +221,8 @@ object SparkBuild extends Build { def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") - def mlSettings = sharedSettings ++ Seq( - name := "spark-ml", + def mllibSettings = sharedSettings ++ Seq( + name := "spark-mllib", libraryDependencies ++= Seq( "org.jblas" % "jblas" % "1.2.3" ) -- cgit v1.2.3 From de67deeaabc12a62dadee5ec302fe58bee0b3498 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 17:16:49 -0700 Subject: Addressed style comments from Ryan LeCompte --- mllib/src/main/scala/spark/ml/clustering/KMeans.scala | 14 ++++++-------- mllib/src/main/scala/spark/ml/recommendation/ALS.scala | 4 +++- .../main/scala/spark/ml/regression/RidgeRegression.scala | 8 ++++---- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/mllib/src/main/scala/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/spark/ml/clustering/KMeans.scala index 6d78f926c2..b0e141ff32 100644 --- a/mllib/src/main/scala/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/spark/ml/clustering/KMeans.scala @@ -100,7 +100,7 @@ class KMeans private ( val sc = data.sparkContext - var centers = if (initializationMode == KMeans.RANDOM) { + val centers = if (initializationMode == KMeans.RANDOM) { initRandom(data) } else { initKMeansParallel(data) @@ -131,13 +131,11 @@ class KMeans private ( val sums = Array.fill(runs, k)(new DoubleMatrix(dims)) val counts = Array.fill(runs, k)(0L) - for (point <- points) { - for ((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 - } + 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 { diff --git a/mllib/src/main/scala/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/spark/ml/recommendation/ALS.scala index 0c6fa6f741..6c9fb2359c 100644 --- a/mllib/src/main/scala/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/ml/recommendation/ALS.scala @@ -168,7 +168,9 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l val grouped = ratings.partitionBy(new HashPartitioner(numBlocks)) val links = grouped.mapPartitionsWithIndex((blockId, elements) => { val ratings = elements.map(_._2).toArray - Iterator((blockId, (makeInLinkBlock(numBlocks, ratings), makeOutLinkBlock(numBlocks, ratings)))) + 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)) diff --git a/mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala index 2d07c77141..a6ececbeb6 100644 --- a/mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala @@ -140,8 +140,8 @@ class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) object RidgeRegression { /** - * Train a ridge regression model given an RDD of (response, features) pairs. - * We use the closed form solution to compute the cross-validation score for + * Train a ridge regression model given an RDD of (response, features) pairs. + * We use the closed form solution to compute the cross-validation score for * a given lambda. The optimal lambda is computed by performing binary search * between the provided bounds of lambda. * @@ -159,8 +159,8 @@ object RidgeRegression { } /** - * Train a ridge regression model given an RDD of (response, features) pairs. - * We use the closed form solution to compute the cross-validation score for + * Train a ridge regression model given an RDD of (response, features) pairs. + * We use the closed form solution to compute the cross-validation score for * a given lambda. The optimal lambda is computed by performing binary search * between lambda values of 0 and 100. * -- cgit v1.2.3 From 653043beb6681c57a02a3e8dde837d7dbc1e44bf Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 17:18:55 -0700 Subject: Renamed files to match package --- .../main/scala/spark/ml/clustering/KMeans.scala | 317 ----------------- .../scala/spark/ml/clustering/KMeansModel.scala | 27 -- .../scala/spark/ml/clustering/LocalKMeans.scala | 88 ----- .../scala/spark/ml/optimization/Gradient.scala | 33 -- .../spark/ml/optimization/GradientDescent.scala | 62 ---- .../main/scala/spark/ml/optimization/Updater.scala | 27 -- .../main/scala/spark/ml/recommendation/ALS.scala | 389 --------------------- .../recommendation/MatrixFactorizationModel.scala | 23 -- .../spark/ml/regression/LogisticRegression.scala | 158 --------- .../regression/LogisticRegressionGenerator.scala | 41 --- .../scala/spark/ml/regression/Regression.scala | 21 -- .../spark/ml/regression/RidgeRegression.scala | 183 ---------- .../ml/regression/RidgeRegressionGenerator.scala | 55 --- mllib/src/main/scala/spark/ml/util/MLUtils.scala | 95 ----- .../main/scala/spark/mllib/clustering/KMeans.scala | 317 +++++++++++++++++ .../scala/spark/mllib/clustering/KMeansModel.scala | 27 ++ .../scala/spark/mllib/clustering/LocalKMeans.scala | 88 +++++ .../scala/spark/mllib/optimization/Gradient.scala | 33 ++ .../spark/mllib/optimization/GradientDescent.scala | 62 ++++ .../scala/spark/mllib/optimization/Updater.scala | 27 ++ .../scala/spark/mllib/recommendation/ALS.scala | 389 +++++++++++++++++++++ .../recommendation/MatrixFactorizationModel.scala | 23 ++ .../mllib/regression/LogisticRegression.scala | 158 +++++++++ .../regression/LogisticRegressionGenerator.scala | 41 +++ .../scala/spark/mllib/regression/Regression.scala | 21 ++ .../spark/mllib/regression/RidgeRegression.scala | 183 ++++++++++ .../regression/RidgeRegressionGenerator.scala | 55 +++ .../src/main/scala/spark/mllib/util/MLUtils.scala | 95 +++++ .../scala/spark/ml/clustering/KMeansSuite.scala | 150 -------- .../scala/spark/ml/recommendation/ALSSuite.scala | 80 ----- .../ml/regression/LogisticRegressionSuite.scala | 57 --- .../spark/ml/regression/RidgeRegressionSuite.scala | 47 --- .../scala/spark/mllib/clustering/KMeansSuite.scala | 150 ++++++++ .../spark/mllib/recommendation/ALSSuite.scala | 80 +++++ .../mllib/regression/LogisticRegressionSuite.scala | 57 +++ .../mllib/regression/RidgeRegressionSuite.scala | 47 +++ 36 files changed, 1853 insertions(+), 1853 deletions(-) delete mode 100644 mllib/src/main/scala/spark/ml/clustering/KMeans.scala delete mode 100644 mllib/src/main/scala/spark/ml/clustering/KMeansModel.scala delete mode 100644 mllib/src/main/scala/spark/ml/clustering/LocalKMeans.scala delete mode 100644 mllib/src/main/scala/spark/ml/optimization/Gradient.scala delete mode 100644 mllib/src/main/scala/spark/ml/optimization/GradientDescent.scala delete mode 100644 mllib/src/main/scala/spark/ml/optimization/Updater.scala delete mode 100644 mllib/src/main/scala/spark/ml/recommendation/ALS.scala delete mode 100644 mllib/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala delete mode 100644 mllib/src/main/scala/spark/ml/regression/LogisticRegression.scala delete mode 100644 mllib/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala delete mode 100644 mllib/src/main/scala/spark/ml/regression/Regression.scala delete mode 100644 mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala delete mode 100644 mllib/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala delete mode 100644 mllib/src/main/scala/spark/ml/util/MLUtils.scala create mode 100644 mllib/src/main/scala/spark/mllib/clustering/KMeans.scala create mode 100644 mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala create mode 100644 mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala create mode 100644 mllib/src/main/scala/spark/mllib/optimization/Gradient.scala create mode 100644 mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala create mode 100644 mllib/src/main/scala/spark/mllib/optimization/Updater.scala create mode 100644 mllib/src/main/scala/spark/mllib/recommendation/ALS.scala create mode 100644 mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala create mode 100644 mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala create mode 100644 mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala create mode 100644 mllib/src/main/scala/spark/mllib/regression/Regression.scala create mode 100644 mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala create mode 100644 mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala create mode 100644 mllib/src/main/scala/spark/mllib/util/MLUtils.scala delete mode 100644 mllib/src/test/scala/spark/ml/clustering/KMeansSuite.scala delete mode 100644 mllib/src/test/scala/spark/ml/recommendation/ALSSuite.scala delete mode 100644 mllib/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala delete mode 100644 mllib/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala create mode 100644 mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala create mode 100644 mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala create mode 100644 mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala create mode 100644 mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala diff --git a/mllib/src/main/scala/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/spark/ml/clustering/KMeans.scala deleted file mode 100644 index b0e141ff32..0000000000 --- a/mllib/src/main/scala/spark/ml/clustering/KMeans.scala +++ /dev/null @@ -1,317 +0,0 @@ -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 train(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()) - Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k)) - } - - /** - * 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) - 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) - .train(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 sc = new SparkContext(master, "KMeans") - val data = sc.textFile(inputFile).map(line => line.split(' ').map(_.toDouble)) - val model = KMeans.train(data, k, iters) - 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/ml/clustering/KMeansModel.scala b/mllib/src/main/scala/spark/ml/clustering/KMeansModel.scala deleted file mode 100644 index 4fd0646160..0000000000 --- a/mllib/src/main/scala/spark/ml/clustering/KMeansModel.scala +++ /dev/null @@ -1,27 +0,0 @@ -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/ml/clustering/LocalKMeans.scala b/mllib/src/main/scala/spark/ml/clustering/LocalKMeans.scala deleted file mode 100644 index e12b3be251..0000000000 --- a/mllib/src/main/scala/spark/ml/clustering/LocalKMeans.scala +++ /dev/null @@ -1,88 +0,0 @@ -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/ml/optimization/Gradient.scala b/mllib/src/main/scala/spark/ml/optimization/Gradient.scala deleted file mode 100644 index 90b0999a5e..0000000000 --- a/mllib/src/main/scala/spark/ml/optimization/Gradient.scala +++ /dev/null @@ -1,33 +0,0 @@ -package spark.mllib.optimization - -import org.jblas.DoubleMatrix - -abstract class Gradient extends Serializable { - /** - * Compute the gradient for a given row of data. - * - * @param data - One row of data. Row matrix of size 1xn where n is the number of features. - * @param label - Label for this data item. - * @param weights - Column matrix containing weights for every feature. - */ - def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): - (DoubleMatrix, Double) -} - -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) - } -} diff --git a/mllib/src/main/scala/spark/ml/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/ml/optimization/GradientDescent.scala deleted file mode 100644 index eff853f379..0000000000 --- a/mllib/src/main/scala/spark/ml/optimization/GradientDescent.scala +++ /dev/null @@ -1,62 +0,0 @@ -package spark.mllib.optimization - -import spark.{Logging, RDD, SparkContext} -import spark.SparkContext._ - -import org.jblas.DoubleMatrix - -import scala.collection.mutable.ArrayBuffer - - -object GradientDescent { - - /** - * Run gradient descent in parallel using mini batches. - * Based on Matlab code written by John Duchi. - * - * @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 numIters - number of iterations that SGD should be run. - * @param miniBatchFraction - fraction of the input data set that should be used for - * one iteration of SGD. Default value 1.0. - * - * @return weights - Column matrix containing weights for every feature. - * @return lossHistory - Array containing the loss computed for every iteration. - */ - def runMiniBatchSGD( - data: RDD[(Double, Array[Double])], - gradient: Gradient, - updater: Updater, - stepSize: Double, - numIters: Int, - miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { - - val lossHistory = new ArrayBuffer[Double](numIters) - - val nfeatures: Int = data.take(1)(0)._2.length - val nexamples: Long = data.count() - val miniBatchSize = nexamples * miniBatchFraction - - // Initialize weights as a column matrix - var weights = DoubleMatrix.ones(nfeatures) - var reg_val = 0.0 - - for (i <- 1 to numIters) { - val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42+i).map { - case (y, features) => - val featuresRow = new DoubleMatrix(features.length, 1, features:_*) - val (grad, loss) = gradient.compute(featuresRow, y, weights) - (grad, loss) - }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) - - lossHistory.append(lossSum / miniBatchSize + reg_val) - val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i) - weights = update._1 - reg_val = update._2 - } - - (weights, lossHistory.toArray) - } -} diff --git a/mllib/src/main/scala/spark/ml/optimization/Updater.scala b/mllib/src/main/scala/spark/ml/optimization/Updater.scala deleted file mode 100644 index ea80bfcbfd..0000000000 --- a/mllib/src/main/scala/spark/ml/optimization/Updater.scala +++ /dev/null @@ -1,27 +0,0 @@ -package spark.mllib.optimization - -import org.jblas.DoubleMatrix - -abstract class Updater extends Serializable { - /** - * Compute an updated value for weights given the gradient, stepSize and iteration number. - * - * @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 - * - * @return weightsNew - Column matrix containing updated weights - * @return reg_val - regularization value - */ - def compute(weightsOlds: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): - (DoubleMatrix, Double) -} - -class SimpleUpdater extends Updater { - override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, - stepSize: Double, iter: Int): (DoubleMatrix, Double) = { - val normGradient = gradient.mul(stepSize / math.sqrt(iter)) - (weightsOld.sub(normGradient), 0) - } -} diff --git a/mllib/src/main/scala/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/spark/ml/recommendation/ALS.scala deleted file mode 100644 index 6c9fb2359c..0000000000 --- a/mllib/src/main/scala/spark/ml/recommendation/ALS.scala +++ /dev/null @@ -1,389 +0,0 @@ -package spark.mllib.recommendation - -import scala.collection.mutable.{ArrayBuffer, BitSet} -import scala.util.Random - -import spark.{HashPartitioner, Partitioner, SparkContext, RDD} -import spark.storage.StorageLevel -import spark.SparkContext._ - -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])]]) - - -/** - * 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 parmeters on an input RDD of (user, product, rating) triples. - * Returns a MatrixFactorizationModel with feature vectors for each user and product. - */ - def train(ratings: RDD[(Int, Int, Double)]): MatrixFactorizationModel = { - val numBlocks = if (this.numBlocks == -1) { - math.max(ratings.context.defaultParallelism, ratings.partitions.size) - } else { - this.numBlocks - } - - val partitioner = new HashPartitioner(numBlocks) - - val ratingsByUserBlock = ratings.map{ case (u, p, r) => (u % numBlocks, (u, p, r)) } - val ratingsByProductBlock = ratings.map{ case (u, p, r) => (p % numBlocks, (p, u, r)) } - - val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock) - val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) - - // Initialize user and product factors randomly - val seed = new Random().nextInt() - var users = userOutLinks.mapValues(_.elementIds.map(u => randomFactor(rank, seed ^ u))) - var products = productOutLinks.mapValues(_.elementIds.map(p => randomFactor(rank, seed ^ ~p))) - - 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[(Int, Int, Double)]): OutLinkBlock = { - val userIds = ratings.map(_._1).distinct.sorted - val numUsers = userIds.length - val userIdToPos = userIds.zipWithIndex.toMap - val shouldSend = Array.fill(numUsers)(new BitSet(numBlocks)) - for ((u, p, r) <- ratings) { - shouldSend(userIdToPos(u))(p % 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[(Int, Int, Double)]): InLinkBlock = { - val userIds = ratings.map(_._1).distinct.sorted - val numUsers = userIds.length - val userIdToPos = userIds.zipWithIndex.toMap - val ratingsForBlock = new Array[Array[(Array[Int], Array[Double])]](numBlocks) - for (productBlock <- 0 until numBlocks) { - val ratingsInBlock = ratings.filter(t => t._2 % numBlocks == productBlock) - val ratingsByProduct = ratingsInBlock.groupBy(_._2) // (p, Seq[(u, p, r)]) - .toArray - .sortBy(_._1) - .map{case (p, rs) => (rs.map(t => userIdToPos(t._1)), rs.map(_._3))} - ratingsForBlock(productBlock) = ratingsByProduct - } - 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, (Int, Int, Double))]) - : (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 seed. - * TODO: Initialize things using mapPartitionsWithIndex to make it faster? - */ - private def randomFactor(rank: Int, seed: Int): Array[Double] = { - val rand = new Random(seed) - 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[(Int, Int, Double)], - rank: Int, - iterations: Int, - lambda: Double, - blocks: Int) - : MatrixFactorizationModel = - { - new ALS(blocks, rank, iterations, lambda).train(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[(Int, Int, Double)], 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[(Int, Int, Double)], rank: Int, iterations: Int) - : MatrixFactorizationModel = - { - train(ratings, rank, iterations, 0.01, -1) - } - - def main(args: Array[String]) { - if (args.length != 5) { - 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 sc = new SparkContext(master, "ALS") - val ratings = sc.textFile(ratingsFile).map { line => - val fields = line.split(',') - (fields(0).toInt, fields(1).toInt, fields(2).toDouble) - } - val model = ALS.train(ratings, rank, iters) - 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/ml/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala deleted file mode 100644 index fb812a6dbe..0000000000 --- a/mllib/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala +++ /dev/null @@ -1,23 +0,0 @@ -package spark.mllib.recommendation - -import spark.RDD -import spark.SparkContext._ - -import org.jblas._ - -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/ml/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/ml/regression/LogisticRegression.scala deleted file mode 100644 index 448ab9dce9..0000000000 --- a/mllib/src/main/scala/spark/ml/regression/LogisticRegression.scala +++ /dev/null @@ -1,158 +0,0 @@ -package spark.mllib.regression - -import spark.{Logging, RDD, SparkContext} -import spark.mllib.optimization._ -import spark.mllib.util.MLUtils - -import org.jblas.DoubleMatrix - -/** - * Logistic Regression using Stochastic Gradient Descent. - * Based on Matlab code written by John Duchi. - */ -class LogisticRegressionModel( - val weights: DoubleMatrix, - val intercept: Double, - val losses: Array[Double]) extends RegressionModel { - - override def predict(testData: spark.RDD[Array[Double]]) = { - testData.map { x => - val margin = new DoubleMatrix(1, x.length, x:_*).mmul(this.weights).get(0) + this.intercept - 1.0/ (1.0 + math.exp(margin * -1)) - } - } - - override def predict(testData: Array[Double]): Double = { - val dataMat = new DoubleMatrix(1, testData.length, testData:_*) - val margin = dataMat.mmul(this.weights).get(0) + this.intercept - 1.0/ (1.0 + math.exp(margin * -1)) - } -} - -class LogisticRegression private (var stepSize: Double, var miniBatchFraction: Double, - var numIters: Int) - extends Logging { - - /** - * Construct a LogisticRegression object with default parameters - */ - def this() = this(1.0, 1.0, 100) - - /** - * Set the step size per-iteration of SGD. Default 1.0. - */ - def setStepSize(step: Double) = { - this.stepSize = step - this - } - - /** - * Set fraction of data to be used for each SGD iteration. Default 1.0. - */ - def setMiniBatchFraction(fraction: Double) = { - this.miniBatchFraction = fraction - this - } - - /** - * Set the number of iterations for SGD. Default 100. - */ - def setNumIterations(iters: Int) = { - this.numIters = iters - this - } - - def train(input: RDD[(Double, Array[Double])]): LogisticRegressionModel = { - // Add a extra variable consisting of all 1.0's for the intercept. - val data = input.map { case (y, features) => - (y, Array(1.0, features:_*)) - } - - val (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, miniBatchFraction) - - val weightsScaled = weights.getRange(1, weights.length) - val intercept = weights.get(0) - - val model = new LogisticRegressionModel(weightsScaled, intercept, losses) - - logInfo("Final model weights " + model.weights) - logInfo("Final model intercept " + model.intercept) - logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) - model - } -} - -/** - * Top-level methods for calling Logistic Regression. - */ -object LogisticRegression { - - /** - * 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. - * - * @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[(Double, Array[Double])], - numIterations: Int, - stepSize: Double, - miniBatchFraction: Double) - : LogisticRegressionModel = - { - new LogisticRegression(stepSize, miniBatchFraction, numIterations).train(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. - * - * @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[(Double, Array[Double])], - 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. - * - * @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[(Double, Array[Double])], - 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.loadData(sc, args(1)) - val model = LogisticRegression.train(data, args(3).toInt, args(2).toDouble) - - sc.stop() - } -} diff --git a/mllib/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala deleted file mode 100644 index 9f6abab70b..0000000000 --- a/mllib/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala +++ /dev/null @@ -1,41 +0,0 @@ -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.saveData(data, outputPath) - sc.stop() - } -} diff --git a/mllib/src/main/scala/spark/ml/regression/Regression.scala b/mllib/src/main/scala/spark/ml/regression/Regression.scala deleted file mode 100644 index f79974c191..0000000000 --- a/mllib/src/main/scala/spark/ml/regression/Regression.scala +++ /dev/null @@ -1,21 +0,0 @@ -package spark.mllib.regression - -import spark.RDD - -trait RegressionModel { - /** - * 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/ml/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala deleted file mode 100644 index a6ececbeb6..0000000000 --- a/mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala +++ /dev/null @@ -1,183 +0,0 @@ -package spark.mllib.regression - -import spark.{Logging, RDD, SparkContext} -import spark.SparkContext._ -import spark.mllib.util.MLUtils - -import org.jblas.DoubleMatrix -import org.jblas.Solve - -/** - * Ridge Regression from Joseph Gonzalez's implementation in MLBase - */ -class RidgeRegressionModel( - val weights: DoubleMatrix, - val intercept: Double, - val lambdaOpt: Double, - val lambdas: List[(Double, Double, DoubleMatrix)]) - extends RegressionModel { - - override def predict(testData: RDD[Array[Double]]): RDD[Double] = { - testData.map { x => - (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept - } - } - - override def predict(testData: Array[Double]): Double = { - (new DoubleMatrix(1, testData.length, testData:_*).mmul(this.weights)).get(0) + this.intercept - } -} - -class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) - extends Logging { - - def this() = this(0.0, 100.0) - - /** - * Set the lower bound on binary search for lambda's. Default is 0. - */ - def setLowLambda(low: Double) = { - this.lambdaLow = low - this - } - - /** - * Set the upper bound on binary search for lambda's. Default is 100.0. - */ - def setHighLambda(hi: Double) = { - this.lambdaHigh = hi - this - } - - def train(input: RDD[(Double, Array[Double])]): RidgeRegressionModel = { - val nfeatures: Int = input.take(1)(0)._2.length - val nexamples: Long = input.count() - - val (yMean, xColMean, xColSd) = MLUtils.computeStats(input, nfeatures, nexamples) - - val data = input.map { case(y, features) => - val yNormalized = y - yMean - val featuresMat = new DoubleMatrix(nfeatures, 1, features:_*) - val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd) - (yNormalized, featuresNormalized.toArray) - } - - // Compute XtX - Size of XtX is nfeatures by nfeatures - val XtX: DoubleMatrix = data.map { case (y, features) => - val x = new DoubleMatrix(1, features.length, features:_*) - x.transpose().mmul(x) - }.reduce(_.addi(_)) - - // Compute Xt*y - Size of Xty is nfeatures by 1 - val Xty: DoubleMatrix = data.map { case (y, features) => - new DoubleMatrix(features.length, 1, features:_*).mul(y) - }.reduce(_.addi(_)) - - // Define a function to compute the leave one out cross validation error - // for a single example - def crossValidate(lambda: Double): (Double, Double, DoubleMatrix) = { - // Compute the MLE ridge regression parameter value - - // Ridge Regression parameter = inv(XtX + \lambda*I) * Xty - val XtXlambda = DoubleMatrix.eye(nfeatures).muli(lambda).addi(XtX) - val w = Solve.solveSymmetric(XtXlambda, Xty) - - val invXtX = Solve.solveSymmetric(XtXlambda, DoubleMatrix.eye(nfeatures)) - - // compute the generalized cross validation score - val cverror = data.map { - case (y, features) => - val x = new DoubleMatrix(features.length, 1, features:_*) - val yhat = w.transpose().mmul(x).get(0) - val H_ii = x.transpose().mmul(invXtX).mmul(x).get(0) - val residual = (y - yhat) / (1.0 - H_ii) - residual * residual - }.reduce(_ + _) / nexamples - - (lambda, cverror, w) - } - - // Binary search for the best assignment to lambda. - def binSearch(low: Double, high: Double): List[(Double, Double, DoubleMatrix)] = { - val mid = (high - low) / 2 + low - val lowValue = crossValidate((mid - low) / 2 + low) - val highValue = crossValidate((high - mid) / 2 + mid) - val (newLow, newHigh) = if (lowValue._2 < highValue._2) { - (low, mid + (high-low)/4) - } else { - (mid - (high-low)/4, high) - } - if (newHigh - newLow > 1.0E-7) { - // :: is list prepend in Scala. - lowValue :: highValue :: binSearch(newLow, newHigh) - } else { - List(lowValue, highValue) - } - } - - // Actually compute the best lambda - val lambdas = binSearch(lambdaLow, lambdaHigh).sortBy(_._1) - - // Find the best parameter set by taking the lowest cverror. - val (lambdaOpt, cverror, weights) = lambdas.reduce((a, b) => if (a._2 < b._2) a else b) - - // Return the model which contains the solution - val weightsScaled = weights.div(xColSd) - val intercept = yMean - (weights.transpose().mmul(xColMean.div(xColSd)).get(0)) - val model = new RidgeRegressionModel(weightsScaled, intercept, lambdaOpt, lambdas) - - logInfo("RidgeRegression: optimal lambda " + model.lambdaOpt) - logInfo("RidgeRegression: optimal weights " + model.weights) - logInfo("RidgeRegression: optimal intercept " + model.intercept) - logInfo("RidgeRegression: cross-validation error " + cverror) - - model - } -} -/** - * Top-level methods for calling Ridge Regression. - */ -object RidgeRegression { - - /** - * Train a ridge regression model given an RDD of (response, features) pairs. - * We use the closed form solution to compute the cross-validation score for - * a given lambda. The optimal lambda is computed by performing binary search - * between the provided bounds of lambda. - * - * @param input RDD of (response, array of features) pairs. - * @param lambdaLow lower bound used in binary search for lambda - * @param lambdaHigh upper bound used in binary search for lambda - */ - def train( - input: RDD[(Double, Array[Double])], - lambdaLow: Double, - lambdaHigh: Double) - : RidgeRegressionModel = - { - new RidgeRegression(lambdaLow, lambdaHigh).train(input) - } - - /** - * Train a ridge regression model given an RDD of (response, features) pairs. - * We use the closed form solution to compute the cross-validation score for - * a given lambda. The optimal lambda is computed by performing binary search - * between lambda values of 0 and 100. - * - * @param input RDD of (response, array of features) pairs. - */ - def train(input: RDD[(Double, Array[Double])]) : RidgeRegressionModel = { - train(input, 0.0, 100.0) - } - - def main(args: Array[String]) { - if (args.length != 2) { - println("Usage: RidgeRegression ") - System.exit(1) - } - val sc = new SparkContext(args(0), "RidgeRegression") - val data = MLUtils.loadData(sc, args(1)) - val model = RidgeRegression.train(data, 0, 1000) - sc.stop() - } -} diff --git a/mllib/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala deleted file mode 100644 index c9ac4a8b07..0000000000 --- a/mllib/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala +++ /dev/null @@ -1,55 +0,0 @@ -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.saveData(data, outputPath) - sc.stop() - } -} diff --git a/mllib/src/main/scala/spark/ml/util/MLUtils.scala b/mllib/src/main/scala/spark/ml/util/MLUtils.scala deleted file mode 100644 index 0a4a037c71..0000000000 --- a/mllib/src/main/scala/spark/ml/util/MLUtils.scala +++ /dev/null @@ -1,95 +0,0 @@ -package spark.mllib.util - -import spark.{RDD, SparkContext} -import spark.SparkContext._ - -import org.jblas.DoubleMatrix - -/** - * Helper methods to load and save data - * Data format: - * , ... - * where , are feature values in Double and is the corresponding label as Double. - */ -object MLUtils { - - /** - * @param sc SparkContext - * @param dir Directory to the input data files. - * @return An RDD of tuples. For each tuple, the first element is the label, and the second - * element represents the feature values (an array of Double). - */ - def loadData(sc: SparkContext, dir: String): RDD[(Double, Array[Double])] = { - sc.textFile(dir).map { line => - val parts = line.split(",") - val label = parts(0).toDouble - val features = parts(1).trim().split(" ").map(_.toDouble) - (label, features) - } - } - - def saveData(data: RDD[(Double, Array[Double])], dir: String) { - val dataStr = data.map(x => x._1 + "," + x._2.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[(Double, Array[Double])], nfeatures: Int, nexamples: Long): - (Double, DoubleMatrix, DoubleMatrix) = { - val yMean: Double = data.map { case (y, features) => y }.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 { case(y, features) => - val nCols = features.length - // Traverse over every column and emit (col, value, value^2) - Iterator.tabulate(nCols) { i => - (i, (features(i), features(i)*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/clustering/KMeans.scala b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala new file mode 100644 index 0000000000..b0e141ff32 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala @@ -0,0 +1,317 @@ +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 train(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()) + Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k)) + } + + /** + * 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) + 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) + .train(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 sc = new SparkContext(master, "KMeans") + val data = sc.textFile(inputFile).map(line => line.split(' ').map(_.toDouble)) + val model = KMeans.train(data, k, iters) + 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 new file mode 100644 index 0000000000..4fd0646160 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala @@ -0,0 +1,27 @@ +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 new file mode 100644 index 0000000000..e12b3be251 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala @@ -0,0 +1,88 @@ +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 new file mode 100644 index 0000000000..90b0999a5e --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala @@ -0,0 +1,33 @@ +package spark.mllib.optimization + +import org.jblas.DoubleMatrix + +abstract class Gradient extends Serializable { + /** + * Compute the gradient for a given row of data. + * + * @param data - One row of data. Row matrix of size 1xn where n is the number of features. + * @param label - Label for this data item. + * @param weights - Column matrix containing weights for every feature. + */ + def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): + (DoubleMatrix, Double) +} + +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) + } +} diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala new file mode 100644 index 0000000000..eff853f379 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -0,0 +1,62 @@ +package spark.mllib.optimization + +import spark.{Logging, RDD, SparkContext} +import spark.SparkContext._ + +import org.jblas.DoubleMatrix + +import scala.collection.mutable.ArrayBuffer + + +object GradientDescent { + + /** + * Run gradient descent in parallel using mini batches. + * Based on Matlab code written by John Duchi. + * + * @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 numIters - number of iterations that SGD should be run. + * @param miniBatchFraction - fraction of the input data set that should be used for + * one iteration of SGD. Default value 1.0. + * + * @return weights - Column matrix containing weights for every feature. + * @return lossHistory - Array containing the loss computed for every iteration. + */ + def runMiniBatchSGD( + data: RDD[(Double, Array[Double])], + gradient: Gradient, + updater: Updater, + stepSize: Double, + numIters: Int, + miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { + + val lossHistory = new ArrayBuffer[Double](numIters) + + val nfeatures: Int = data.take(1)(0)._2.length + val nexamples: Long = data.count() + val miniBatchSize = nexamples * miniBatchFraction + + // Initialize weights as a column matrix + var weights = DoubleMatrix.ones(nfeatures) + var reg_val = 0.0 + + for (i <- 1 to numIters) { + val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42+i).map { + case (y, features) => + val featuresRow = new DoubleMatrix(features.length, 1, features:_*) + val (grad, loss) = gradient.compute(featuresRow, y, weights) + (grad, loss) + }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) + + lossHistory.append(lossSum / miniBatchSize + reg_val) + val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i) + weights = update._1 + reg_val = update._2 + } + + (weights, lossHistory.toArray) + } +} diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala new file mode 100644 index 0000000000..ea80bfcbfd --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -0,0 +1,27 @@ +package spark.mllib.optimization + +import org.jblas.DoubleMatrix + +abstract class Updater extends Serializable { + /** + * Compute an updated value for weights given the gradient, stepSize and iteration number. + * + * @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 + * + * @return weightsNew - Column matrix containing updated weights + * @return reg_val - regularization value + */ + def compute(weightsOlds: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): + (DoubleMatrix, Double) +} + +class SimpleUpdater extends Updater { + override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, + stepSize: Double, iter: Int): (DoubleMatrix, Double) = { + val normGradient = gradient.mul(stepSize / math.sqrt(iter)) + (weightsOld.sub(normGradient), 0) + } +} diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala new file mode 100644 index 0000000000..6c9fb2359c --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -0,0 +1,389 @@ +package spark.mllib.recommendation + +import scala.collection.mutable.{ArrayBuffer, BitSet} +import scala.util.Random + +import spark.{HashPartitioner, Partitioner, SparkContext, RDD} +import spark.storage.StorageLevel +import spark.SparkContext._ + +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])]]) + + +/** + * 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 parmeters on an input RDD of (user, product, rating) triples. + * Returns a MatrixFactorizationModel with feature vectors for each user and product. + */ + def train(ratings: RDD[(Int, Int, Double)]): MatrixFactorizationModel = { + val numBlocks = if (this.numBlocks == -1) { + math.max(ratings.context.defaultParallelism, ratings.partitions.size) + } else { + this.numBlocks + } + + val partitioner = new HashPartitioner(numBlocks) + + val ratingsByUserBlock = ratings.map{ case (u, p, r) => (u % numBlocks, (u, p, r)) } + val ratingsByProductBlock = ratings.map{ case (u, p, r) => (p % numBlocks, (p, u, r)) } + + val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock) + val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) + + // Initialize user and product factors randomly + val seed = new Random().nextInt() + var users = userOutLinks.mapValues(_.elementIds.map(u => randomFactor(rank, seed ^ u))) + var products = productOutLinks.mapValues(_.elementIds.map(p => randomFactor(rank, seed ^ ~p))) + + 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[(Int, Int, Double)]): OutLinkBlock = { + val userIds = ratings.map(_._1).distinct.sorted + val numUsers = userIds.length + val userIdToPos = userIds.zipWithIndex.toMap + val shouldSend = Array.fill(numUsers)(new BitSet(numBlocks)) + for ((u, p, r) <- ratings) { + shouldSend(userIdToPos(u))(p % 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[(Int, Int, Double)]): InLinkBlock = { + val userIds = ratings.map(_._1).distinct.sorted + val numUsers = userIds.length + val userIdToPos = userIds.zipWithIndex.toMap + val ratingsForBlock = new Array[Array[(Array[Int], Array[Double])]](numBlocks) + for (productBlock <- 0 until numBlocks) { + val ratingsInBlock = ratings.filter(t => t._2 % numBlocks == productBlock) + val ratingsByProduct = ratingsInBlock.groupBy(_._2) // (p, Seq[(u, p, r)]) + .toArray + .sortBy(_._1) + .map{case (p, rs) => (rs.map(t => userIdToPos(t._1)), rs.map(_._3))} + ratingsForBlock(productBlock) = ratingsByProduct + } + 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, (Int, Int, Double))]) + : (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 seed. + * TODO: Initialize things using mapPartitionsWithIndex to make it faster? + */ + private def randomFactor(rank: Int, seed: Int): Array[Double] = { + val rand = new Random(seed) + 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[(Int, Int, Double)], + rank: Int, + iterations: Int, + lambda: Double, + blocks: Int) + : MatrixFactorizationModel = + { + new ALS(blocks, rank, iterations, lambda).train(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[(Int, Int, Double)], 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[(Int, Int, Double)], rank: Int, iterations: Int) + : MatrixFactorizationModel = + { + train(ratings, rank, iterations, 0.01, -1) + } + + def main(args: Array[String]) { + if (args.length != 5) { + 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 sc = new SparkContext(master, "ALS") + val ratings = sc.textFile(ratingsFile).map { line => + val fields = line.split(',') + (fields(0).toInt, fields(1).toInt, fields(2).toDouble) + } + val model = ALS.train(ratings, rank, iters) + 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 new file mode 100644 index 0000000000..fb812a6dbe --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -0,0 +1,23 @@ +package spark.mllib.recommendation + +import spark.RDD +import spark.SparkContext._ + +import org.jblas._ + +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/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala new file mode 100644 index 0000000000..448ab9dce9 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -0,0 +1,158 @@ +package spark.mllib.regression + +import spark.{Logging, RDD, SparkContext} +import spark.mllib.optimization._ +import spark.mllib.util.MLUtils + +import org.jblas.DoubleMatrix + +/** + * Logistic Regression using Stochastic Gradient Descent. + * Based on Matlab code written by John Duchi. + */ +class LogisticRegressionModel( + val weights: DoubleMatrix, + val intercept: Double, + val losses: Array[Double]) extends RegressionModel { + + override def predict(testData: spark.RDD[Array[Double]]) = { + testData.map { x => + val margin = new DoubleMatrix(1, x.length, x:_*).mmul(this.weights).get(0) + this.intercept + 1.0/ (1.0 + math.exp(margin * -1)) + } + } + + override def predict(testData: Array[Double]): Double = { + val dataMat = new DoubleMatrix(1, testData.length, testData:_*) + val margin = dataMat.mmul(this.weights).get(0) + this.intercept + 1.0/ (1.0 + math.exp(margin * -1)) + } +} + +class LogisticRegression private (var stepSize: Double, var miniBatchFraction: Double, + var numIters: Int) + extends Logging { + + /** + * Construct a LogisticRegression object with default parameters + */ + def this() = this(1.0, 1.0, 100) + + /** + * Set the step size per-iteration of SGD. Default 1.0. + */ + def setStepSize(step: Double) = { + this.stepSize = step + this + } + + /** + * Set fraction of data to be used for each SGD iteration. Default 1.0. + */ + def setMiniBatchFraction(fraction: Double) = { + this.miniBatchFraction = fraction + this + } + + /** + * Set the number of iterations for SGD. Default 100. + */ + def setNumIterations(iters: Int) = { + this.numIters = iters + this + } + + def train(input: RDD[(Double, Array[Double])]): LogisticRegressionModel = { + // Add a extra variable consisting of all 1.0's for the intercept. + val data = input.map { case (y, features) => + (y, Array(1.0, features:_*)) + } + + val (weights, losses) = GradientDescent.runMiniBatchSGD( + data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, miniBatchFraction) + + val weightsScaled = weights.getRange(1, weights.length) + val intercept = weights.get(0) + + val model = new LogisticRegressionModel(weightsScaled, intercept, losses) + + logInfo("Final model weights " + model.weights) + logInfo("Final model intercept " + model.intercept) + logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) + model + } +} + +/** + * Top-level methods for calling Logistic Regression. + */ +object LogisticRegression { + + /** + * 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. + * + * @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[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double) + : LogisticRegressionModel = + { + new LogisticRegression(stepSize, miniBatchFraction, numIterations).train(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. + * + * @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[(Double, Array[Double])], + 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. + * + * @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[(Double, Array[Double])], + 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.loadData(sc, args(1)) + val model = LogisticRegression.train(data, args(3).toInt, args(2).toDouble) + + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala new file mode 100644 index 0000000000..9f6abab70b --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala @@ -0,0 +1,41 @@ +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.saveData(data, outputPath) + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/mllib/regression/Regression.scala b/mllib/src/main/scala/spark/mllib/regression/Regression.scala new file mode 100644 index 0000000000..f79974c191 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/regression/Regression.scala @@ -0,0 +1,21 @@ +package spark.mllib.regression + +import spark.RDD + +trait RegressionModel { + /** + * 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 new file mode 100644 index 0000000000..a6ececbeb6 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -0,0 +1,183 @@ +package spark.mllib.regression + +import spark.{Logging, RDD, SparkContext} +import spark.SparkContext._ +import spark.mllib.util.MLUtils + +import org.jblas.DoubleMatrix +import org.jblas.Solve + +/** + * Ridge Regression from Joseph Gonzalez's implementation in MLBase + */ +class RidgeRegressionModel( + val weights: DoubleMatrix, + val intercept: Double, + val lambdaOpt: Double, + val lambdas: List[(Double, Double, DoubleMatrix)]) + extends RegressionModel { + + override def predict(testData: RDD[Array[Double]]): RDD[Double] = { + testData.map { x => + (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept + } + } + + override def predict(testData: Array[Double]): Double = { + (new DoubleMatrix(1, testData.length, testData:_*).mmul(this.weights)).get(0) + this.intercept + } +} + +class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) + extends Logging { + + def this() = this(0.0, 100.0) + + /** + * Set the lower bound on binary search for lambda's. Default is 0. + */ + def setLowLambda(low: Double) = { + this.lambdaLow = low + this + } + + /** + * Set the upper bound on binary search for lambda's. Default is 100.0. + */ + def setHighLambda(hi: Double) = { + this.lambdaHigh = hi + this + } + + def train(input: RDD[(Double, Array[Double])]): RidgeRegressionModel = { + val nfeatures: Int = input.take(1)(0)._2.length + val nexamples: Long = input.count() + + val (yMean, xColMean, xColSd) = MLUtils.computeStats(input, nfeatures, nexamples) + + val data = input.map { case(y, features) => + val yNormalized = y - yMean + val featuresMat = new DoubleMatrix(nfeatures, 1, features:_*) + val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd) + (yNormalized, featuresNormalized.toArray) + } + + // Compute XtX - Size of XtX is nfeatures by nfeatures + val XtX: DoubleMatrix = data.map { case (y, features) => + val x = new DoubleMatrix(1, features.length, features:_*) + x.transpose().mmul(x) + }.reduce(_.addi(_)) + + // Compute Xt*y - Size of Xty is nfeatures by 1 + val Xty: DoubleMatrix = data.map { case (y, features) => + new DoubleMatrix(features.length, 1, features:_*).mul(y) + }.reduce(_.addi(_)) + + // Define a function to compute the leave one out cross validation error + // for a single example + def crossValidate(lambda: Double): (Double, Double, DoubleMatrix) = { + // Compute the MLE ridge regression parameter value + + // Ridge Regression parameter = inv(XtX + \lambda*I) * Xty + val XtXlambda = DoubleMatrix.eye(nfeatures).muli(lambda).addi(XtX) + val w = Solve.solveSymmetric(XtXlambda, Xty) + + val invXtX = Solve.solveSymmetric(XtXlambda, DoubleMatrix.eye(nfeatures)) + + // compute the generalized cross validation score + val cverror = data.map { + case (y, features) => + val x = new DoubleMatrix(features.length, 1, features:_*) + val yhat = w.transpose().mmul(x).get(0) + val H_ii = x.transpose().mmul(invXtX).mmul(x).get(0) + val residual = (y - yhat) / (1.0 - H_ii) + residual * residual + }.reduce(_ + _) / nexamples + + (lambda, cverror, w) + } + + // Binary search for the best assignment to lambda. + def binSearch(low: Double, high: Double): List[(Double, Double, DoubleMatrix)] = { + val mid = (high - low) / 2 + low + val lowValue = crossValidate((mid - low) / 2 + low) + val highValue = crossValidate((high - mid) / 2 + mid) + val (newLow, newHigh) = if (lowValue._2 < highValue._2) { + (low, mid + (high-low)/4) + } else { + (mid - (high-low)/4, high) + } + if (newHigh - newLow > 1.0E-7) { + // :: is list prepend in Scala. + lowValue :: highValue :: binSearch(newLow, newHigh) + } else { + List(lowValue, highValue) + } + } + + // Actually compute the best lambda + val lambdas = binSearch(lambdaLow, lambdaHigh).sortBy(_._1) + + // Find the best parameter set by taking the lowest cverror. + val (lambdaOpt, cverror, weights) = lambdas.reduce((a, b) => if (a._2 < b._2) a else b) + + // Return the model which contains the solution + val weightsScaled = weights.div(xColSd) + val intercept = yMean - (weights.transpose().mmul(xColMean.div(xColSd)).get(0)) + val model = new RidgeRegressionModel(weightsScaled, intercept, lambdaOpt, lambdas) + + logInfo("RidgeRegression: optimal lambda " + model.lambdaOpt) + logInfo("RidgeRegression: optimal weights " + model.weights) + logInfo("RidgeRegression: optimal intercept " + model.intercept) + logInfo("RidgeRegression: cross-validation error " + cverror) + + model + } +} +/** + * Top-level methods for calling Ridge Regression. + */ +object RidgeRegression { + + /** + * Train a ridge regression model given an RDD of (response, features) pairs. + * We use the closed form solution to compute the cross-validation score for + * a given lambda. The optimal lambda is computed by performing binary search + * between the provided bounds of lambda. + * + * @param input RDD of (response, array of features) pairs. + * @param lambdaLow lower bound used in binary search for lambda + * @param lambdaHigh upper bound used in binary search for lambda + */ + def train( + input: RDD[(Double, Array[Double])], + lambdaLow: Double, + lambdaHigh: Double) + : RidgeRegressionModel = + { + new RidgeRegression(lambdaLow, lambdaHigh).train(input) + } + + /** + * Train a ridge regression model given an RDD of (response, features) pairs. + * We use the closed form solution to compute the cross-validation score for + * a given lambda. The optimal lambda is computed by performing binary search + * between lambda values of 0 and 100. + * + * @param input RDD of (response, array of features) pairs. + */ + def train(input: RDD[(Double, Array[Double])]) : RidgeRegressionModel = { + train(input, 0.0, 100.0) + } + + def main(args: Array[String]) { + if (args.length != 2) { + println("Usage: RidgeRegression ") + System.exit(1) + } + val sc = new SparkContext(args(0), "RidgeRegression") + val data = MLUtils.loadData(sc, args(1)) + val model = RidgeRegression.train(data, 0, 1000) + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala new file mode 100644 index 0000000000..c9ac4a8b07 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala @@ -0,0 +1,55 @@ +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.saveData(data, outputPath) + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala new file mode 100644 index 0000000000..0a4a037c71 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala @@ -0,0 +1,95 @@ +package spark.mllib.util + +import spark.{RDD, SparkContext} +import spark.SparkContext._ + +import org.jblas.DoubleMatrix + +/** + * Helper methods to load and save data + * Data format: + * , ... + * where , are feature values in Double and is the corresponding label as Double. + */ +object MLUtils { + + /** + * @param sc SparkContext + * @param dir Directory to the input data files. + * @return An RDD of tuples. For each tuple, the first element is the label, and the second + * element represents the feature values (an array of Double). + */ + def loadData(sc: SparkContext, dir: String): RDD[(Double, Array[Double])] = { + sc.textFile(dir).map { line => + val parts = line.split(",") + val label = parts(0).toDouble + val features = parts(1).trim().split(" ").map(_.toDouble) + (label, features) + } + } + + def saveData(data: RDD[(Double, Array[Double])], dir: String) { + val dataStr = data.map(x => x._1 + "," + x._2.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[(Double, Array[Double])], nfeatures: Int, nexamples: Long): + (Double, DoubleMatrix, DoubleMatrix) = { + val yMean: Double = data.map { case (y, features) => y }.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 { case(y, features) => + val nCols = features.length + // Traverse over every column and emit (col, value, value^2) + Iterator.tabulate(nCols) { i => + (i, (features(i), features(i)*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/test/scala/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/spark/ml/clustering/KMeansSuite.scala deleted file mode 100644 index ae7cf57c42..0000000000 --- a/mllib/src/test/scala/spark/ml/clustering/KMeansSuite.scala +++ /dev/null @@ -1,150 +0,0 @@ -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 { - val sc = new SparkContext("local", "test") - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - val EPSILON = 1e-4 - - 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||") - 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||") - 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/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/spark/ml/recommendation/ALSSuite.scala deleted file mode 100644 index 2ada9ae76b..0000000000 --- a/mllib/src/test/scala/spark/ml/recommendation/ALSSuite.scala +++ /dev/null @@ -1,80 +0,0 @@ -package spark.mllib.recommendation - -import scala.util.Random - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import spark.SparkContext -import spark.SparkContext._ - -import org.jblas._ - - -class ALSSuite extends FunSuite with BeforeAndAfterAll { - val 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 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 (u, p, trueRatings.get(u, p)) - } - - 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/ml/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala deleted file mode 100644 index 04d3400cb4..0000000000 --- a/mllib/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala +++ /dev/null @@ -1,57 +0,0 @@ -package spark.mllib.regression - -import scala.util.Random - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import spark.SparkContext -import spark.SparkContext._ - - -class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { - val sc = new SparkContext("local", "test") - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - // Test if we can correctly learn A, B where Y = logistic(A + B*X) - test("logistic regression") { - val nPoints = 10000 - val rnd = new Random(42) - - val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - - val A = 2.0 - val B = -1.5 - - // 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(100) - // y <- as.numeric(y > 0) - val y = (0 until nPoints).map { i => - val yVal = A + B * x1(i) + rLogis(i) - if (yVal > 0) 1.0 else 0.0 - } - - val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i)))).toArray - - val testRDD = sc.parallelize(testData, 2) - testRDD.cache() - val lr = new LogisticRegression().setStepSize(10.0) - .setNumIterations(20) - - val model = lr.train(testRDD) - - val weight0 = model.weights.get(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]") - } -} diff --git a/mllib/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala deleted file mode 100644 index df41dbbdff..0000000000 --- a/mllib/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala +++ /dev/null @@ -1,47 +0,0 @@ -package spark.mllib.regression - -import scala.util.Random - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import spark.SparkContext -import spark.SparkContext._ - - -class RidgeRegressionSuite extends FunSuite with BeforeAndAfterAll { - val sc = new SparkContext("local", "test") - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - // Test if we can correctly learn Y = 3 + X1 + X2 when - // X1 and X2 are collinear. - test("multi-collinear variables") { - val rnd = new Random(43) - val x1 = Array.fill[Double](20)(rnd.nextGaussian()) - - // Pick a mean close to mean of x1 - val rnd1 = new Random(42) //new NormalDistribution(0.1, 0.01) - val x2 = Array.fill[Double](20)(0.1 + rnd1.nextGaussian() * 0.01) - - val xMat = (0 until 20).map(i => Array(x1(i), x2(i))).toArray - - val y = xMat.map(i => 3 + i(0) + i(1)) - val testData = (0 until 20).map(i => (y(i), xMat(i))).toArray - - val testRDD = sc.parallelize(testData, 2) - testRDD.cache() - val ridgeReg = new RidgeRegression().setLowLambda(0) - .setHighLambda(10) - - val model = ridgeReg.train(testRDD) - - assert(model.intercept >= 2.9 && model.intercept <= 3.1) - assert(model.weights.length === 2) - assert(model.weights.get(0) >= 0.9 && model.weights.get(0) <= 1.1) - assert(model.weights.get(1) >= 0.9 && model.weights.get(1) <= 1.1) - } -} diff --git a/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala new file mode 100644 index 0000000000..ae7cf57c42 --- /dev/null +++ b/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala @@ -0,0 +1,150 @@ +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 { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + val EPSILON = 1e-4 + + 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||") + 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||") + 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 new file mode 100644 index 0000000000..2ada9ae76b --- /dev/null +++ b/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala @@ -0,0 +1,80 @@ +package spark.mllib.recommendation + +import scala.util.Random + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import spark.SparkContext +import spark.SparkContext._ + +import org.jblas._ + + +class ALSSuite extends FunSuite with BeforeAndAfterAll { + val 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 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 (u, p, trueRatings.get(u, p)) + } + + 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/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala new file mode 100644 index 0000000000..04d3400cb4 --- /dev/null +++ b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala @@ -0,0 +1,57 @@ +package spark.mllib.regression + +import scala.util.Random + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import spark.SparkContext +import spark.SparkContext._ + + +class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + // Test if we can correctly learn A, B where Y = logistic(A + B*X) + test("logistic regression") { + val nPoints = 10000 + val rnd = new Random(42) + + val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + + val A = 2.0 + val B = -1.5 + + // 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(100) + // y <- as.numeric(y > 0) + val y = (0 until nPoints).map { i => + val yVal = A + B * x1(i) + rLogis(i) + if (yVal > 0) 1.0 else 0.0 + } + + val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i)))).toArray + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + val lr = new LogisticRegression().setStepSize(10.0) + .setNumIterations(20) + + val model = lr.train(testRDD) + + val weight0 = model.weights.get(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]") + } +} diff --git a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala new file mode 100644 index 0000000000..df41dbbdff --- /dev/null +++ b/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala @@ -0,0 +1,47 @@ +package spark.mllib.regression + +import scala.util.Random + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import spark.SparkContext +import spark.SparkContext._ + + +class RidgeRegressionSuite extends FunSuite with BeforeAndAfterAll { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + // Test if we can correctly learn Y = 3 + X1 + X2 when + // X1 and X2 are collinear. + test("multi-collinear variables") { + val rnd = new Random(43) + val x1 = Array.fill[Double](20)(rnd.nextGaussian()) + + // Pick a mean close to mean of x1 + val rnd1 = new Random(42) //new NormalDistribution(0.1, 0.01) + val x2 = Array.fill[Double](20)(0.1 + rnd1.nextGaussian() * 0.01) + + val xMat = (0 until 20).map(i => Array(x1(i), x2(i))).toArray + + val y = xMat.map(i => 3 + i(0) + i(1)) + val testData = (0 until 20).map(i => (y(i), xMat(i))).toArray + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + val ridgeReg = new RidgeRegression().setLowLambda(0) + .setHighLambda(10) + + val model = ridgeReg.train(testRDD) + + assert(model.intercept >= 2.9 && model.intercept <= 3.1) + assert(model.weights.length === 2) + assert(model.weights.get(0) >= 0.9 && model.weights.get(0) <= 1.1) + assert(model.weights.get(1) >= 0.9 && model.weights.get(1) <= 1.1) + } +} -- cgit v1.2.3 From 84b7fc54e6777167f96742cd61326581f342fb03 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 5 Jul 2013 17:21:01 -0700 Subject: Enforcing correct sort order for formatted strings --- core/src/main/scala/spark/deploy/master/ui/IndexPage.scala | 10 +++++++--- core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala | 4 +++- core/src/main/scala/spark/ui/jobs/StagePage.scala | 4 +++- core/src/main/scala/spark/ui/storage/RDDPage.scala | 8 ++++++-- 4 files changed, 19 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index 2ed566a5bc..c6de2bafa3 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -90,8 +90,10 @@ private[spark] class IndexPage(parent: MasterWebUI) { {worker.host}:{worker.port} {worker.state} {worker.cores} ({worker.coresUsed} Used) - {Utils.memoryMegabytesToString(worker.memory)} - ({Utils.memoryMegabytesToString(worker.memoryUsed)} Used) + + {Utils.memoryMegabytesToString(worker.memory)} + ({Utils.memoryMegabytesToString(worker.memoryUsed)} Used) + } @@ -105,7 +107,9 @@ private[spark] class IndexPage(parent: MasterWebUI) { {app.coresGranted} - {Utils.memoryMegabytesToString(app.desc.memoryPerSlave)} + + {Utils.memoryMegabytesToString(app.desc.memoryPerSlave)} + {DeployWebUI.formatDate(app.submitDate)} {app.desc.user} {app.state.toString} diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index c65d5b4faf..e466129c1a 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -77,7 +77,9 @@ private[spark] class IndexPage(parent: WorkerWebUI) { {executor.execId} {executor.cores} - {Utils.memoryMegabytesToString(executor.memory)} + + {Utils.memoryMegabytesToString(executor.memory)} +
    • ID: {executor.appId}
    • diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index c6f87fc652..49e84880cf 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -93,7 +93,9 @@ private[spark] class StagePage(parent: JobProgressUI) { val (info, metrics, exception) = taskData {info.taskId} - {Option(metrics).map{m => parent.formatDuration(m.executorRunTime)}.getOrElse("")} + m.executorRunTime.toString}.getOrElse("1")}> + {Option(metrics).map{m => parent.formatDuration(m.executorRunTime)}.getOrElse("")} + {info.taskLocality} {info.hostPort} {dateFmt.format(new Date(info.launchTime))} diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 65952f711a..0cb1e47ea5 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -81,8 +81,12 @@ private[spark] class RDDPage(parent: BlockManagerUI) { {block.storageLevel.description} - {Utils.memoryBytesToString(block.memSize)} - {Utils.memoryBytesToString(block.diskSize)} + + {Utils.memoryBytesToString(block.memSize)} + + + {Utils.memoryBytesToString(block.diskSize)} + } -- cgit v1.2.3 From 8bbe907556041443a411b69c95d7a9cd3eb69dcc Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 17:25:23 -0700 Subject: Replaced string constants in test --- mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala index ae7cf57c42..cb096f39a9 100644 --- a/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala @@ -21,6 +21,8 @@ class KMeansSuite extends FunSuite with BeforeAndAfterAll { 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 = { @@ -82,10 +84,11 @@ class KMeansSuite extends FunSuite with BeforeAndAfterAll { 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") + 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||") + 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))) } @@ -115,10 +118,10 @@ class KMeansSuite extends FunSuite with BeforeAndAfterAll { 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") + 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||") + 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))) } -- cgit v1.2.3 From 757e56dfc7bd900d5b3f3f145eabe8198bfbe7cc Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Fri, 5 Jul 2013 19:54:28 -0700 Subject: make binSearch a tail-recursive method --- .../spark/mllib/regression/RidgeRegression.scala | 37 ++++++++++++++-------- 1 file changed, 23 insertions(+), 14 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index a6ececbeb6..8343f28139 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -1,12 +1,13 @@ package spark.mllib.regression import spark.{Logging, RDD, SparkContext} -import spark.SparkContext._ import spark.mllib.util.MLUtils import org.jblas.DoubleMatrix import org.jblas.Solve +import scala.annotation.tailrec + /** * Ridge Regression from Joseph Gonzalez's implementation in MLBase */ @@ -99,20 +100,28 @@ class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) // Binary search for the best assignment to lambda. def binSearch(low: Double, high: Double): List[(Double, Double, DoubleMatrix)] = { - val mid = (high - low) / 2 + low - val lowValue = crossValidate((mid - low) / 2 + low) - val highValue = crossValidate((high - mid) / 2 + mid) - val (newLow, newHigh) = if (lowValue._2 < highValue._2) { - (low, mid + (high-low)/4) - } else { - (mid - (high-low)/4, high) - } - if (newHigh - newLow > 1.0E-7) { - // :: is list prepend in Scala. - lowValue :: highValue :: binSearch(newLow, newHigh) - } else { - List(lowValue, highValue) + @tailrec + def loop( + low: Double, + high: Double, + acc: List[(Double, Double, DoubleMatrix)]): List[(Double, Double, DoubleMatrix)] = { + val mid = (high - low) / 2 + low + val lowValue = crossValidate((mid - low) / 2 + low) + val highValue = crossValidate((high - mid) / 2 + mid) + val (newLow, newHigh) = if (lowValue._2 < highValue._2) { + (low, mid + (high-low)/4) + } else { + (mid - (high-low)/4, high) + } + if (newHigh - newLow > 1.0E-7) { + // :: is list prepend in Scala. + loop(newLow, newHigh, lowValue :: highValue :: acc) + } else { + lowValue :: highValue :: acc + } } + + loop(low, high, Nil) } // Actually compute the best lambda -- cgit v1.2.3 From 280418ac452b029b15a83d8e2fe05a96417294d1 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 5 Jul 2013 21:38:21 -0700 Subject: Reduced the number of Iterator to ArrayBuffer copies in NetworkReceiver. --- .../spark/streaming/dstream/NetworkInputDStream.scala | 18 ++++++------------ .../spark/streaming/receivers/ActorReceiver.scala | 7 +++++-- 2 files changed, 11 insertions(+), 14 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala index 26805e9621..122a529bb7 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala @@ -140,12 +140,10 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log /** - * Pushes a block (as iterator of values) into the block manager. + * Pushes a block (as an ArrayBuffer filled with data) into the block manager. */ - def pushBlock(blockId: String, iterator: Iterator[T], metadata: Any, level: StorageLevel) { - val buffer = new ArrayBuffer[T] ++ iterator - env.blockManager.put(blockId, buffer.asInstanceOf[ArrayBuffer[Any]], level) - + def pushBlock(blockId: String, arrayBuffer: ArrayBuffer[T], metadata: Any, level: StorageLevel) { + env.blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], level) actor ! ReportBlock(blockId, metadata) } @@ -195,7 +193,7 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log class BlockGenerator(storageLevel: StorageLevel) extends Serializable with Logging { - case class Block(id: String, iterator: Iterator[T], metadata: Any = null) + case class Block(id: String, buffer: ArrayBuffer[T], metadata: Any = null) val clock = new SystemClock() val blockInterval = System.getProperty("spark.streaming.blockInterval", "200").toLong @@ -222,17 +220,13 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log currentBuffer += obj } - private def createBlock(blockId: String, iterator: Iterator[T]) : Block = { - new Block(blockId, iterator) - } - private def updateCurrentBuffer(time: Long) { try { val newBlockBuffer = currentBuffer currentBuffer = new ArrayBuffer[T] if (newBlockBuffer.size > 0) { val blockId = "input-" + NetworkReceiver.this.streamId + "-" + (time - blockInterval) - val newBlock = createBlock(blockId, newBlockBuffer.toIterator) + val newBlock = new Block(blockId, newBlockBuffer) blocksForPushing.add(newBlock) } } catch { @@ -248,7 +242,7 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log try { while(true) { val block = blocksForPushing.take() - NetworkReceiver.this.pushBlock(block.id, block.iterator, block.metadata, storageLevel) + NetworkReceiver.this.pushBlock(block.id, block.buffer, block.metadata, storageLevel) } } catch { case ie: InterruptedException => diff --git a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala index b3201d0b28..036c95a860 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala @@ -9,6 +9,8 @@ import spark.streaming.dstream.NetworkReceiver import java.util.concurrent.atomic.AtomicInteger +import scala.collection.mutable.ArrayBuffer + /** A helper with set of defaults for supervisor strategy **/ object ReceiverSupervisorStrategy { @@ -136,8 +138,9 @@ private[streaming] class ActorReceiver[T: ClassManifest]( } protected def pushBlock(iter: Iterator[T]) { - pushBlock("block-" + streamId + "-" + System.nanoTime(), - iter, null, storageLevel) + val buffer = new ArrayBuffer[T] + buffer ++= iter + pushBlock("block-" + streamId + "-" + System.nanoTime(), buffer, null, storageLevel) } protected def onStart() = { -- cgit v1.2.3 From 37abe84212c6dad6fb87a3b47666d6a3c14c1f66 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 5 Jul 2013 22:54:06 -0700 Subject: Tracking some task metrics even during failures. --- core/src/main/scala/spark/TaskEndReason.scala | 4 +++- core/src/main/scala/spark/executor/Executor.scala | 10 ++++++++-- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 2 +- .../spark/scheduler/cluster/ClusterTaskSetManager.scala | 2 +- .../main/scala/spark/scheduler/local/LocalScheduler.scala | 14 ++++++++++---- .../scala/spark/scheduler/local/LocalTaskSetManager.scala | 2 +- core/src/main/scala/spark/ui/UIWorkloadGenerator.scala | 12 +++++++++++- core/src/main/scala/spark/ui/jobs/JobProgressUI.scala | 8 ++++---- 8 files changed, 39 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/TaskEndReason.scala b/core/src/main/scala/spark/TaskEndReason.scala index 8140cba084..bb75ec208c 100644 --- a/core/src/main/scala/spark/TaskEndReason.scala +++ b/core/src/main/scala/spark/TaskEndReason.scala @@ -1,5 +1,6 @@ package spark +import spark.executor.TaskMetrics import spark.storage.BlockManagerId /** @@ -24,7 +25,8 @@ private[spark] case class FetchFailed( private[spark] case class ExceptionFailure( className: String, description: String, - stackTrace: Array[StackTraceElement]) + stackTrace: Array[StackTraceElement], + metrics: Option[TaskMetrics]) extends TaskEndReason private[spark] case class OtherFailure(message: String) extends TaskEndReason diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 8bebfafce4..f01be68d14 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -92,15 +92,18 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert val ser = SparkEnv.get.closureSerializer.newInstance() logInfo("Running task ID " + taskId) context.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER) + var attemptedTask: Option[Task[Any]] = None + var taskStart: Long = 0 try { SparkEnv.set(env) Accumulators.clear() val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask) updateDependencies(taskFiles, taskJars) val task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader) + attemptedTask = Some(task) logInfo("Its generation is " + task.generation) env.mapOutputTracker.updateGeneration(task.generation) - val taskStart = System.currentTimeMillis() + taskStart = System.currentTimeMillis() val value = task.run(taskId.toInt) val taskFinish = System.currentTimeMillis() task.metrics.foreach{ m => @@ -128,7 +131,10 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert } case t: Throwable => { - val reason = ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace) + val serviceTime = (System.currentTimeMillis() - taskStart).toInt + val metrics = attemptedTask.flatMap(t => t.metrics) + metrics.foreach{m => m.executorRunTime = serviceTime} + val reason = ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace, metrics) context.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) // TODO: Should we exit the whole executor here? On the one hand, the failed task may diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index d9ddc41aa2..1945a4a514 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -618,7 +618,7 @@ class DAGScheduler( handleExecutorLost(bmAddress.executorId, Some(task.generation)) } - case ExceptionFailure(className, description, stackTrace) => + case ExceptionFailure(className, description, stackTrace, metrics) => // Do nothing here, left up to the TaskScheduler to decide how to handle user failures case other => diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 6965cde5da..fe6420a522 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -571,7 +571,7 @@ private[spark] class ClusterTaskSetManager( return case ef: ExceptionFailure => - sched.listener.taskEnded(tasks(index), ef, null, null, info, null) + sched.listener.taskEnded(tasks(index), ef, null, null, info, ef.metrics.getOrElse(null)) val key = ef.description val now = System.currentTimeMillis val (printFull, dupCount) = { diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 9d375e1db8..b000e328e6 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -145,6 +145,9 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: // Set the Spark execution environment for the worker thread SparkEnv.set(env) val ser = SparkEnv.get.closureSerializer.newInstance() + var attemptedTask: Option[Task[_]] = None + val start = System.currentTimeMillis() + var taskStart: Long = 0 try { Accumulators.clear() Thread.currentThread().setContextClassLoader(classLoader) @@ -153,10 +156,11 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: // this adds a bit of unnecessary overhead but matches how the Mesos Executor works. val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(bytes) updateDependencies(taskFiles, taskJars) // Download any files added with addFile - val taskStart = System.currentTimeMillis() val deserializedTask = ser.deserialize[Task[_]]( taskBytes, Thread.currentThread.getContextClassLoader) - val deserTime = System.currentTimeMillis() - taskStart + attemptedTask = Some(deserializedTask) + val deserTime = System.currentTimeMillis() - start + taskStart = System.currentTimeMillis() // Run it val result: Any = deserializedTask.run(taskId) @@ -174,13 +178,15 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: logInfo("Finished " + taskId) deserializedTask.metrics.get.executorRunTime = serviceTime.toInt deserializedTask.metrics.get.executorDeserializeTime = deserTime.toInt - val taskResult = new TaskResult(result, accumUpdates, deserializedTask.metrics.getOrElse(null)) val serializedResult = ser.serialize(taskResult) localActor ! LocalStatusUpdate(taskId, TaskState.FINISHED, serializedResult) } catch { case t: Throwable => { - val failure = new ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace) + val serviceTime = System.currentTimeMillis() - taskStart + val metrics = attemptedTask.flatMap(t => t.metrics) + metrics.foreach{m => m.executorRunTime = serviceTime.toInt} + val failure = new ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace, metrics) localActor ! LocalStatusUpdate(taskId, TaskState.FAILED, ser.serialize(failure)) } } diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index 499116f653..f12fec41d5 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -152,7 +152,7 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas info.markFailed() decreaseRunningTasks(1) val reason: ExceptionFailure = ser.deserialize[ExceptionFailure](serializedData, getClass.getClassLoader) - sched.listener.taskEnded(task, reason, null, null, info, null) + sched.listener.taskEnded(task, reason, null, null, info, reason.metrics.getOrElse(null)) if (!finished(index)) { copiesRunning(index) -= 1 numFailures(index) += 1 diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala index 24cfe36aaa..8bbc6ce88e 100644 --- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala @@ -41,7 +41,17 @@ private[spark] object UIWorkloadGenerator { 1 }.count }), - ("Job with delays", baseData.map(x => Thread.sleep(1000)).count) + ("Partially failed phase (longer tasks)", { + baseData.map{x => + val probFailure = (4.0 / NUM_PARTITIONS) + if (nextFloat() < probFailure) { + Thread.sleep(100) + throw new Exception("This is a task failure") + } + 1 + }.count + }), + ("Job with delays", baseData.map(x => Thread.sleep(100)).count) ) while (true) { diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index aafa414055..36b1cd00ed 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -78,18 +78,18 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId - val failureInfo: Option[ExceptionFailure] = + val (failureInfo, metrics): (Option[ExceptionFailure], TaskMetrics) = taskEnd.reason match { case e: ExceptionFailure => stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 - Some(e) + (Some(e), e.metrics.get) case _ => stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 - None + (None, taskEnd.taskMetrics) } val taskList = stageToTaskInfos.getOrElse( sid, ArrayBuffer[(TaskInfo, TaskMetrics, Option[ExceptionFailure])]()) - taskList += ((taskEnd.taskInfo, taskEnd.taskMetrics, failureInfo)) + taskList += ((taskEnd.taskInfo, metrics, failureInfo)) stageToTaskInfos(sid) = taskList } -- cgit v1.2.3 From 44a2440039a35784b7dfed2e36b96096c3424d33 Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Sun, 7 Jul 2013 01:33:09 +0800 Subject: Remove active job from idToActiveJob when job finished or aborted --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 51b10ed045..cbd375e5c1 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -524,6 +524,7 @@ class DAGScheduler( job.numFinished += 1 // If the whole job has finished, remove it if (job.numFinished == job.numPartitions) { + idToActiveJob -= stage.priority activeJobs -= job resultStageToJob -= stage markStageAsFinished(stage) @@ -671,6 +672,7 @@ class DAGScheduler( val error = new SparkException("Job failed: " + reason) job.listener.jobFailed(error) sparkListeners.foreach(_.onJobEnd(SparkListenerJobEnd(job, JobFailed(error)))) + idToActiveJob -= resultStage.priority activeJobs -= job resultStageToJob -= resultStage } -- cgit v1.2.3 From 32b9d21a97d1c93f174551000d06cc429f317827 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 6 Jul 2013 16:36:13 -0700 Subject: Fix occasional failure in UI listener. If a task fails before the metrics are initialized, it remains possible that the metrics field will be `None`. This patch accounts for that possbility by keeping metrics as an `Option` at all times. --- core/src/main/scala/spark/ui/jobs/JobProgressUI.scala | 14 +++++++------- core/src/main/scala/spark/ui/jobs/StagePage.scala | 16 ++++++++-------- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 36b1cd00ed..84730cc091 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -51,7 +51,7 @@ private[spark] class JobProgressListener extends SparkListener { val stageToTasksComplete = HashMap[Int, Int]() val stageToTasksFailed = HashMap[Int, Int]() val stageToTaskInfos = - HashMap[Int, ArrayBuffer[(TaskInfo, TaskMetrics, Option[ExceptionFailure])]]() + HashMap[Int, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() override def onJobStart(jobStart: SparkListenerJobStart) {} @@ -78,17 +78,17 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId - val (failureInfo, metrics): (Option[ExceptionFailure], TaskMetrics) = + val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 - (Some(e), e.metrics.get) + (Some(e), e.metrics) case _ => stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 - (None, taskEnd.taskMetrics) + (None, Some(taskEnd.taskMetrics)) } val taskList = stageToTaskInfos.getOrElse( - sid, ArrayBuffer[(TaskInfo, TaskMetrics, Option[ExceptionFailure])]()) + sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList += ((taskEnd.taskInfo, metrics, failureInfo)) stageToTaskInfos(sid) = taskList } @@ -111,7 +111,7 @@ private[spark] class JobProgressListener extends SparkListener { def hasShuffleRead(stageID: Int): Boolean = { // This is written in a slightly complicated way to avoid having to scan all tasks for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { - if (s._2 != null) return s._2.shuffleReadMetrics.isDefined + if (s._2 != null) return s._2.flatMap(m => m.shuffleReadMetrics).isDefined } return false // No tasks have finished for this stage } @@ -120,7 +120,7 @@ private[spark] class JobProgressListener extends SparkListener { def hasShuffleWrite(stageID: Int): Boolean = { // This is written in a slightly complicated way to avoid having to scan all tasks for (s <- stageToTaskInfos.get(stageID).getOrElse(Seq())) { - if (s._2 != null) return s._2.shuffleWriteMetrics.isDefined + if (s._2 != null) return s._2.flatMap(m => m.shuffleWriteMetrics).isDefined } return false // No tasks have finished for this stage } diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 49e84880cf..51b82b6a8c 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -52,7 +52,7 @@ private[spark] class StagePage(parent: JobProgressUI) { } else { val serviceTimes = validTasks.map{case (info, metrics, exception) => - metrics.executorRunTime.toDouble} + metrics.get.executorRunTime.toDouble} val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles().map( ms => parent.formatDuration(ms.toLong)) @@ -61,13 +61,13 @@ private[spark] class StagePage(parent: JobProgressUI) { val shuffleReadSizes = validTasks.map { case(info, metrics, exception) => - metrics.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble + metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble } val shuffleReadQuantiles = "Shuffle Read (Remote)" +: getQuantileCols(shuffleReadSizes) val shuffleWriteSizes = validTasks.map { case(info, metrics, exception) => - metrics.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble + metrics.get.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble } val shuffleWriteQuantiles = "Shuffle Write" +: getQuantileCols(shuffleWriteSizes) @@ -87,21 +87,21 @@ private[spark] class StagePage(parent: JobProgressUI) { } - def taskRow(taskData: (TaskInfo, TaskMetrics, Option[ExceptionFailure])): Seq[Node] = { + def taskRow(taskData: (TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])): Seq[Node] = { def fmtStackTrace(trace: Seq[StackTraceElement]): Seq[Node] = trace.map(e => {e.toString}) val (info, metrics, exception) = taskData {info.taskId} - m.executorRunTime.toString}.getOrElse("1")}> - {Option(metrics).map{m => parent.formatDuration(m.executorRunTime)}.getOrElse("")} + m.executorRunTime.toString}.getOrElse("1")}> + {metrics.map{m => parent.formatDuration(m.executorRunTime)}.getOrElse("")} {info.taskLocality} {info.hostPort} {dateFmt.format(new Date(info.launchTime))} - {Option(metrics).flatMap{m => m.shuffleReadMetrics}.map{s => + {metrics.flatMap{m => m.shuffleReadMetrics}.map{s => {Utils.memoryBytesToString(s.remoteBytesRead)}}.getOrElse("")} - {Option(metrics).flatMap{m => m.shuffleWriteMetrics}.map{s => + {metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => {Utils.memoryBytesToString(s.shuffleBytesWritten)}}.getOrElse("")} {exception.map(e => -- cgit v1.2.3 From fd6665122b4bd3aa2d56ec2f6c6028e1b8a18b7f Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 6 Jul 2013 16:45:15 -0700 Subject: Fix some other references to Cloudera Avro and updated Avro version --- pom.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index 3184fd376a..017c242464 100644 --- a/pom.xml +++ b/pom.xml @@ -512,12 +512,12 @@ org.apache.avro avro - 1.7.1.cloudera.2 + 1.7.4 org.apache.avro avro-ipc - 1.7.1.cloudera.2 + 1.7.4 org.jboss.netty @@ -579,12 +579,12 @@ org.apache.avro avro - 1.7.1 + 1.7.4 org.apache.avro avro-ipc - 1.7.1 + 1.7.4 -- cgit v1.2.3 From f78f8d0b416ef4d88883d8f32382661f4c2ac52d Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Sat, 6 Jul 2013 16:46:53 -0700 Subject: fix formatting and use Vector instead of List to maintain order --- .../scala/spark/mllib/regression/RidgeRegression.scala | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index 8343f28139..36cda721dd 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -15,7 +15,7 @@ class RidgeRegressionModel( val weights: DoubleMatrix, val intercept: Double, val lambdaOpt: Double, - val lambdas: List[(Double, Double, DoubleMatrix)]) + val lambdas: Seq[(Double, Double, DoubleMatrix)]) extends RegressionModel { override def predict(testData: RDD[Array[Double]]): RDD[Double] = { @@ -99,12 +99,10 @@ class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) } // Binary search for the best assignment to lambda. - def binSearch(low: Double, high: Double): List[(Double, Double, DoubleMatrix)] = { + def binSearch(low: Double, high: Double): Seq[(Double, Double, DoubleMatrix)] = { @tailrec - def loop( - low: Double, - high: Double, - acc: List[(Double, Double, DoubleMatrix)]): List[(Double, Double, DoubleMatrix)] = { + def loop(low: Double, high: Double, acc: Seq[(Double, Double, DoubleMatrix)]) + : Seq[(Double, Double, DoubleMatrix)] = { val mid = (high - low) / 2 + low val lowValue = crossValidate((mid - low) / 2 + low) val highValue = crossValidate((high - mid) / 2 + mid) @@ -114,14 +112,13 @@ class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) (mid - (high-low)/4, high) } if (newHigh - newLow > 1.0E-7) { - // :: is list prepend in Scala. - loop(newLow, newHigh, lowValue :: highValue :: acc) + loop(newLow, newHigh, acc :+ lowValue :+ highValue) } else { - lowValue :: highValue :: acc + acc :+ lowValue :+ highValue } } - loop(low, high, Nil) + loop(low, high, Vector.empty) } // Actually compute the best lambda @@ -143,6 +140,7 @@ class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) model } } + /** * Top-level methods for calling Ridge Regression. */ -- cgit v1.2.3 From a948f0672541bd68be020f07134659aee2f38403 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 02:30:45 -0700 Subject: Suppress log messages in sbt test with two changes: 1. Set akka log level to ERROR before shutting down the actorSystem. This avoids akka log messages (like Spray) from falling back to INFO on the Stdout logger 2. Initialize netty to use SLF4J in LocalSparkContext. This ensures that stack trace thrown during shutdown is handled by SLF4J instead of stdout --- core/src/main/scala/spark/SparkEnv.scala | 2 ++ core/src/main/scala/spark/deploy/LocalSparkCluster.scala | 4 ++++ core/src/test/scala/spark/FileServerSuite.scala | 1 - core/src/test/scala/spark/LocalSparkContext.scala | 11 ++++++++++- 4 files changed, 16 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index ec59b4f48f..16b00d15aa 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -5,6 +5,7 @@ import serializer.Serializer import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem} import akka.remote.RemoteActorRefProvider +import akka.event.{Logging => AkkaLogging} import spark.broadcast.BroadcastManager import spark.storage.BlockManager @@ -51,6 +52,7 @@ class SparkEnv ( broadcastManager.stop() blockManager.stop() blockManager.master.stop() + actorSystem.eventStream.setLogLevel(AkkaLogging.ErrorLevel) actorSystem.shutdown() // Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut // down, but let's call it anyway in case it gets fixed in a later release diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala index 55bb61b0cc..cb85419ae4 100644 --- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala @@ -1,6 +1,7 @@ package spark.deploy import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} +import akka.event.{Logging => AkkaLogging} import spark.deploy.worker.Worker import spark.deploy.master.Master @@ -43,8 +44,11 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I def stop() { logInfo("Shutting down local Spark cluster.") // Stop the workers before the master so they don't get upset that it disconnected + workerActorSystems.foreach(_.eventStream.setLogLevel(AkkaLogging.ErrorLevel)) workerActorSystems.foreach(_.shutdown()) workerActorSystems.foreach(_.awaitTermination()) + + masterActorSystems.foreach(_.eventStream.setLogLevel(AkkaLogging.ErrorLevel)) masterActorSystems.foreach(_.shutdown()) masterActorSystems.foreach(_.awaitTermination()) } diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/spark/FileServerSuite.scala index f1a35bced3..9c24ca430d 100644 --- a/core/src/test/scala/spark/FileServerSuite.scala +++ b/core/src/test/scala/spark/FileServerSuite.scala @@ -85,7 +85,6 @@ class FileServerSuite extends FunSuite with LocalSparkContext { in.close() _ * fileVal + _ * fileVal }.collect - println(result) assert(result.toSet === Set((1,200), (2,300), (3,500))) } diff --git a/core/src/test/scala/spark/LocalSparkContext.scala b/core/src/test/scala/spark/LocalSparkContext.scala index 76d5258b02..bd184222ed 100644 --- a/core/src/test/scala/spark/LocalSparkContext.scala +++ b/core/src/test/scala/spark/LocalSparkContext.scala @@ -2,12 +2,21 @@ package spark import org.scalatest.Suite import org.scalatest.BeforeAndAfterEach +import org.scalatest.BeforeAndAfterAll + +import org.jboss.netty.logging.InternalLoggerFactory +import org.jboss.netty.logging.Slf4JLoggerFactory /** Manages a local `sc` {@link SparkContext} variable, correctly stopping it after each test. */ -trait LocalSparkContext extends BeforeAndAfterEach { self: Suite => +trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self: Suite => @transient var sc: SparkContext = _ + override def beforeAll() { + InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory()); + super.beforeAll() + } + override def afterEach() { resetSparkContext() super.afterEach() -- cgit v1.2.3 From 7d6d9e6ab226579518f1c7fbe108c4e66acc6ed0 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 02:45:55 -0700 Subject: Set DriverSuite log level to WARN --- core/src/test/scala/spark/DriverSuite.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/test/scala/spark/DriverSuite.scala b/core/src/test/scala/spark/DriverSuite.scala index 5e84b3a66a..31c3dd75fb 100644 --- a/core/src/test/scala/spark/DriverSuite.scala +++ b/core/src/test/scala/spark/DriverSuite.scala @@ -2,6 +2,9 @@ package spark import java.io.File +import org.apache.log4j.Logger +import org.apache.log4j.Level + import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts import org.scalatest.prop.TableDrivenPropertyChecks._ @@ -27,6 +30,7 @@ class DriverSuite extends FunSuite with Timeouts { */ object DriverWithoutCleanup { def main(args: Array[String]) { + Logger.getRootLogger().setLevel(Level.WARN) val sc = new SparkContext(args(0), "DriverWithoutCleanup") sc.parallelize(1 to 100, 4).count() } -- cgit v1.2.3 From 3350ad0d7fc3ecece78f87d7aa6a727e48b21c8c Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 03:32:26 -0700 Subject: Catch RejectedExecution exception in Checkpoint handler. --- streaming/src/main/scala/spark/streaming/Checkpoint.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala index 66e67cbfa1..450e48d66e 100644 --- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala @@ -8,7 +8,7 @@ import org.apache.hadoop.conf.Configuration import java.io._ import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import java.util.concurrent.Executors - +import java.util.concurrent.RejectedExecutionException private[streaming] class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) @@ -91,7 +91,12 @@ class CheckpointWriter(checkpointDir: String) extends Logging { oos.writeObject(checkpoint) oos.close() bos.close() - executor.execute(new CheckpointWriteHandler(checkpoint.checkpointTime, bos.toByteArray)) + try { + executor.execute(new CheckpointWriteHandler(checkpoint.checkpointTime, bos.toByteArray)) + } catch { + case rej: RejectedExecutionException => + logError("Could not submit checkpoint task to the thread pool executor", rej) + } } def stop() { -- cgit v1.2.3 From d362d0f4117268bdef265041ff291700ddd49b43 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 03:33:32 -0700 Subject: Ignore stderr when calling cat on a non-existing file --- core/src/test/scala/spark/PipedRDDSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/spark/PipedRDDSuite.scala index 1c9ca50811..d263bb00e9 100644 --- a/core/src/test/scala/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/spark/PipedRDDSuite.scala @@ -67,7 +67,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { test("pipe with non-zero exit status") { val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) - val piped = nums.pipe("cat nonexistent_file") + val piped = nums.pipe(Seq("cat nonexistent_file", "2>", "/dev/null")) intercept[SparkException] { piped.collect() } -- cgit v1.2.3 From 4af0d63cb14db902cbd1dbdeeb68f1fcec4b2e97 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 10:42:43 -0700 Subject: Remove akka LogLevel fix as we no longer use spray --- core/src/main/scala/spark/SparkEnv.scala | 2 -- core/src/main/scala/spark/deploy/LocalSparkCluster.scala | 3 --- 2 files changed, 5 deletions(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 16b00d15aa..ec59b4f48f 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -5,7 +5,6 @@ import serializer.Serializer import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem} import akka.remote.RemoteActorRefProvider -import akka.event.{Logging => AkkaLogging} import spark.broadcast.BroadcastManager import spark.storage.BlockManager @@ -52,7 +51,6 @@ class SparkEnv ( broadcastManager.stop() blockManager.stop() blockManager.master.stop() - actorSystem.eventStream.setLogLevel(AkkaLogging.ErrorLevel) actorSystem.shutdown() // Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut // down, but let's call it anyway in case it gets fixed in a later release diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala index cb85419ae4..939f26b6f4 100644 --- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala @@ -1,7 +1,6 @@ package spark.deploy import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} -import akka.event.{Logging => AkkaLogging} import spark.deploy.worker.Worker import spark.deploy.master.Master @@ -44,11 +43,9 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I def stop() { logInfo("Shutting down local Spark cluster.") // Stop the workers before the master so they don't get upset that it disconnected - workerActorSystems.foreach(_.eventStream.setLogLevel(AkkaLogging.ErrorLevel)) workerActorSystems.foreach(_.shutdown()) workerActorSystems.foreach(_.awaitTermination()) - masterActorSystems.foreach(_.eventStream.setLogLevel(AkkaLogging.ErrorLevel)) masterActorSystems.foreach(_.shutdown()) masterActorSystems.foreach(_.awaitTermination()) } -- cgit v1.2.3 From be123aa6ef90480ad61663eed6e8ea479b047fad Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Sun, 7 Jul 2013 15:35:06 -0700 Subject: update to use ListBuffer, faster than Vector for append operations --- .../scala/spark/mllib/regression/RidgeRegression.scala | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index 36cda721dd..f66025bc0b 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -7,6 +7,7 @@ import org.jblas.DoubleMatrix import org.jblas.Solve import scala.annotation.tailrec +import scala.collection.mutable /** * Ridge Regression from Joseph Gonzalez's implementation in MLBase @@ -100,9 +101,10 @@ class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) // Binary search for the best assignment to lambda. def binSearch(low: Double, high: Double): Seq[(Double, Double, DoubleMatrix)] = { + val buffer = mutable.ListBuffer.empty[(Double, Double, DoubleMatrix)] + @tailrec - def loop(low: Double, high: Double, acc: Seq[(Double, Double, DoubleMatrix)]) - : Seq[(Double, Double, DoubleMatrix)] = { + def loop(low: Double, high: Double): Seq[(Double, Double, DoubleMatrix)] = { val mid = (high - low) / 2 + low val lowValue = crossValidate((mid - low) / 2 + low) val highValue = crossValidate((high - mid) / 2 + mid) @@ -112,13 +114,15 @@ class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) (mid - (high-low)/4, high) } if (newHigh - newLow > 1.0E-7) { - loop(newLow, newHigh, acc :+ lowValue :+ highValue) + buffer += lowValue += highValue + loop(newLow, newHigh) } else { - acc :+ lowValue :+ highValue + buffer += lowValue += highValue + buffer.result() } } - loop(low, high, Vector.empty) + loop(low, high) } // Actually compute the best lambda -- cgit v1.2.3 From 8c1d1c98e0cea8d2b20ac10b84aa76d8e22a1661 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Mon, 8 Jul 2013 12:25:46 -0700 Subject: Explicitly set class loader for MesosSchedulerDriver callbacks. --- .../scala/spark/scheduler/mesos/MesosSchedulerBackend.scala | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index ca7fab4cc5..e73b780fcb 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -43,8 +43,12 @@ private[spark] class MesosSchedulerBackend( // An ExecutorInfo for our tasks var execArgs: Array[Byte] = null + var classLoader: ClassLoader = null + override def start() { synchronized { + classLoader = Thread.currentThread.getContextClassLoader + new Thread("MesosSchedulerBackend driver") { setDaemon(true) override def run() { @@ -114,9 +118,16 @@ private[spark] class MesosSchedulerBackend( return execArgs } + private def setClassLoader() { + // Since native code starts the thread our callbacks run in, it may not correctly + // inherit and custom class loaders. Therefore, set the class loader manually. + Thread.currentThread.setContextClassLoader(classLoader) + } + override def offerRescinded(d: SchedulerDriver, o: OfferID) {} override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { + setClassLoader() logInfo("Registered as framework ID " + frameworkId.getValue) registeredLock.synchronized { isRegistered = true @@ -142,6 +153,7 @@ private[spark] class MesosSchedulerBackend( * tasks are balanced across the cluster. */ override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { + setClassLoader() synchronized { // Build a big list of the offerable workers, and remember their indices so that we can // figure out which Offer to reply to for each worker @@ -224,6 +236,7 @@ private[spark] class MesosSchedulerBackend( } override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { + setClassLoader() val tid = status.getTaskId.getValue.toLong val state = TaskState.fromMesos(status.getState) synchronized { -- cgit v1.2.3 From bf4c9a5e0fca2dfc960120a7f3c5fab0b87e3850 Mon Sep 17 00:00:00 2001 From: Ameet Talwalkar Date: Mon, 8 Jul 2013 14:37:42 -0700 Subject: renamed with labeled prefix --- mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala | 2 +- .../scala/spark/mllib/regression/LogisticRegressionGenerator.scala | 2 +- mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala | 2 +- .../main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala | 2 +- mllib/src/main/scala/spark/mllib/util/MLUtils.scala | 4 ++-- 5 files changed, 6 insertions(+), 6 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index 448ab9dce9..e4db7bb9b7 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -150,7 +150,7 @@ object LogisticRegression { System.exit(1) } val sc = new SparkContext(args(0), "LogisticRegression") - val data = MLUtils.loadData(sc, args(1)) + val data = MLUtils.loadLabeledData(sc, args(1)) val model = LogisticRegression.train(data, args(3).toInt, args(2).toDouble) sc.stop() diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala index 9f6abab70b..6e7c023bac 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala @@ -35,7 +35,7 @@ object LogisticRegressionGenerator { (y, x) } - MLUtils.saveData(data, outputPath) + MLUtils.saveLabeledData(data, outputPath) sc.stop() } } diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index f66025bc0b..5f813df402 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -187,7 +187,7 @@ object RidgeRegression { System.exit(1) } val sc = new SparkContext(args(0), "RidgeRegression") - val data = MLUtils.loadData(sc, args(1)) + val data = MLUtils.loadLabeledData(sc, args(1)) val model = RidgeRegression.train(data, 0, 1000) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala index c9ac4a8b07..b83f505d8e 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala @@ -49,7 +49,7 @@ object RidgeRegressionGenerator { } } - MLUtils.saveData(data, outputPath) + MLUtils.saveLabeledData(data, outputPath) sc.stop() } } diff --git a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala index 0a4a037c71..08a031dded 100644 --- a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala @@ -19,7 +19,7 @@ object MLUtils { * @return An RDD of tuples. For each tuple, the first element is the label, and the second * element represents the feature values (an array of Double). */ - def loadData(sc: SparkContext, dir: String): RDD[(Double, Array[Double])] = { + def loadLabeledData(sc: SparkContext, dir: String): RDD[(Double, Array[Double])] = { sc.textFile(dir).map { line => val parts = line.split(",") val label = parts(0).toDouble @@ -28,7 +28,7 @@ object MLUtils { } } - def saveData(data: RDD[(Double, Array[Double])], dir: String) { + def saveLabeledData(data: RDD[(Double, Array[Double])], dir: String) { val dataStr = data.map(x => x._1 + "," + x._2.mkString(" ")) dataStr.saveAsTextFile(dir) } -- cgit v1.2.3 From afdaf430bd8d7da3ee3323cf03b00d4214159626 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Mon, 8 Jul 2013 15:38:29 -0700 Subject: Explicit dependencies for scala-library and scalap to prevent 2.9.2 vs. 2.9.3 problems --- core/pom.xml | 10 ++++++++++ examples/pom.xml | 5 +++++ streaming/pom.xml | 5 +++++ 3 files changed, 20 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index 5edafb3706..39dba46d7a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -55,6 +55,16 @@ com.typesafe.akka akka-slf4j + + org.scala-lang + scalap + 2.9.3 + + + org.scala-lang + scala-library + 2.9.3 + net.liftweb lift-json_2.9.2 diff --git a/examples/pom.xml b/examples/pom.xml index 78ec58729b..7d9769e8e4 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -15,6 +15,11 @@ http://spark-project.org/ + + org.scala-lang + scala-library + 2.9.3 + org.eclipse.jetty jetty-server diff --git a/streaming/pom.xml b/streaming/pom.xml index 4dc9a19d51..07725f9484 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -58,6 +58,11 @@ twitter4j-stream 3.0.3 + + org.scala-lang + scala-library + 2.9.3 + com.typesafe.akka akka-zeromq -- cgit v1.2.3 From 0b39d66f3f815f2d406d83a41db0f46d097baaa7 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Mon, 8 Jul 2013 16:07:09 -0700 Subject: pom cleanup --- core/pom.xml | 2 -- examples/pom.xml | 1 - pom.xml | 10 ++++++++++ streaming/pom.xml | 1 - 4 files changed, 10 insertions(+), 4 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 39dba46d7a..dbb2da9a9c 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -58,12 +58,10 @@ org.scala-lang scalap - 2.9.3 org.scala-lang scala-library - 2.9.3 net.liftweb diff --git a/examples/pom.xml b/examples/pom.xml index 7d9769e8e4..1976765c3d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -18,7 +18,6 @@ org.scala-lang scala-library - 2.9.3 org.eclipse.jetty diff --git a/pom.xml b/pom.xml index 017c242464..63dd80b5fe 100644 --- a/pom.xml +++ b/pom.xml @@ -241,6 +241,16 @@ jline ${scala.version} + + org.scala-lang + scala-library + ${scala.version} + + + org.scala-lang + scalap + ${scala.version} + log4j diff --git a/streaming/pom.xml b/streaming/pom.xml index 07725f9484..2fb5bbdeb5 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -61,7 +61,6 @@ org.scala-lang scala-library - 2.9.3 com.typesafe.akka -- cgit v1.2.3 From c1d44be80580f0fad6bb1805bbcf74a34f536d8c Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Tue, 9 Jul 2013 15:18:28 +0800 Subject: Bug fix: SPARK-796 --- .../spark/deploy/yarn/ApplicationMaster.scala | 49 ++++++++++++++-------- 1 file changed, 32 insertions(+), 17 deletions(-) diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index f19648ec68..9bc692d480 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -27,6 +27,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) private var yarnAllocator: YarnAllocationHandler = null + private var isFinished:Boolean = false def run() { @@ -68,10 +69,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // Wait for the user class to Finish userThread.join() - - // Finish the ApplicationMaster - finishApplicationMaster() - // TODO: Exit based on success/failure + System.exit(0) } @@ -131,10 +129,17 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e .getMethod("main", classOf[Array[String]]) val t = new Thread { override def run() { - // Copy - var mainArgs: Array[String] = new Array[String](args.userArgs.size()) - args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size()) - mainMethod.invoke(null, mainArgs) + try{ + // Copy + var mainArgs: Array[String] = new Array[String](args.userArgs.size()) + args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size()) + mainMethod.invoke(null, mainArgs) + ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) + } catch { + case th: Throwable => + ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.FAILED) + logError("Finish ApplicationMaster with ",th) + } } } t.start() @@ -235,14 +240,22 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } } */ - - def finishApplicationMaster() { - val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) - .asInstanceOf[FinishApplicationMasterRequest] - finishReq.setAppAttemptId(appAttemptId) - // TODO: Check if the application has failed or succeeded - finishReq.setFinishApplicationStatus(FinalApplicationStatus.SUCCEEDED) - resourceManager.finishApplicationMaster(finishReq) + + def finishApplicationMaster(status: FinalApplicationStatus) { + + synchronized { + if(isFinished){ + return + } + isFinished = true + + logInfo("finishApplicationMaster with "+status) + val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) + .asInstanceOf[FinishApplicationMasterRequest] + finishReq.setAppAttemptId(appAttemptId) + finishReq.setFinishApplicationStatus(status) + resourceManager.finishApplicationMaster(finishReq) + } } } @@ -291,7 +304,9 @@ object ApplicationMaster { logInfo("Invoking sc stop from shutdown hook") sc.stop() // best case ... - for (master <- applicationMasters) master.finishApplicationMaster + for (master <- applicationMasters) { + master.finishApplicationMaster(FinalApplicationStatus.KILLED) + } } } ) } -- cgit v1.2.3 From e47253e0cca1359b49f113dbf258c2c204e3bfc1 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Tue, 9 Jul 2013 01:13:12 -0700 Subject: Reset ClassLoader in MesosSchedulerBackend, too. (per review comments). Also set ClassLoader for all mesos callbacks, not just statusUpdate, registered. --- .../scheduler/mesos/MesosSchedulerBackend.scala | 152 ++++++++++++--------- 1 file changed, 89 insertions(+), 63 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index e73b780fcb..e83368b98d 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -118,20 +118,28 @@ private[spark] class MesosSchedulerBackend( return execArgs } - private def setClassLoader() { - // Since native code starts the thread our callbacks run in, it may not correctly - // inherit and custom class loaders. Therefore, set the class loader manually. + private def setClassLoader(): ClassLoader = { + val oldClassLoader = Thread.currentThread.getContextClassLoader Thread.currentThread.setContextClassLoader(classLoader) + return oldClassLoader + } + + private def restoreClassLoader(oldClassLoader: ClassLoader) { + Thread.currentThread.setContextClassLoader(oldClassLoader) } override def offerRescinded(d: SchedulerDriver, o: OfferID) {} override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { - setClassLoader() - logInfo("Registered as framework ID " + frameworkId.getValue) - registeredLock.synchronized { - isRegistered = true - registeredLock.notifyAll() + val oldClassLoader = setClassLoader() + try { + logInfo("Registered as framework ID " + frameworkId.getValue) + registeredLock.synchronized { + isRegistered = true + registeredLock.notifyAll() + } + } finally { + restoreClassLoader(oldClassLoader) } } @@ -153,50 +161,54 @@ private[spark] class MesosSchedulerBackend( * tasks are balanced across the cluster. */ override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { - setClassLoader() - synchronized { - // Build a big list of the offerable workers, and remember their indices so that we can - // figure out which Offer to reply to for each worker - val offerableIndices = new ArrayBuffer[Int] - val offerableWorkers = new ArrayBuffer[WorkerOffer] - - def enoughMemory(o: Offer) = { - val mem = getResource(o.getResourcesList, "mem") - val slaveId = o.getSlaveId.getValue - mem >= executorMemory || slaveIdsWithExecutors.contains(slaveId) - } + val oldClassLoader = setClassLoader() + try { + synchronized { + // Build a big list of the offerable workers, and remember their indices so that we can + // figure out which Offer to reply to for each worker + val offerableIndices = new ArrayBuffer[Int] + val offerableWorkers = new ArrayBuffer[WorkerOffer] + + def enoughMemory(o: Offer) = { + val mem = getResource(o.getResourcesList, "mem") + val slaveId = o.getSlaveId.getValue + mem >= executorMemory || slaveIdsWithExecutors.contains(slaveId) + } - for ((offer, index) <- offers.zipWithIndex if enoughMemory(offer)) { - offerableIndices += index - offerableWorkers += new WorkerOffer( - offer.getSlaveId.getValue, - offer.getHostname, - getResource(offer.getResourcesList, "cpus").toInt) - } + for ((offer, index) <- offers.zipWithIndex if enoughMemory(offer)) { + offerableIndices += index + offerableWorkers += new WorkerOffer( + offer.getSlaveId.getValue, + offer.getHostname, + getResource(offer.getResourcesList, "cpus").toInt) + } - // Call into the ClusterScheduler - val taskLists = scheduler.resourceOffers(offerableWorkers) - - // Build a list of Mesos tasks for each slave - val mesosTasks = offers.map(o => Collections.emptyList[MesosTaskInfo]()) - for ((taskList, index) <- taskLists.zipWithIndex) { - if (!taskList.isEmpty) { - val offerNum = offerableIndices(index) - val slaveId = offers(offerNum).getSlaveId.getValue - slaveIdsWithExecutors += slaveId - mesosTasks(offerNum) = new JArrayList[MesosTaskInfo](taskList.size) - for (taskDesc <- taskList) { - taskIdToSlaveId(taskDesc.taskId) = slaveId - mesosTasks(offerNum).add(createMesosTask(taskDesc, slaveId)) + // Call into the ClusterScheduler + val taskLists = scheduler.resourceOffers(offerableWorkers) + + // Build a list of Mesos tasks for each slave + val mesosTasks = offers.map(o => Collections.emptyList[MesosTaskInfo]()) + for ((taskList, index) <- taskLists.zipWithIndex) { + if (!taskList.isEmpty) { + val offerNum = offerableIndices(index) + val slaveId = offers(offerNum).getSlaveId.getValue + slaveIdsWithExecutors += slaveId + mesosTasks(offerNum) = new JArrayList[MesosTaskInfo](taskList.size) + for (taskDesc <- taskList) { + taskIdToSlaveId(taskDesc.taskId) = slaveId + mesosTasks(offerNum).add(createMesosTask(taskDesc, slaveId)) + } } } - } - // Reply to the offers - val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? - for (i <- 0 until offers.size) { - d.launchTasks(offers(i).getId, mesosTasks(i), filters) + // Reply to the offers + val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? + for (i <- 0 until offers.size) { + d.launchTasks(offers(i).getId, mesosTasks(i), filters) + } } + } finally { + restoreClassLoader(oldClassLoader) } } @@ -236,24 +248,33 @@ private[spark] class MesosSchedulerBackend( } override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { - setClassLoader() - val tid = status.getTaskId.getValue.toLong - val state = TaskState.fromMesos(status.getState) - synchronized { - if (status.getState == MesosTaskState.TASK_LOST && taskIdToSlaveId.contains(tid)) { - // We lost the executor on this slave, so remember that it's gone - slaveIdsWithExecutors -= taskIdToSlaveId(tid) - } - if (isFinished(status.getState)) { - taskIdToSlaveId.remove(tid) + val oldClassLoader = setClassLoader() + try { + val tid = status.getTaskId.getValue.toLong + val state = TaskState.fromMesos(status.getState) + synchronized { + if (status.getState == MesosTaskState.TASK_LOST && taskIdToSlaveId.contains(tid)) { + // We lost the executor on this slave, so remember that it's gone + slaveIdsWithExecutors -= taskIdToSlaveId(tid) + } + if (isFinished(status.getState)) { + taskIdToSlaveId.remove(tid) + } } + scheduler.statusUpdate(tid, state, status.getData.asReadOnlyByteBuffer) + } finally { + restoreClassLoader(oldClassLoader) } - scheduler.statusUpdate(tid, state, status.getData.asReadOnlyByteBuffer) } override def error(d: SchedulerDriver, message: String) { - logError("Mesos error: " + message) - scheduler.error(message) + val oldClassLoader = setClassLoader() + try { + logError("Mesos error: " + message) + scheduler.error(message) + } finally { + restoreClassLoader(oldClassLoader) + } } override def stop() { @@ -269,11 +290,16 @@ private[spark] class MesosSchedulerBackend( override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} private def recordSlaveLost(d: SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason) { - logInfo("Mesos slave lost: " + slaveId.getValue) - synchronized { - slaveIdsWithExecutors -= slaveId.getValue + val oldClassLoader = setClassLoader() + try { + logInfo("Mesos slave lost: " + slaveId.getValue) + synchronized { + slaveIdsWithExecutors -= slaveId.getValue + } + scheduler.executorLost(slaveId.getValue, reason) + } finally { + restoreClassLoader(oldClassLoader) } - scheduler.executorLost(slaveId.getValue, reason) } override def slaveLost(d: SchedulerDriver, slaveId: SlaveID) { -- cgit v1.2.3 From aaa7b081df760a29ce5cdcd51d6b71422cba68d5 Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Tue, 9 Jul 2013 20:03:01 +0800 Subject: according to mridulm's comments to adjust the code --- .../spark/deploy/yarn/ApplicationMaster.scala | 31 +++++++++++++--------- 1 file changed, 19 insertions(+), 12 deletions(-) diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index 9bc692d480..776db201f9 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -122,23 +122,26 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e } } } - + private def startUserClass(): Thread = { logInfo("Starting the user JAR in a separate Thread") val mainMethod = Class.forName(args.userClass, false, Thread.currentThread.getContextClassLoader) .getMethod("main", classOf[Array[String]]) val t = new Thread { override def run() { + var successed = false try{ // Copy var mainArgs: Array[String] = new Array[String](args.userArgs.size()) args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size()) mainMethod.invoke(null, mainArgs) - ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) - } catch { - case th: Throwable => + successed = true + } finally { + if(successed){ + ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) + }else{ ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.FAILED) - logError("Finish ApplicationMaster with ",th) + } } } } @@ -248,14 +251,15 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e return } isFinished = true - - logInfo("finishApplicationMaster with "+status) - val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) - .asInstanceOf[FinishApplicationMasterRequest] - finishReq.setAppAttemptId(appAttemptId) - finishReq.setFinishApplicationStatus(status) - resourceManager.finishApplicationMaster(finishReq) } + + logInfo("finishApplicationMaster with " + status) + val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) + .asInstanceOf[FinishApplicationMasterRequest] + finishReq.setAppAttemptId(appAttemptId) + finishReq.setFinishApplicationStatus(status) + resourceManager.finishApplicationMaster(finishReq) + } } @@ -304,6 +308,9 @@ object ApplicationMaster { logInfo("Invoking sc stop from shutdown hook") sc.stop() // best case ... + // due to the sparkContext is stopped and ApplicationMaster is down, + // the status of registered masters should be set KILLED better than FAILED. + // need discussion for (master <- applicationMasters) { master.finishApplicationMaster(FinalApplicationStatus.KILLED) } -- cgit v1.2.3 From 13fc6f248c7231501d8c53ad13641fa996e06be4 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 9 Jul 2013 13:42:51 -0700 Subject: Clean commit of log paging --- .../spark/deploy/master/ui/ApplicationPage.scala | 8 ++- .../scala/spark/deploy/worker/ui/IndexPage.scala | 8 ++- .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 61 ++++++++++++++++++++++ 3 files changed, 75 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index 33a16b5d84..ea88421532 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -38,7 +38,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { state.completedApps.find(_.id == appId).getOrElse(null) }) - val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Logs") + val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Logs", "Log Pages") val executors = app.executors.values.toSeq val executorTable = UIUtils.listingTable(executorHeaders, executorRow, executors) @@ -95,6 +95,12 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { stderr + + stdout + stderr + } } diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index e466129c1a..7cf98b473e 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -29,7 +29,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] val workerState = Await.result(stateFuture, 30 seconds) - val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") + val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs", "Log Pages") val runningExecutorTable = UIUtils.listingTable(executorHeaders, executorRow, workerState.executors) val finishedExecutorTable = @@ -93,6 +93,12 @@ private[spark] class IndexPage(parent: WorkerWebUI) { stderr + + stdout-page + stderr-page + } diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 16564d5619..5b0c785b00 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -13,6 +13,10 @@ import spark.{Utils, Logging} import spark.ui.JettyUtils import spark.ui.JettyUtils._ +import scala.xml._ +import spark.ui.UIUtils +import scala.io.Source._ + /** * Web UI server for the standalone worker. */ @@ -33,6 +37,7 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val handlers = Array[(String, Handler)]( ("/static", createStaticHandler(WorkerWebUI.STATIC_RESOURCE_DIR)), ("/log", (request: HttpServletRequest) => log(request)), + ("/logPage", (request: HttpServletRequest) => logPage(request)), ("/json", (request: HttpServletRequest) => indexPage.renderJson(request)), ("*", (request: HttpServletRequest) => indexPage.render(request)) ) @@ -65,6 +70,62 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option pre + Utils.lastNBytes(path, math.min(numBytes, maxBytes)) } + def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { + val appId = request.getParameter("appId") + val executorId = request.getParameter("executorId") + val logType = request.getParameter("logType") + val getOffset = request.getParameter("offset") + val getLineLength = request.getParameter("lineLength") + val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) + val source = fromFile(path) + val lines = source.getLines().toArray + val logLength = lines.length + val offset = { + if (getOffset == null) 0 + else if (getOffset.toInt < 0) 0 + else getOffset.toInt + } + val lineLength = { + if (getLineLength == null) 0 + else getLineLength.toInt + } + val logText = "" + lines.slice(offset, offset+lineLength).mkString("\n") + "" + val logXML = XML.loadString(logText) + val backButton = + if (offset > 0) { + if (offset-lineLength < 0) { + + } + else { + + } + } + else { + + } + val nextButton = + if (offset+lineLength < logLength) { + + } + else { + + } + val content = + + +
      + {backButton} + {nextButton} +

      +
      {logXML}
      + {backButton} + {nextButton} + + + source.close() + UIUtils.basicSparkPage(content, "Log Page for " + appId) + } + def stop() { server.foreach(_.stop()) } -- cgit v1.2.3 From b6072b58bf795093df0e5f8424413fcd0cce6323 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 9 Jul 2013 17:25:10 -0700 Subject: Fixes style, makes "std__-page" consistent, reads only parts of files --- core/src/main/scala/spark/Utils.scala | 14 ++++++ .../spark/deploy/master/ui/ApplicationPage.scala | 8 ++-- .../scala/spark/deploy/worker/ui/IndexPage.scala | 4 +- .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 54 ++++++++++------------ 4 files changed, 45 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 64547bbdcd..dfc30469cf 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -635,6 +635,20 @@ private object Utils extends Logging { Source.fromBytes(buff).mkString } + /** Return an array containing part of a file from byte 'a' to 'b'. */ + def offsetBytes(path: String, a: Int, b: Int): String = { + val file = new File(path) + val length = file.length() + val buff = new Array[Byte](math.min(b-a, length.toInt)) + val skip = math.max(0, a) + val stream = new FileInputStream(file) + + stream.skip(skip) + stream.read(buff) + stream.close() + Source.fromBytes(buff).mkString + } + /** * Clone an object using a Spark serializer. */ diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index ea88421532..cc32728c1c 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -96,10 +96,10 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { .format(executor.worker.webUiAddress, executor.application.id, executor.id)}>stderr - stdout - stderr + stdout-page + stderr-page } diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index 7cf98b473e..d532aa9e95 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -94,9 +94,9 @@ private[spark] class IndexPage(parent: WorkerWebUI) { .format(executor.appId, executor.execId)}>stderr - stdout-page - stderr-page diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 5b0c785b00..b018f80c85 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -9,13 +9,13 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} +import scala.io.Source._ +import scala.xml._ + import spark.{Utils, Logging} import spark.ui.JettyUtils import spark.ui.JettyUtils._ - -import scala.xml._ import spark.ui.UIUtils -import scala.io.Source._ /** * Web UI server for the standalone worker. @@ -74,42 +74,39 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - val getOffset = request.getParameter("offset") - val getLineLength = request.getParameter("lineLength") + val offset = Option(request.getParameter("offset")).map(_.toInt).getOrElse(0) + + val maxBytes = 1024 * 1024 + val defaultBytes = 100 * 1024 + val byteLength = Option(request.getParameter("byteLength")).flatMap(s => Some(s.toInt)).getOrElse(defaultBytes) + val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val source = fromFile(path) - val lines = source.getLines().toArray - val logLength = lines.length - val offset = { - if (getOffset == null) 0 - else if (getOffset.toInt < 0) 0 - else getOffset.toInt - } - val lineLength = { - if (getLineLength == null) 0 - else getLineLength.toInt - } - val logText = "" + lines.slice(offset, offset+lineLength).mkString("\n") + "" - val logXML = XML.loadString(logText) + val logLength = new File(path).length() + val logPageLength = math.min(byteLength, maxBytes) + val logText = {Utils.offsetBytes(path, offset, offset+logPageLength)} + val backButton = if (offset > 0) { - if (offset-lineLength < 0) { - - } - else { - - } + + + } else { } + val nextButton = - if (offset+lineLength < logLength) { - + if (offset+logPageLength < logLength) { + + + } else { } + val content = @@ -117,12 +114,11 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option {backButton} {nextButton}

      -
      {logXML}
      +
      {logText}
      {backButton} {nextButton} - source.close() UIUtils.basicSparkPage(content, "Log Page for " + appId) } -- cgit v1.2.3 From ce18b50d5ff37dc5c558d0602321a61887dd8b48 Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Wed, 10 Jul 2013 19:11:43 +0800 Subject: set SUCCEEDED for all master in shutdown hook --- .../hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index 776db201f9..68bb36d316 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -135,6 +135,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e var mainArgs: Array[String] = new Array[String](args.userArgs.size()) args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size()) mainMethod.invoke(null, mainArgs) + // some job script has "System.exit(0)" at the end, for example SparkPi, SparkLR + // userThread will stop here unless it has uncaught exception thrown out + // It need shutdown hook to set SUCCEEDED successed = true } finally { if(successed){ @@ -308,11 +311,8 @@ object ApplicationMaster { logInfo("Invoking sc stop from shutdown hook") sc.stop() // best case ... - // due to the sparkContext is stopped and ApplicationMaster is down, - // the status of registered masters should be set KILLED better than FAILED. - // need discussion for (master <- applicationMasters) { - master.finishApplicationMaster(FinalApplicationStatus.KILLED) + master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) } } } ) -- cgit v1.2.3 From 620a6974c6603f1c0e5a7cea8f0387a5d18f2e5e Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 10:20:53 -0700 Subject: Allows for larger files, refactors lastNBytes, removes old Log column, fixes imports, uses map --- core/src/main/scala/spark/Utils.scala | 21 +++++++-------------- .../spark/deploy/master/ui/ApplicationPage.scala | 12 +++--------- .../scala/spark/deploy/worker/ui/IndexPage.scala | 12 +++--------- .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 7 ++----- 4 files changed, 15 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index dfc30469cf..849edc13ee 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -621,12 +621,12 @@ private object Utils extends Logging { callSiteInfo.firstUserLine) } - /** Return a string containing the last `n` bytes of a file. */ - def lastNBytes(path: String, n: Int): String = { + /** Return a string containing part of a file from byte 'a' to 'b'. */ + def offsetBytes(path: String, a: Long, b: Long): String = { val file = new File(path) val length = file.length() - val buff = new Array[Byte](math.min(n, length.toInt)) - val skip = math.max(0, length - n) + val buff = new Array[Byte](math.min((b-a).toInt, length.toInt)) + val skip = math.max(0, a) val stream = new FileInputStream(file) stream.skip(skip) @@ -635,18 +635,11 @@ private object Utils extends Logging { Source.fromBytes(buff).mkString } - /** Return an array containing part of a file from byte 'a' to 'b'. */ - def offsetBytes(path: String, a: Int, b: Int): String = { + /** Return a string containing the last `n` bytes of a file. */ + def lastNBytes(path: String, n: Int): String = { val file = new File(path) val length = file.length() - val buff = new Array[Byte](math.min(b-a, length.toInt)) - val skip = math.max(0, a) - val stream = new FileInputStream(file) - - stream.skip(skip) - stream.read(buff) - stream.close() - Source.fromBytes(buff).mkString + offsetBytes(path, length-n, length) } /** diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index cc32728c1c..49ced0d320 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -38,7 +38,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { state.completedApps.find(_.id == appId).getOrElse(null) }) - val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Logs", "Log Pages") + val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Log Pages") val executors = app.executors.values.toSeq val executorTable = UIUtils.listingTable(executorHeaders, executorRow, executors) @@ -89,17 +89,11 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { {executor.cores} {executor.memory} {executor.state} - - stdout - stderr - stdout-page + .format(executor.worker.webUiAddress, executor.application.id, executor.id)}>stdout stderr-page + .format(executor.worker.webUiAddress, executor.application.id, executor.id)}>stderr } diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index d532aa9e95..af9943853f 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -29,7 +29,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] val workerState = Await.result(stateFuture, 30 seconds) - val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs", "Log Pages") + val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Log Pages") val runningExecutorTable = UIUtils.listingTable(executorHeaders, executorRow, workerState.executors) val finishedExecutorTable = @@ -87,17 +87,11 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
    • User: {executor.appDesc.user}
    - - stdout - stderr - stdout-page + .format(executor.appId, executor.execId)}>stdout stderr-page + .format(executor.appId, executor.execId)}>stderr } diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index b018f80c85..602881d5f1 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -9,9 +9,6 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} -import scala.io.Source._ -import scala.xml._ - import spark.{Utils, Logging} import spark.ui.JettyUtils import spark.ui.JettyUtils._ @@ -74,11 +71,11 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toInt).getOrElse(0) + val offset = Option(request.getParameter("offset")).map(_.toLong).getOrElse(0).asInstanceOf[Long] val maxBytes = 1024 * 1024 val defaultBytes = 100 * 1024 - val byteLength = Option(request.getParameter("byteLength")).flatMap(s => Some(s.toInt)).getOrElse(defaultBytes) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) val logLength = new File(path).length() -- cgit v1.2.3 From 24705d0f46ce536bf829660f4506dcffd9ff799a Mon Sep 17 00:00:00 2001 From: seanm Date: Wed, 10 Jul 2013 10:33:11 -0700 Subject: adding takeOrdered() to RDD --- core/src/main/scala/spark/RDD.scala | 12 ++++++++++++ core/src/test/scala/spark/RDDSuite.scala | 18 ++++++++++++++++++ 2 files changed, 30 insertions(+) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 106fb2960f..af52040fa6 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -781,6 +781,18 @@ abstract class RDD[T: ClassManifest]( }.toArray } + /** + * Returns the top K elements from this RDD as defined by + * the specified implicit Ordering[T] and maintains the + * ordering. + * @param num the number of top elements to return + * @param ord the implicit ordering for T + * @return an array of top elements + */ + def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = { + top(num)(ord.reverse).sorted(ord) + } + /** * Save this RDD as a text file, using string representations of elements. */ diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index e41ae385c0..fe17d1d5e7 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -252,6 +252,24 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(topK.sorted === Array("b", "a")) } + test("takeOrdered with predefined ordering") { + val nums = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) + val rdd = sc.makeRDD(nums, 2) + val sortedTopK = rdd.takeOrdered(5) + assert(sortedTopK.size === 5) + assert(sortedTopK === Array(1, 2, 3, 4, 5)) + } + + test("takeOrdered with custom ordering") { + val nums = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) + implicit val ord = implicitly[Ordering[Int]].reverse + val rdd = sc.makeRDD(nums, 2) + val sortedTopK = rdd.takeOrdered(5) + assert(sortedTopK.size === 5) + assert(sortedTopK === Array(10, 9, 8, 7, 6)) + assert(sortedTopK === nums.sorted(ord).take(5)) + } + test("takeSample") { val data = sc.parallelize(1 to 100, 2) for (seed <- 1 to 5) { -- cgit v1.2.3 From ee4ce2fc51112387f28d7b422969ca2e9736e95f Mon Sep 17 00:00:00 2001 From: seanm Date: Wed, 10 Jul 2013 10:46:04 -0700 Subject: adding takeOrdered to java API --- .../main/scala/spark/api/java/JavaRDDLike.scala | 25 ++++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index b555f2030a..94b95af714 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -384,4 +384,29 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[T]] top(num, comp) } + + /** + * Returns the top K elements from this RDD as defined by + * the specified Comparator[T] and maintains the order. + * @param num the number of top elements to return + * @param comp the comparator that defines the order + * @return an array of top elements + */ + def takeOrdered(num: Int, comp: Comparator[T]): JList[T] = { + import scala.collection.JavaConversions._ + val topElems = rdd.takeOrdered(num)(Ordering.comparatorToOrdering(comp)) + val arr: java.util.Collection[T] = topElems.toSeq + new java.util.ArrayList(arr) + } + + /** + * Returns the top K elements from this RDD using the + * natural ordering for T while maintain the order. + * @param num the number of top elements to return + * @return an array of top elements + */ + def takeOrdered(num: Int): JList[T] = { + val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[T]] + takeOrdered(num, comp) + } } -- cgit v1.2.3 From cfb6447ac4903a870dd268836dc0d8952491d591 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 11:47:57 -0700 Subject: Fixed for nonexistent bytes, added unit tests, changed stdout-page to stdout --- core/src/main/scala/spark/Utils.scala | 6 +++-- .../spark/deploy/master/ui/ApplicationPage.scala | 2 +- .../scala/spark/deploy/worker/ui/IndexPage.scala | 2 +- .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 18 +++++++++----- core/src/test/scala/spark/ui/UISuite.scala | 28 ++++++++++++++++++++++ 5 files changed, 46 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 849edc13ee..512ac92d89 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -625,8 +625,10 @@ private object Utils extends Logging { def offsetBytes(path: String, a: Long, b: Long): String = { val file = new File(path) val length = file.length() - val buff = new Array[Byte](math.min((b-a).toInt, length.toInt)) - val skip = math.max(0, a) + val B = math.min(length, b) + val A = math.max(0, a) + val buff = new Array[Byte]((B-A).toInt) + val skip = A val stream = new FileInputStream(file) stream.skip(skip) diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index 49ced0d320..dae9995779 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -38,7 +38,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { state.completedApps.find(_.id == appId).getOrElse(null) }) - val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Log Pages") + val executorHeaders = Seq("ExecutorID", "Worker", "Cores", "Memory", "State", "Logs") val executors = app.executors.values.toSeq val executorTable = UIUtils.listingTable(executorHeaders, executorRow, executors) diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index af9943853f..a9c6c6d519 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -29,7 +29,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] val workerState = Await.result(stateFuture, 30 seconds) - val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Log Pages") + val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") val runningExecutorTable = UIUtils.listingTable(executorHeaders, executorRow, workerState.executors) val finishedExecutorTable = diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 602881d5f1..24356b0f63 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -71,7 +71,7 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toLong).getOrElse(0).asInstanceOf[Long] + val offset = Option(request.getParameter("offset")).map(_.toLong).getOrElse(0L) val maxBytes = 1024 * 1024 val defaultBytes = 100 * 1024 @@ -80,12 +80,18 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) val logLength = new File(path).length() val logPageLength = math.min(byteLength, maxBytes) - val logText = {Utils.offsetBytes(path, offset, offset+logPageLength)} + + val fixedOffset = + if (offset < 0) 0 + else if (offset > logLength) logLength + else offset + + val logText = {Utils.offsetBytes(path, fixedOffset, fixedOffset+logPageLength)} val backButton = - if (offset > 0) { + if (fixedOffset > 0) { + .format(appId, executorId, logType, math.max(fixedOffset-logPageLength, 0), logPageLength)}> } @@ -94,9 +100,9 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option } val nextButton = - if (offset+logPageLength < logLength) { + if (fixedOffset+logPageLength < logLength) { + format(appId, executorId, logType, fixedOffset+logPageLength, logPageLength)}> } diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index e4bb3abc33..b7a822c4bc 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -74,4 +74,32 @@ class UISuite extends FunSuite { FileUtils.deleteDirectory(tmpDir) } + + test("reading offset bytes of a file") { + val tmpDir2 = Files.createTempDir() + val f1Path = tmpDir2 + "/f1" + val f1 = new FileOutputStream(f1Path) + f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(Charsets.UTF_8)) + f1.close() + + // Read first few bytes + assert(Utils.offsetBytes(f1Path, 0, 5) === "1\n2\n3") + + // Read some middle bytes + assert(Utils.offsetBytes(f1Path, 4, 11) === "3\n4\n5\n6") + + // Read last few bytes + assert(Utils.offsetBytes(f1Path, 12, 18) === "7\n8\n9\n") + + //Read some nonexistent bytes in the beginning + assert(Utils.offsetBytes(f1Path, -5, 5) === "1\n2\n3") + + //Read some nonexistent bytes at the end + assert(Utils.offsetBytes(f1Path, 12, 22) === "7\n8\n9\n") + + //Read some nonexistent bytes on both ends + assert(Utils.offsetBytes(f1Path, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") + + FileUtils.deleteDirectory(tmpDir2) + } } -- cgit v1.2.3 From 04263e4d4609355d8e0779dea472fbb1aff5ef4f Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 13:15:42 -0700 Subject: Made some minor style changes --- core/src/main/scala/spark/Utils.scala | 11 +++++------ core/src/test/scala/spark/ui/UISuite.scala | 6 +++--- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 512ac92d89..c5627c8419 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -622,16 +622,15 @@ private object Utils extends Logging { } /** Return a string containing part of a file from byte 'a' to 'b'. */ - def offsetBytes(path: String, a: Long, b: Long): String = { + def offsetBytes(path: String, start: Long, end: Long): String = { val file = new File(path) val length = file.length() - val B = math.min(length, b) - val A = math.max(0, a) - val buff = new Array[Byte]((B-A).toInt) - val skip = A + val effectiveStart = math.min(length, start) + val effectiveEnd = math.max(0, end) + val buff = new Array[Byte]((effectiveEnd-effectiveStart).toInt) val stream = new FileInputStream(file) - stream.skip(skip) + stream.skip(effectiveStart) stream.read(buff) stream.close() Source.fromBytes(buff).mkString diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index b7a822c4bc..ab174732e3 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -91,13 +91,13 @@ class UISuite extends FunSuite { // Read last few bytes assert(Utils.offsetBytes(f1Path, 12, 18) === "7\n8\n9\n") - //Read some nonexistent bytes in the beginning + // Read some nonexistent bytes in the beginning assert(Utils.offsetBytes(f1Path, -5, 5) === "1\n2\n3") - //Read some nonexistent bytes at the end + // Read some nonexistent bytes at the end assert(Utils.offsetBytes(f1Path, 12, 22) === "7\n8\n9\n") - //Read some nonexistent bytes on both ends + // Read some nonexistent bytes on both ends assert(Utils.offsetBytes(f1Path, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") FileUtils.deleteDirectory(tmpDir2) -- cgit v1.2.3 From 0d4580360ba4bbcd2f73877743d746c071a92e34 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 13:24:26 -0700 Subject: Fixed docstring of offsetBytes to match params and wrapped for 100+ character lines --- core/src/main/scala/spark/Utils.scala | 2 +- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index c5627c8419..1c5af2700a 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -621,7 +621,7 @@ private object Utils extends Logging { callSiteInfo.firstUserLine) } - /** Return a string containing part of a file from byte 'a' to 'b'. */ + /** Return a string containing part of a file from byte 'start' to 'end'. */ def offsetBytes(path: String, start: Long, end: Long): String = { val file = new File(path) val length = file.length() diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 24356b0f63..1f4ba5de60 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -91,7 +91,8 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val backButton = if (fixedOffset > 0) { + .format(appId, executorId, logType, math.max(fixedOffset-logPageLength, 0), + logPageLength)}> } -- cgit v1.2.3 From 5f8a20b4a8a4cb7e4b14ee989f1b4f83981ce61a Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 13:53:39 -0700 Subject: Moved unit tests for Utils from UISuite to UtilsSuite --- core/src/test/scala/spark/UtilsSuite.scala | 71 ++++++++++++++++++++++++++++++ core/src/test/scala/spark/ui/UISuite.scala | 71 ------------------------------ 2 files changed, 71 insertions(+), 71 deletions(-) diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala index 4a113e16bf..73050a1d3b 100644 --- a/core/src/test/scala/spark/UtilsSuite.scala +++ b/core/src/test/scala/spark/UtilsSuite.scala @@ -71,5 +71,76 @@ class UtilsSuite extends FunSuite { assert(Utils.splitCommandString("''") === Seq("")) assert(Utils.splitCommandString("\"\"") === Seq("")) } + + test("string formatting of time durations") { + val second = 1000 + val minute = second * 60 + val hour = minute * 60 + def str = Utils.msDurationToString(_) + + assert(str(123) === "123 ms") + assert(str(second) === "1.0 s") + assert(str(second + 462) === "1.5 s") + assert(str(hour) === "1.00 h") + assert(str(minute) === "1.0 m") + assert(str(minute + 4 * second + 34) === "1.1 m") + assert(str(10 * hour + minute + 4 * second) === "10.02 h") + assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11.00 h") + } + + test("reading last n bytes of a file") { + val tmpDir = Files.createTempDir() + + // File smaller than limit + val f1Path = tmpDir + "/f1" + val f1 = new FileOutputStream(f1Path) + f1.write("a\nb\nc\nd".getBytes(Charsets.UTF_8)) + f1.close() + assert(Utils.lastNBytes(f1Path, 1024) === "a\nb\nc\nd") + + // File larger than limit + val f2Path = tmpDir + "/f2" + val f2 = new FileOutputStream(f2Path) + f2.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) + f2.close() + assert(Utils.lastNBytes(f2Path, 8) === "5\n6\n7\n8\n") + + // Request limit too + val f3Path = tmpDir + "/f2" + val f3 = new FileOutputStream(f3Path) + f3.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) + f3.close() + assert(Utils.lastNBytes(f3Path, 8) === "5\n6\n7\n8\n") + + FileUtils.deleteDirectory(tmpDir) + } + + test("reading offset bytes of a file") { + val tmpDir2 = Files.createTempDir() + val f1Path = tmpDir2 + "/f1" + val f1 = new FileOutputStream(f1Path) + f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(Charsets.UTF_8)) + f1.close() + + // Read first few bytes + assert(Utils.offsetBytes(f1Path, 0, 5) === "1\n2\n3") + + // Read some middle bytes + assert(Utils.offsetBytes(f1Path, 4, 11) === "3\n4\n5\n6") + + // Read last few bytes + assert(Utils.offsetBytes(f1Path, 12, 18) === "7\n8\n9\n") + + // Read some nonexistent bytes in the beginning + assert(Utils.offsetBytes(f1Path, -5, 5) === "1\n2\n3") + + // Read some nonexistent bytes at the end + assert(Utils.offsetBytes(f1Path, 12, 22) === "7\n8\n9\n") + + // Read some nonexistent bytes on both ends + assert(Utils.offsetBytes(f1Path, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") + + FileUtils.deleteDirectory(tmpDir2) + } } diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index ab174732e3..ef2e9c2ec4 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -31,75 +31,4 @@ class UISuite extends FunSuite { case Failure (e) => } } - - test("string formatting of time durations") { - val second = 1000 - val minute = second * 60 - val hour = minute * 60 - def str = Utils.msDurationToString(_) - - assert(str(123) === "123 ms") - assert(str(second) === "1.0 s") - assert(str(second + 462) === "1.5 s") - assert(str(hour) === "1.00 h") - assert(str(minute) === "1.0 m") - assert(str(minute + 4 * second + 34) === "1.1 m") - assert(str(10 * hour + minute + 4 * second) === "10.02 h") - assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11.00 h") - } - - test("reading last n bytes of a file") { - val tmpDir = Files.createTempDir() - - // File smaller than limit - val f1Path = tmpDir + "/f1" - val f1 = new FileOutputStream(f1Path) - f1.write("a\nb\nc\nd".getBytes(Charsets.UTF_8)) - f1.close() - assert(Utils.lastNBytes(f1Path, 1024) === "a\nb\nc\nd") - - // File larger than limit - val f2Path = tmpDir + "/f2" - val f2 = new FileOutputStream(f2Path) - f2.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) - f2.close() - assert(Utils.lastNBytes(f2Path, 8) === "5\n6\n7\n8\n") - - // Request limit too - val f3Path = tmpDir + "/f2" - val f3 = new FileOutputStream(f3Path) - f3.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) - f3.close() - assert(Utils.lastNBytes(f3Path, 8) === "5\n6\n7\n8\n") - - FileUtils.deleteDirectory(tmpDir) - } - - test("reading offset bytes of a file") { - val tmpDir2 = Files.createTempDir() - val f1Path = tmpDir2 + "/f1" - val f1 = new FileOutputStream(f1Path) - f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(Charsets.UTF_8)) - f1.close() - - // Read first few bytes - assert(Utils.offsetBytes(f1Path, 0, 5) === "1\n2\n3") - - // Read some middle bytes - assert(Utils.offsetBytes(f1Path, 4, 11) === "3\n4\n5\n6") - - // Read last few bytes - assert(Utils.offsetBytes(f1Path, 12, 18) === "7\n8\n9\n") - - // Read some nonexistent bytes in the beginning - assert(Utils.offsetBytes(f1Path, -5, 5) === "1\n2\n3") - - // Read some nonexistent bytes at the end - assert(Utils.offsetBytes(f1Path, 12, 22) === "7\n8\n9\n") - - // Read some nonexistent bytes on both ends - assert(Utils.offsetBytes(f1Path, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") - - FileUtils.deleteDirectory(tmpDir2) - } } -- cgit v1.2.3 From dbe948d9a2198845feb7c8bfa738286db41344d9 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 14:15:41 -0700 Subject: Moved appropriate import files from UISuite to UtilsSuite --- core/src/test/scala/spark/UtilsSuite.scala | 4 ++++ core/src/test/scala/spark/ui/UISuite.scala | 5 ----- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala index 73050a1d3b..942702c826 100644 --- a/core/src/test/scala/spark/UtilsSuite.scala +++ b/core/src/test/scala/spark/UtilsSuite.scala @@ -3,6 +3,10 @@ package spark import org.scalatest.FunSuite import java.io.{ByteArrayOutputStream, ByteArrayInputStream} import scala.util.Random +import com.google.common.io.Files +import org.apache.commons.io.FileUtils +import java.io.{FileOutputStream, File} +import com.google.common.base.Charsets class UtilsSuite extends FunSuite { diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index ef2e9c2ec4..cd7d105928 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -4,11 +4,6 @@ import org.scalatest.FunSuite import org.eclipse.jetty.server.Server import java.net.ServerSocket import scala.util.{Failure, Success, Try} -import spark.Utils -import com.google.common.io.Files -import org.apache.commons.io.FileUtils -import java.io.{FileOutputStream, File} -import com.google.common.base.Charsets class UISuite extends FunSuite { test("jetty port increases under contention") { -- cgit v1.2.3 From f5f3b272f874a3ac22f4e0b2caab9053657a9124 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 14:52:29 -0700 Subject: Fixed mixup of start/end, moved more import files --- core/src/main/scala/spark/Utils.scala | 4 ++-- core/src/test/scala/spark/UtilsSuite.scala | 9 ++++----- core/src/test/scala/spark/ui/UISuite.scala | 4 ++-- 3 files changed, 8 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 1c5af2700a..1da9c9574e 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -625,8 +625,8 @@ private object Utils extends Logging { def offsetBytes(path: String, start: Long, end: Long): String = { val file = new File(path) val length = file.length() - val effectiveStart = math.min(length, start) - val effectiveEnd = math.max(0, end) + val effectiveEnd = math.min(length, end) + val effectiveStart = math.max(0, start) val buff = new Array[Byte]((effectiveEnd-effectiveStart).toInt) val stream = new FileInputStream(file) diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala index 942702c826..d83a0307fa 100644 --- a/core/src/test/scala/spark/UtilsSuite.scala +++ b/core/src/test/scala/spark/UtilsSuite.scala @@ -1,12 +1,11 @@ package spark -import org.scalatest.FunSuite -import java.io.{ByteArrayOutputStream, ByteArrayInputStream} -import scala.util.Random +import com.google.common.base.Charsets import com.google.common.io.Files +import java.io.{ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream, File} +import org.scalatest.FunSuite import org.apache.commons.io.FileUtils -import java.io.{FileOutputStream, File} -import com.google.common.base.Charsets +import scala.util.Random class UtilsSuite extends FunSuite { diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index cd7d105928..fc0c160720 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -1,9 +1,9 @@ package spark.ui +import scala.util.{Failure, Success, Try} +import java.net.ServerSocket import org.scalatest.FunSuite import org.eclipse.jetty.server.Server -import java.net.ServerSocket -import scala.util.{Failure, Success, Try} class UISuite extends FunSuite { test("jetty port increases under contention") { -- cgit v1.2.3 From 24196c91f0bd560c08399eed3cb1903248ca497a Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 15:27:52 -0700 Subject: Changed buffer to 10,000 bytes, created scrollbar for fixed-height log --- core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala | 4 ++-- core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala | 4 ++-- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 6 +++--- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index dae9995779..27549e3b4a 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -90,9 +90,9 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { {executor.memory} {executor.state} - stdout - stderr diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index a9c6c6d519..313ecec084 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -88,9 +88,9 @@ private[spark] class IndexPage(parent: WorkerWebUI) { - stdout - stderr diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 1f4ba5de60..490fe15fcf 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -118,9 +118,9 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option {backButton} {nextButton}

    -
    {logText}
    - {backButton} - {nextButton} +
    +
    {logText}
    +
    UIUtils.basicSparkPage(content, "Log Page for " + appId) -- cgit v1.2.3 From 74bd3fc6806a4185a95a19c107e1a4776582db44 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 15:44:28 -0700 Subject: Added byte range on log pages --- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 490fe15fcf..cac281eef6 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -86,7 +86,10 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option else if (offset > logLength) logLength else offset - val logText = {Utils.offsetBytes(path, fixedOffset, fixedOffset+logPageLength)} + val endOffset = math.min(fixedOffset+logPageLength, logLength) + + val range =

    Bytes {fixedOffset.toString} - {(endOffset).toString} of {logLength}

    + val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} val backButton = if (fixedOffset > 0) { @@ -101,9 +104,9 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option } val nextButton = - if (fixedOffset+logPageLength < logLength) { + if (endOffset < logLength) { + format(appId, executorId, logType, endOffset, logPageLength)}> } @@ -114,6 +117,7 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val content = + {range}
    {backButton} {nextButton} -- cgit v1.2.3 From ebf5b8c7c5fe72fbe2d74a1bfa67cbdc9f9be0ea Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Wed, 10 Jul 2013 22:16:06 -0700 Subject: Updating README to reflect Scala 2.9.3 requirements --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index ba24ab43b1..1dd96a0a4a 100644 --- a/README.md +++ b/README.md @@ -12,7 +12,7 @@ This README file only contains basic setup instructions. ## Building -Spark requires Scala 2.9.2 (Scala 2.10 is not yet supported). The project is +Spark requires Scala 2.9.3 (Scala 2.10 is not yet supported). The project is built using Simple Build Tool (SBT), which is packaged with it. To build Spark and its example programs, run: -- cgit v1.2.3 From 0ecc33f0c80733ff88518157be543ec458a76bdb Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 11:25:58 -0700 Subject: Added byte range, page title with log name, previous/next bytes buttons, initialization to end of log, large default buffer, buggy back to master link --- .../spark/deploy/master/ui/ApplicationPage.scala | 4 +-- .../main/scala/spark/deploy/worker/Worker.scala | 2 +- .../scala/spark/deploy/worker/ui/IndexPage.scala | 9 +++--- .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 37 +++++++++++++--------- 4 files changed, 30 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index 27549e3b4a..b2589abb89 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -90,9 +90,9 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { {executor.memory} {executor.state} - stdout - stderr diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 6ae1cef940..f20ea42d7f 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -77,7 +77,7 @@ private[spark] class Worker( sparkHome = new File(Option(System.getenv("SPARK_HOME")).getOrElse(".")) logInfo("Spark home: " + sparkHome) createWorkDir() - webUi = new WorkerWebUI(self, workDir, Some(webUiPort)) + webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) webUi.start() connectToMaster() } diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index 313ecec084..fc1ec31b4d 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -16,17 +16,18 @@ import spark.Utils import spark.ui.UIUtils private[spark] class IndexPage(parent: WorkerWebUI) { + val workerActor = parent.worker.self val worker = parent.worker val timeout = parent.timeout def renderJson(request: HttpServletRequest): JValue = { - val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] + val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerState] val workerState = Await.result(stateFuture, 30 seconds) JsonProtocol.writeWorkerState(workerState) } def render(request: HttpServletRequest): Seq[Node] = { - val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] + val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerState] val workerState = Await.result(stateFuture, 30 seconds) val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") @@ -88,9 +89,9 @@ private[spark] class IndexPage(parent: WorkerWebUI) { - stdout - stderr diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index cac281eef6..3ab0f2eded 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -9,6 +9,7 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} +import spark.deploy.worker.Worker import spark.{Utils, Logging} import spark.ui.JettyUtils import spark.ui.JettyUtils._ @@ -18,7 +19,7 @@ import spark.ui.UIUtils * Web UI server for the standalone worker. */ private[spark] -class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option[Int] = None) +class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None) extends Logging { implicit val timeout = Timeout( Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")) @@ -71,14 +72,14 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toLong).getOrElse(0L) val maxBytes = 1024 * 1024 - val defaultBytes = 100 * 1024 + val defaultBytes = 10000 val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) val logLength = new File(path).length() + val offset = Option(request.getParameter("offset")).map(_.toLong).getOrElse(logLength-10000) val logPageLength = math.min(byteLength, maxBytes) val fixedOffset = @@ -88,46 +89,52 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option val endOffset = math.min(fixedOffset+logPageLength, logLength) - val range =

    Bytes {fixedOffset.toString} - {(endOffset).toString} of {logLength}

    - val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} + val linkToMaster =

    Back to Master

    + + val range = Bytes {fixedOffset.toString} - {(endOffset).toString} of {logLength} val backButton = if (fixedOffset > 0) { - + } else { - + } val nextButton = if (endOffset < logLength) { - + } else { - + } + val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} + val content = - {range} -
    - {backButton} - {nextButton} -

    + {linkToMaster} +
    +
    +
    {backButton}
    +
    {range}
    +
    {nextButton}
    +
    +
    {logText}
    - UIUtils.basicSparkPage(content, "Log Page for " + appId) + UIUtils.basicSparkPage(content, logType + " log Page for " + appId) } def stop() { -- cgit v1.2.3 From 044d4577ec564fe0f0bd9c1ad86f4749b50fc100 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 12:02:15 -0700 Subject: Fixed capitalization of log page --- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 3ab0f2eded..923b835c3f 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -134,7 +134,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I - UIUtils.basicSparkPage(content, logType + " log Page for " + appId) + UIUtils.basicSparkPage(content, logType + " log page for " + appId) } def stop() { -- cgit v1.2.3 From e3a3fcf61b117d63db33ee28928dfd77cfd935b8 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 12:16:38 -0700 Subject: Scrollbar on log pages appear automatically --- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 923b835c3f..a3cbe4f5d3 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -129,7 +129,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
    {nextButton}

    -
    +
    {logText}
    -- cgit v1.2.3 From 11872888ca44d0c08c157973a8e35d344b0119e4 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 14:56:37 -0700 Subject: Created getByteRange function for logs and log pages, removed lastNBytes function --- core/src/main/scala/spark/Utils.scala | 30 +++++++++++---- .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 45 ++++++++++------------ 2 files changed, 44 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 1da9c9574e..04041d1179 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -621,6 +621,29 @@ private object Utils extends Logging { callSiteInfo.firstUserLine) } + /** Determine the byte range for a log or log page. */ + def getByteRange(path: String, offset: Option[String], byteLength: Option[String]) + : (Long, Long, Long, Int) = { + val defaultBytes = 10000 + val maxBytes = 1024 * 1024 + + val file = new File(path) + val logLength = file.length() + val getOffset = offset.map(_.toLong).getOrElse(logLength-defaultBytes) + + val fixedOffset = + if (getOffset < 0) 0L + else if (getOffset > logLength) logLength + else getOffset + + val getByteLength = byteLength.map(_.toInt).getOrElse(defaultBytes) + val logPageLength = math.min(getByteLength, maxBytes) + + val endOffset = math.min(fixedOffset+logPageLength, logLength) + + (fixedOffset, endOffset, logLength, logPageLength) + } + /** Return a string containing part of a file from byte 'start' to 'end'. */ def offsetBytes(path: String, start: Long, end: Long): String = { val file = new File(path) @@ -636,13 +659,6 @@ private object Utils extends Logging { Source.fromBytes(buff).mkString } - /** Return a string containing the last `n` bytes of a file. */ - def lastNBytes(path: String, n: Int): String = { - val file = new File(path) - val length = file.length() - offsetBytes(path, length-n, length) - } - /** * Clone an object using a Spark serializer. */ diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index a3cbe4f5d3..f8ac682dbf 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -57,41 +57,39 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") + val offset = Option(request.getParameter("offset")) + val byteLength = Option(request.getParameter("byteLength")) + val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val maxBytes = 1024 * 1024 // Guard against OOM - val defaultBytes = 100 * 1024 - val numBytes = Option(request.getParameter("numBytes")) - .flatMap(s => Some(s.toInt)).getOrElse(defaultBytes) + val offsetBytes = Utils.getByteRange(path, offset, byteLength) + val fixedOffset = offsetBytes._1 + val endOffset = offsetBytes._2 + val logLength = offsetBytes._3 - val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val pre = "==== Last %s bytes of %s/%s/%s ====\n".format(numBytes, appId, executorId, logType) - pre + Utils.lastNBytes(path, math.min(numBytes, maxBytes)) + val pre = "==== Bytes %s-%s of %s of %s/%s/%s ====\n" + .format(fixedOffset, endOffset, logLength, appId, executorId, logType) + pre + Utils.offsetBytes(path, fixedOffset, endOffset) } def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - - val maxBytes = 1024 * 1024 - val defaultBytes = 10000 - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) - + val offset = Option(request.getParameter("offset")) + val byteLength = Option(request.getParameter("byteLength")) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val logLength = new File(path).length() - val offset = Option(request.getParameter("offset")).map(_.toLong).getOrElse(logLength-10000) - val logPageLength = math.min(byteLength, maxBytes) - val fixedOffset = - if (offset < 0) 0 - else if (offset > logLength) logLength - else offset + val offsetBytes = Utils.getByteRange(path, offset, byteLength) + val fixedOffset = offsetBytes._1 + val endOffset = offsetBytes._2 + val logLength = offsetBytes._3 + val logPageLength = offsetBytes._4 - val endOffset = math.min(fixedOffset+logPageLength, logLength) + val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} val linkToMaster =

    Back to Master

    - val range = Bytes {fixedOffset.toString} - {(endOffset).toString} of {logLength} + val range = Bytes {fixedOffset.toString} - {endOffset.toString} of {logLength} val backButton = if (fixedOffset > 0) { @@ -116,8 +114,6 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } - val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} - val content = @@ -134,7 +130,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
    - UIUtils.basicSparkPage(content, logType + " log page for " + appId) + UIUtils.basicSparkPage(content, request.getParameter("logType") + " log page for " + + request.getParameter("appId")) } def stop() { -- cgit v1.2.3 From 15fd11d65735265f6fcc25419b43c8c104a4bd17 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 15:01:50 -0700 Subject: Removed redundant calls to request by logPage --- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index f8ac682dbf..676de231f0 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -130,8 +130,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I - UIUtils.basicSparkPage(content, request.getParameter("logType") + " log page for " + - request.getParameter("appId")) + UIUtils.basicSparkPage(content, logType + " log page for " + appId) } def stop() { -- cgit v1.2.3 From 5d5dbc39f603201a3596031542d52742cfb30139 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 15:22:45 -0700 Subject: getByteRange moved to WorkerWebUI, takes converted parameters, returns only start/end offset --- core/src/main/scala/spark/Utils.scala | 23 ------------ .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 43 +++++++++++++++++----- 2 files changed, 34 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 04041d1179..5a37532306 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -621,29 +621,6 @@ private object Utils extends Logging { callSiteInfo.firstUserLine) } - /** Determine the byte range for a log or log page. */ - def getByteRange(path: String, offset: Option[String], byteLength: Option[String]) - : (Long, Long, Long, Int) = { - val defaultBytes = 10000 - val maxBytes = 1024 * 1024 - - val file = new File(path) - val logLength = file.length() - val getOffset = offset.map(_.toLong).getOrElse(logLength-defaultBytes) - - val fixedOffset = - if (getOffset < 0) 0L - else if (getOffset > logLength) logLength - else getOffset - - val getByteLength = byteLength.map(_.toInt).getOrElse(defaultBytes) - val logPageLength = math.min(getByteLength, maxBytes) - - val endOffset = math.min(fixedOffset+logPageLength, logLength) - - (fixedOffset, endOffset, logLength, logPageLength) - } - /** Return a string containing part of a file from byte 'start' to 'end'. */ def offsetBytes(path: String, start: Long, end: Long): String = { val file = new File(path) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 676de231f0..1fb59de1d8 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -57,14 +57,15 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")) - val byteLength = Option(request.getParameter("byteLength")) + val offset = Option(request.getParameter("offset")).map(_.toLong) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val offsetBytes = Utils.getByteRange(path, offset, byteLength) + val offsetBytes = getByteRange(path, offset, byteLength) val fixedOffset = offsetBytes._1 val endOffset = offsetBytes._2 - val logLength = offsetBytes._3 + val file = new File(path) + val logLength = file.length val pre = "==== Bytes %s-%s of %s of %s/%s/%s ====\n" .format(fixedOffset, endOffset, logLength, appId, executorId, logType) @@ -75,15 +76,16 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")) - val byteLength = Option(request.getParameter("byteLength")) + val offset = Option(request.getParameter("offset")).map(_.toLong) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val offsetBytes = Utils.getByteRange(path, offset, byteLength) + val offsetBytes = getByteRange(path, offset, byteLength) val fixedOffset = offsetBytes._1 val endOffset = offsetBytes._2 - val logLength = offsetBytes._3 - val logPageLength = offsetBytes._4 + val file = new File(path) + val logLength = file.length + val logPageLength = endOffset-fixedOffset val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} @@ -133,6 +135,29 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I UIUtils.basicSparkPage(content, logType + " log page for " + appId) } + /** Determine the byte range for a log or log page. */ + def getByteRange(path: String, offset: Option[Long], byteLength: Option[Int]) + : (Long, Long) = { + val defaultBytes = 10000 + val maxBytes = 1024 * 1024 + + val file = new File(path) + val logLength = file.length() + val getOffset = offset.getOrElse(logLength-defaultBytes) + + val fixedOffset = + if (getOffset < 0) 0L + else if (getOffset > logLength) logLength + else getOffset + + val getByteLength = byteLength.getOrElse(defaultBytes) + val logPageLength = math.min(getByteLength, maxBytes) + + val endOffset = math.min(fixedOffset+logPageLength, logLength) + + (fixedOffset, endOffset) + } + def stop() { server.foreach(_.stop()) } -- cgit v1.2.3 From fdc226a14cfdb5f699627191f1682763b8571126 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 15:36:43 -0700 Subject: Clarified start and end byte variable names --- .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 36 ++++++++++------------ 1 file changed, 16 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 1fb59de1d8..2a82c35231 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -61,15 +61,13 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val offsetBytes = getByteRange(path, offset, byteLength) - val fixedOffset = offsetBytes._1 - val endOffset = offsetBytes._2 + val (startByte, endByte) = getByteRange(path, offset, byteLength) val file = new File(path) val logLength = file.length val pre = "==== Bytes %s-%s of %s of %s/%s/%s ====\n" - .format(fixedOffset, endOffset, logLength, appId, executorId, logType) - pre + Utils.offsetBytes(path, fixedOffset, endOffset) + .format(startByte, endByte, logLength, appId, executorId, logType) + pre + Utils.offsetBytes(path, startByte, endByte) } def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { @@ -80,25 +78,23 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) - val offsetBytes = getByteRange(path, offset, byteLength) - val fixedOffset = offsetBytes._1 - val endOffset = offsetBytes._2 + val (startByte, endByte) = getByteRange(path, offset, byteLength) val file = new File(path) val logLength = file.length - val logPageLength = endOffset-fixedOffset + val logPageLength = endByte-startByte - val logText = {Utils.offsetBytes(path, fixedOffset, endOffset)} + val logText = {Utils.offsetBytes(path, startByte, endByte)} val linkToMaster =

    Back to Master

    - val range = Bytes {fixedOffset.toString} - {endOffset.toString} of {logLength} + val range = Bytes {startByte.toString} - {endByte.toString} of {logLength} val backButton = - if (fixedOffset > 0) { + if (startByte > 0) { - + } else { @@ -106,10 +102,10 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } val nextButton = - if (endOffset < logLength) { + if (endByte < logLength) { - + format(appId, executorId, logType, endByte, logPageLength)}> + } else { @@ -145,7 +141,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val logLength = file.length() val getOffset = offset.getOrElse(logLength-defaultBytes) - val fixedOffset = + val startByte = if (getOffset < 0) 0L else if (getOffset > logLength) logLength else getOffset @@ -153,9 +149,9 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val getByteLength = byteLength.getOrElse(defaultBytes) val logPageLength = math.min(getByteLength, maxBytes) - val endOffset = math.min(fixedOffset+logPageLength, logLength) + val endByte = math.min(startByte+logPageLength, logLength) - (fixedOffset, endOffset) + (startByte, endByte) } def stop() { -- cgit v1.2.3 From 9ed036ccdbb1c7aa3279998f3177c1be6b01e16b Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 16:33:53 -0700 Subject: Replaced logPageLength with byteLength to prevent buffer shrink bug --- .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 23 +++++++++++----------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 2a82c35231..94d919f57a 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -54,11 +54,12 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } def log(request: HttpServletRequest): String = { + val defaultBytes = 10000 val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") val offset = Option(request.getParameter("offset")).map(_.toLong) - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) val (startByte, endByte) = getByteRange(path, offset, byteLength) @@ -71,17 +72,18 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { + val defaultBytes = 10000 val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") val offset = Option(request.getParameter("offset")).map(_.toLong) - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) val path = "%s/%s/%s/%s".format(workDir.getPath, appId, executorId, logType) val (startByte, endByte) = getByteRange(path, offset, byteLength) val file = new File(path) val logLength = file.length - val logPageLength = endByte-startByte + //val logPageLength = endByte-startByte val logText = {Utils.offsetBytes(path, startByte, endByte)} @@ -92,9 +94,9 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val backButton = if (startByte > 0) { - + .format(appId, executorId, logType, math.max(startByte-byteLength, 0), + byteLength)}> + } else { @@ -104,8 +106,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val nextButton = if (endByte < logLength) { - + format(appId, executorId, logType, endByte, byteLength)}> + } else { @@ -132,7 +134,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } /** Determine the byte range for a log or log page. */ - def getByteRange(path: String, offset: Option[Long], byteLength: Option[Int]) + def getByteRange(path: String, offset: Option[Long], byteLength: Int) : (Long, Long) = { val defaultBytes = 10000 val maxBytes = 1024 * 1024 @@ -146,8 +148,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I else if (getOffset > logLength) logLength else getOffset - val getByteLength = byteLength.getOrElse(defaultBytes) - val logPageLength = math.min(getByteLength, maxBytes) + val logPageLength = math.min(byteLength, maxBytes) val endByte = math.min(startByte+logPageLength, logLength) -- cgit v1.2.3 From ece2388585a468dd57c973f254ba20c24299a464 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 16:35:56 -0700 Subject: Removed logPageLength from logPage --- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 94d919f57a..1ab65d41e8 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -83,7 +83,6 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val (startByte, endByte) = getByteRange(path, offset, byteLength) val file = new File(path) val logLength = file.length - //val logPageLength = endByte-startByte val logText = {Utils.offsetBytes(path, startByte, endByte)} -- cgit v1.2.3 From a32784109d11e061cdca2eced07ac01be2061056 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 16:57:55 -0700 Subject: Fixed links for "Back to Master" --- core/src/main/scala/spark/deploy/master/Master.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index dbdc8e1057..4dd6c448a9 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -71,7 +71,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } else { addWorker(id, host, workerPort, cores, memory, worker_webUiPort, publicAddress) context.watch(sender) // This doesn't work with remote actors but helps for testing - sender ! RegisteredWorker("http://" + masterPublicAddress + ":" + webUi.boundPort) + sender ! RegisteredWorker("http://" + masterPublicAddress + ":" + webUi.boundPort.get) schedule() } } -- cgit v1.2.3 From 6d054487bf4c9e9aa9033187f73e4e9d8366bc37 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 17:12:17 -0700 Subject: Replace default buffer value to 100 GB, changed buttons to use String notation, removed default buffer parameter in UI URLs --- .../main/scala/spark/deploy/master/ui/ApplicationPage.scala | 4 ++-- core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala | 4 ++-- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 12 +++++++----- 3 files changed, 11 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index b2589abb89..8553377d8f 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -90,9 +90,9 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { {executor.memory} {executor.state} - stdout - stderr diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index fc1ec31b4d..c515f2e238 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -89,9 +89,9 @@ private[spark] class IndexPage(parent: WorkerWebUI) { - stdout - stderr diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 1ab65d41e8..4e7f86d77a 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -54,7 +54,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } def log(request: HttpServletRequest): String = { - val defaultBytes = 10000 + val defaultBytes = 100 * 1024 val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") @@ -72,7 +72,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { - val defaultBytes = 10000 + val defaultBytes = 100 * 1024 val appId = request.getParameter("appId") val executorId = request.getParameter("executorId") val logType = request.getParameter("logType") @@ -95,7 +95,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I - + } else { @@ -106,7 +107,8 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I if (endByte < logLength) { - + } else { @@ -135,7 +137,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I /** Determine the byte range for a log or log page. */ def getByteRange(path: String, offset: Option[Long], byteLength: Int) : (Long, Long) = { - val defaultBytes = 10000 + val defaultBytes = 100 * 1024 val maxBytes = 1024 * 1024 val file = new File(path) -- cgit v1.2.3 From 5c67ca027834582bcd6cdd55baee5fd74e743a71 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 17:31:59 -0700 Subject: Remove "Bytes" in lieu of String notation --- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 4e7f86d77a..ccd55c1ce4 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -95,24 +95,22 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I - + } else { - + } val nextButton = if (endByte < logLength) { - + } else { - + } val content = -- cgit v1.2.3 From a2c915fba84a1deadac199994322e0f15d5750b4 Mon Sep 17 00:00:00 2001 From: seanm Date: Thu, 11 Jul 2013 18:55:00 -0700 Subject: giving order to top and making tests more clear --- core/src/main/scala/spark/RDD.scala | 6 ++---- core/src/test/scala/spark/RDDSuite.scala | 8 ++++---- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index af52040fa6..4c1591ed5a 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -778,7 +778,7 @@ abstract class RDD[T: ClassManifest]( }.reduce { (queue1, queue2) => queue1 ++= queue2 queue1 - }.toArray + }.toArray.sorted(ord.reverse) } /** @@ -789,9 +789,7 @@ abstract class RDD[T: ClassManifest]( * @param ord the implicit ordering for T * @return an array of top elements */ - def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = { - top(num)(ord.reverse).sorted(ord) - } + def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = top(num)(ord.reverse) /** * Save this RDD as a text file, using string representations of elements. diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index fe17d1d5e7..aa3ee5f5ee 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -240,7 +240,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { val ints = sc.makeRDD(scala.util.Random.shuffle(nums), 2) val topK = ints.top(5) assert(topK.size === 5) - assert(topK.sorted === nums.sorted.takeRight(5)) + assert(topK === nums.reverse.take(5)) } test("top with custom ordering") { @@ -255,9 +255,9 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("takeOrdered with predefined ordering") { val nums = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) val rdd = sc.makeRDD(nums, 2) - val sortedTopK = rdd.takeOrdered(5) - assert(sortedTopK.size === 5) - assert(sortedTopK === Array(1, 2, 3, 4, 5)) + val sortedLowerK = rdd.takeOrdered(5) + assert(sortedLowerK.size === 5) + assert(sortedLowerK === Array(1, 2, 3, 4, 5)) } test("takeOrdered with custom ordering") { -- cgit v1.2.3 From a1662326e9b1486361eba2f2caf903875fbba597 Mon Sep 17 00:00:00 2001 From: seanm Date: Fri, 12 Jul 2013 08:38:19 -0700 Subject: comment adjustment to takeOrdered --- core/src/main/scala/spark/RDD.scala | 2 +- core/src/main/scala/spark/api/java/JavaRDDLike.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 4c1591ed5a..8aa77266bc 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -782,7 +782,7 @@ abstract class RDD[T: ClassManifest]( } /** - * Returns the top K elements from this RDD as defined by + * Returns the first K elements from this RDD as defined by * the specified implicit Ordering[T] and maintains the * ordering. * @param num the number of top elements to return diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index 94b95af714..27f40ecdfd 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -386,7 +386,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { } /** - * Returns the top K elements from this RDD as defined by + * Returns the first K elements from this RDD as defined by * the specified Comparator[T] and maintains the order. * @param num the number of top elements to return * @param comp the comparator that defines the order @@ -400,7 +400,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { } /** - * Returns the top K elements from this RDD using the + * Returns the first K elements from this RDD using the * natural ordering for T while maintain the order. * @param num the number of top elements to return * @return an array of top elements -- cgit v1.2.3 From cd7259b4b8d8abbff6db963fd8f84d4bd0b3737b Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Fri, 12 Jul 2013 11:51:14 -0700 Subject: Fixes typos in Spark Streaming Programming Guide These typos were reported on the spark-users mailing list, see: https://groups.google.com/d/msg/spark-users/SyLGgJlKCrI/LpeBypOkSMUJ --- docs/streaming-programming-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index f5788dc467..8cd1b0cd66 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -7,7 +7,7 @@ title: Spark Streaming Programming Guide {:toc} # Overview -A Spark Streaming application is very similar to a Spark application; it consists of a *driver program* that runs the user's `main` function and continuous executes various *parallel operations* on input streams of data. The main abstraction Spark Streaming provides is a *discretized stream* (DStream), which is a continuous sequence of RDDs (distributed collections of elements) representing a continuous stream of data. DStreams can be created from live incoming data (such as data from a socket, Kafka, etc.) or can be generated by transformong existing DStreams using parallel operators like `map`, `reduce`, and `window`. The basic processing model is as follows: +A Spark Streaming application is very similar to a Spark application; it consists of a *driver program* that runs the user's `main` function and continuous executes various *parallel operations* on input streams of data. The main abstraction Spark Streaming provides is a *discretized stream* (DStream), which is a continuous sequence of RDDs (distributed collections of elements) representing a continuous stream of data. DStreams can be created from live incoming data (such as data from a socket, Kafka, etc.) or can be generated by transforming existing DStreams using parallel operators like `map`, `reduce`, and `window`. The basic processing model is as follows: (i) While a Spark Streaming driver program is running, the system receives data from various sources and and divides it into batches. Each batch of data is treated as an RDD, that is, an immutable parallel collection of data. These input RDDs are saved in memory and replicated to two nodes for fault-tolerance. This sequence of RDDs is collectively called an InputDStream. (ii) Data received by InputDStreams are processed using DStream operations. Since all data is represented as RDDs and all DStream operations as RDD operations, data is automatically recovered in the event of node failures. @@ -20,7 +20,7 @@ The first thing a Spark Streaming program must do is create a `StreamingContext` new StreamingContext(master, appName, batchDuration, [sparkHome], [jars]) {% endhighlight %} -The `master` parameter is a standard [Spark cluster URL](scala-programming-guide.html#master-urls) and can be "local" for local testing. The `appName` is a name of your program, which will be shown on your cluster's web UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such the cluster can keep up with the processing of the data streams. Start with something conservative like 5 seconds. See the [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion. Finally, `sparkHome` and `jars` are necessary when running on a cluster to specify the location of your code, as described in the [Spark programming guide](scala-programming-guide.html#deploying-code-on-a-cluster). +The `master` parameter is a standard [Spark cluster URL](scala-programming-guide.html#master-urls) and can be "local" for local testing. The `appName` is a name of your program, which will be shown on your cluster's web UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such that the cluster can keep up with the processing of the data streams. Start with something conservative like 5 seconds. See the [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion. Finally, `sparkHome` and `jars` are necessary when running on a cluster to specify the location of your code, as described in the [Spark programming guide](scala-programming-guide.html#deploying-code-on-a-cluster). This constructor creates a SparkContext for your job as well, which can be accessed with `streamingContext.sparkContext`. -- cgit v1.2.3 From 531a7e55745ad76d17d41f80dbdea0072acce079 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Wed, 10 Jul 2013 17:59:43 -0700 Subject: Pass executor env vars (e.g. SPARK_CLASSPATH) to compute-classpath. --- core/src/main/scala/spark/Utils.scala | 12 +++++++++--- core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala | 4 +++- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 64547bbdcd..d2bf151cbf 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -6,6 +6,7 @@ import java.util.{Locale, Random, UUID} import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadFactory, ThreadPoolExecutor} import java.util.regex.Pattern +import scala.collection.Map import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.collection.JavaConversions._ import scala.io.Source @@ -545,10 +546,15 @@ private object Utils extends Logging { /** * Execute a command and get its output, throwing an exception if it yields a code other than 0. */ - def executeAndGetOutput(command: Seq[String], workingDir: File = new File(".")): String = { - val process = new ProcessBuilder(command: _*) + def executeAndGetOutput(command: Seq[String], workingDir: File = new File("."), + extraEnvironment: Map[String, String] = Map.empty): String = { + val builder = new ProcessBuilder(command: _*) .directory(workingDir) - .start() + val environment = builder.environment() + for ((key, value) <- extraEnvironment) { + environment.put(key, value) + } + val process = builder.start() new Thread("read stderr for " + command(0)) { override def run() { for (line <- Source.fromInputStream(process.getErrorStream).getLines) { diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index d7f58b2cb1..5d3d54c65e 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -98,7 +98,9 @@ private[spark] class ExecutorRunner( // Figure out our classpath with the external compute-classpath script val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh" - val classPath = Utils.executeAndGetOutput(Seq(sparkHome + "/bin/compute-classpath" + ext)) + val classPath = Utils.executeAndGetOutput( + Seq(sparkHome + "/bin/compute-classpath" + ext), + extraEnvironment=appDesc.command.environment) Seq("-cp", classPath) ++ libraryOpts ++ userOpts ++ memoryOpts } -- cgit v1.2.3 From 73984b96a8450674b472676eaa855cc2df68a754 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 12 Jul 2013 14:26:56 -0700 Subject: Removed unit test of nonexistent function Utils.lastNBytes --- core/src/test/scala/spark/UtilsSuite.scala | 27 --------------------------- 1 file changed, 27 deletions(-) diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala index d83a0307fa..1e1260f606 100644 --- a/core/src/test/scala/spark/UtilsSuite.scala +++ b/core/src/test/scala/spark/UtilsSuite.scala @@ -91,33 +91,6 @@ class UtilsSuite extends FunSuite { assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11.00 h") } - test("reading last n bytes of a file") { - val tmpDir = Files.createTempDir() - - // File smaller than limit - val f1Path = tmpDir + "/f1" - val f1 = new FileOutputStream(f1Path) - f1.write("a\nb\nc\nd".getBytes(Charsets.UTF_8)) - f1.close() - assert(Utils.lastNBytes(f1Path, 1024) === "a\nb\nc\nd") - - // File larger than limit - val f2Path = tmpDir + "/f2" - val f2 = new FileOutputStream(f2Path) - f2.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) - f2.close() - assert(Utils.lastNBytes(f2Path, 8) === "5\n6\n7\n8\n") - - // Request limit too - val f3Path = tmpDir + "/f2" - val f3 = new FileOutputStream(f3Path) - f3.write("1\n2\n3\n4\n5\n6\n7\n8\n".getBytes(Charsets.UTF_8)) - f3.close() - assert(Utils.lastNBytes(f3Path, 8) === "5\n6\n7\n8\n") - - FileUtils.deleteDirectory(tmpDir) - } - test("reading offset bytes of a file") { val tmpDir2 = Files.createTempDir() val f1Path = tmpDir2 + "/f1" -- cgit v1.2.3 From 6855338e1400638188358a7d7926eb86f668c160 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 12 Jul 2013 19:24:16 -0700 Subject: Show block locations in Web UI. This fixes SPARK-769. Support is added for enumerating the locations of blocks in the UI. There is also some minor cleanup in StorageUtils. --- core/src/main/scala/spark/storage/StorageUtils.scala | 11 +++++++++-- core/src/main/scala/spark/ui/storage/RDDPage.scala | 17 +++++++++++++---- 2 files changed, 22 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index 950c0cdf35..3e7fa287e5 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -39,12 +39,19 @@ case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, private[spark] object StorageUtils { - /* Given the current storage status of the BlockManager, returns information for each RDD */ - def rddInfoFromStorageStatus(storageStatusList: Array[StorageStatus], + /* Returns RDD-level information, compiled from a list of StorageStatus objects */ + def rddInfoFromStorageStatus(storageStatusList: Seq[StorageStatus], sc: SparkContext) : Array[RDDInfo] = { rddInfoFromBlockStatusList(storageStatusList.flatMap(_.blocks).toMap, sc) } + /* Returns a map of blocks to their locations, compiled from a list of StorageStatus objects */ + def blockLocationsFromStorageStatus(storageStatusList: Seq[StorageStatus]) = { + val blockLocationPairs = storageStatusList + .flatMap(s => s.blocks.map(b => (b._1, s.blockManagerId.hostPort))) + blockLocationPairs.groupBy(_._1).map{case (k, v) => (k, v.unzip._2)}.toMap + } + /* Given a list of BlockStatus objets, returns information for each RDD */ def rddInfoFromBlockStatusList(infos: Map[String, BlockStatus], sc: SparkContext) : Array[RDDInfo] = { diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 0cb1e47ea5..428db6fa95 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -26,8 +26,14 @@ private[spark] class RDDPage(parent: BlockManagerUI) { val workers = filteredStorageStatusList.map((prefix, _)) val workerTable = listingTable(workerHeaders, workerRow, workers) - val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk") - val blocks = filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1) + val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk", + "Locations") + + val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1) + val blockLocations = StorageUtils.blockLocationsFromStorageStatus(filteredStorageStatusList) + val blocks = blockStatuses.map { + case(id, status) => (id, status, blockLocations.get(id).getOrElse(Seq("UNKNOWN"))) + } val blockTable = listingTable(blockHeaders, blockRow, blocks) val content = @@ -74,8 +80,8 @@ private[spark] class RDDPage(parent: BlockManagerUI) { headerSparkPage(content, parent.sc, "RDD Info: " + rddInfo.name, Jobs) } - def blockRow(blk: (String, BlockStatus)): Seq[Node] = { - val (id, block) = blk + def blockRow(row: (String, BlockStatus, Seq[String])): Seq[Node] = { + val (id, block, locations) = row {id} @@ -87,6 +93,9 @@ private[spark] class RDDPage(parent: BlockManagerUI) { {Utils.memoryBytesToString(block.diskSize)} + + {locations.map(l => {l}
    )} + } -- cgit v1.2.3 From 08150f19abcf08f2a18305080b08736fb8a33a12 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 12 Jul 2013 19:32:35 -0700 Subject: Minor style fix --- core/src/main/scala/spark/storage/StorageUtils.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index 3e7fa287e5..aca16f533a 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -47,8 +47,8 @@ object StorageUtils { /* Returns a map of blocks to their locations, compiled from a list of StorageStatus objects */ def blockLocationsFromStorageStatus(storageStatusList: Seq[StorageStatus]) = { - val blockLocationPairs = storageStatusList - .flatMap(s => s.blocks.map(b => (b._1, s.blockManagerId.hostPort))) + val blockLocationPairs = storageStatusList + .flatMap(s => s.blocks.map(b => (b._1, s.blockManagerId.hostPort))) blockLocationPairs.groupBy(_._1).map{case (k, v) => (k, v.unzip._2)}.toMap } -- cgit v1.2.3 From c4d5b01e44c5f289d67670185a14ff4ccd9537db Mon Sep 17 00:00:00 2001 From: seanm Date: Sat, 13 Jul 2013 14:56:23 -0600 Subject: changing com.google.code.findbugs maven coordinates --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 641d379749..0a4c176a29 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -144,7 +144,7 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "com.google.guava" % "guava" % "14.0.1", - "com.google.code.findbugs" % "jsr305" % "1.3.+", + "com.google.code.findbugs" % "jsr305" % "1.3.9", "log4j" % "log4j" % "1.2.16", "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, -- cgit v1.2.3 From 1889f9f450d3e28864a5ac7b5f9d3831cf7b17bf Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 13 Jul 2013 14:43:21 -0700 Subject: Increase PermGen size --- sbt/sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbt/sbt b/sbt/sbt index 850c58e1e9..523fbb346b 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -5,4 +5,4 @@ if [ "$MESOS_HOME" != "" ]; then fi export SPARK_HOME=$(cd "$(dirname $0)/.."; pwd) export SPARK_TESTING=1 # To put test classes on classpath -java -Xmx1200m -XX:MaxPermSize=250m -XX:ReservedCodeCacheSize=128m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" +java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=128m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" -- cgit v1.2.3 From ddb97f0fdf16442afaa9cab656376267e4044510 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 13 Jul 2013 16:05:21 -0700 Subject: Add `Environment` tab to SparkUI. This adds a tab which displays system property and classpath information. This can be useful in debugging various types of issues such as: 1. Extra/incorrect Hadoop jars being included in the classpath 2. Spark launching with a different JRE version than intended 3. Spark system properties not being set to intended values 4. User added jars that conflict with Spark jars --- core/src/main/scala/spark/ui/Page.scala | 2 +- core/src/main/scala/spark/ui/SparkUI.scala | 7 ++- core/src/main/scala/spark/ui/UIUtils.scala | 5 ++ .../main/scala/spark/ui/env/EnvironmentUI.scala | 58 ++++++++++++++++++++++ 4 files changed, 67 insertions(+), 5 deletions(-) create mode 100644 core/src/main/scala/spark/ui/env/EnvironmentUI.scala diff --git a/core/src/main/scala/spark/ui/Page.scala b/core/src/main/scala/spark/ui/Page.scala index c853b44b76..ed8f91842c 100644 --- a/core/src/main/scala/spark/ui/Page.scala +++ b/core/src/main/scala/spark/ui/Page.scala @@ -1,3 +1,3 @@ package spark.ui -private[spark] object Page extends Enumeration { val Storage, Jobs = Value } \ No newline at end of file +private[spark] object Page extends Enumeration { val Storage, Jobs, Environment = Value } \ No newline at end of file diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index b3bdc2c490..874e5ba8ec 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -5,15 +5,13 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} import spark.{Logging, SparkContext, Utils} +import spark.ui.env.EnvironmentUI import spark.ui.storage.BlockManagerUI import spark.ui.jobs.JobProgressUI -import spark.ui.UIUtils._ import spark.ui.JettyUtils._ /** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { - // TODO(pwendell): It would be nice to add a view that prints out environment information - val host = Utils.localHostName() val port = Option(System.getProperty("spark.ui.port")).getOrElse(SparkUI.DEFAULT_PORT).toInt var boundPort: Option[Int] = None @@ -25,7 +23,8 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { ) val storage = new BlockManagerUI(sc) val jobs = new JobProgressUI(sc) - val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ handlers + val env = new EnvironmentUI(sc) + val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ handlers /** Bind the HTTP server which backs this web interface */ def bind() { diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala index 7b79290d1b..36d9c47245 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -19,6 +19,10 @@ private[spark] object UIUtils { case Jobs =>
  • Jobs
  • case _ =>
  • Jobs
  • } + val environment = page match { + case Environment =>
  • Environment
  • + case _ =>
  • Environment
  • + } @@ -44,6 +48,7 @@ private[spark] object UIUtils {
    • Application: {sc.appName}
    • diff --git a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala new file mode 100644 index 0000000000..8a3b330405 --- /dev/null +++ b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala @@ -0,0 +1,58 @@ +package spark.ui.env + +import javax.servlet.http.HttpServletRequest + +import org.eclipse.jetty.server.Handler + +import scala.collection.JavaConversions._ + +import spark.ui.JettyUtils._ +import spark.ui.UIUtils.headerSparkPage +import spark.ui.Page.Environment +import spark.SparkContext +import spark.ui.UIUtils + +import scala.xml.Node + +private[spark] class EnvironmentUI(sc: SparkContext) { + + def getHandlers = Seq[(String, Handler)]( + ("/environment", (request: HttpServletRequest) => envDetails(request)) + ) + + def envDetails(request: HttpServletRequest): Seq[Node] = { + val properties = System.getProperties.iterator.toSeq + + val classPathProperty = properties + .filter{case (k, v) => k.contains("java.class.path")} + .headOption + .getOrElse("", "") + val sparkProperties = properties.filter(_._1.contains("spark")) + val otherProperties = properties.diff(sparkProperties :+ classPathProperty) + + val propertyHeaders = Seq("Name", "Value") + def propertyRow(kv: (String, String)) = {kv._1}{kv._2} + val propertyTable = UIUtils.listingTable( + propertyHeaders, propertyRow, sparkProperties ++ otherProperties) + + val classPathEntries = classPathProperty._2 + .split(System.getProperty("path.separator", ":")) + .filterNot(e => e.isEmpty) + .map(e => (e, "System Classpath")) + val addedJars = sc.addedJars.iterator.toSeq.map{case (path, time) => (path, "Added By User")} + val addedFiles = sc.addedFiles.iterator.toSeq.map{case (path, time) => (path, "Added By User")} + val classPath = addedJars ++ addedFiles ++ classPathEntries + + val classPathHeaders = Seq("Resource", "Source") + def classPathRow(data: (String, String)) = {data._1}{data._2} + val classPathTable = UIUtils.listingTable(classPathHeaders, classPathRow, classPath) + + val content = + +

      System Properties

      {propertyTable} +

      Classpath Entries

      {classPathTable} +
      + + headerSparkPage(content, sc, "Environment", Environment) + } +} -- cgit v1.2.3 From e271fde10b342216e33f4f45af73c5d103215cf2 Mon Sep 17 00:00:00 2001 From: root Date: Sun, 14 Jul 2013 06:24:29 +0000 Subject: Fixed a delay scheduling bug in the YARN branch, found by Patrick --- .../scala/spark/scheduler/cluster/ClusterTaskSetManager.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index fe6420a522..327d6797ae 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -454,10 +454,10 @@ private[spark] class ClusterTaskSetManager( val taskId = sched.newTaskId() // Figure out whether this should count as a preferred launch val taskLocality = - if (isProcessLocalLocation(task, hostPort)) TaskLocality.PROCESS_LOCAL else - if (isHostLocalLocation(task, hostPort)) TaskLocality.NODE_LOCAL else - if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL else - TaskLocality.ANY + if (isProcessLocalLocation(task, hostPort)) TaskLocality.PROCESS_LOCAL + else if (isHostLocalLocation(task, hostPort)) TaskLocality.NODE_LOCAL + else if (isRackLocalLocation(task, hostPort)) TaskLocality.RACK_LOCAL + else TaskLocality.ANY val prefStr = taskLocality.toString logInfo("Starting task %s:%d as TID %s on slave %s: %s (%s)".format( taskSet.id, index, taskId, execId, hostPort, prefStr)) @@ -467,7 +467,7 @@ private[spark] class ClusterTaskSetManager( val info = new TaskInfo(taskId, index, time, execId, hostPort, taskLocality) taskInfos(taskId) = info taskAttempts(index) = info :: taskAttempts(index) - if (TaskLocality.NODE_LOCAL == taskLocality) { + if (taskLocality == TaskLocality.PROCESS_LOCAL || taskLocality == TaskLocality.NODE_LOCAL) { lastPreferredLaunchTime = time } // Serialize and return the task -- cgit v1.2.3 From a44a7b123862202cf97f2de7a96aeaf29a93002a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 14 Jul 2013 07:23:09 +0000 Subject: Determine Spark core classes better in getCallSite --- core/src/main/scala/spark/Utils.scala | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index c6a3f97872..a36186bf8a 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -579,8 +579,15 @@ private object Utils extends Logging { output.toString } + /** + * A regular expression to match classes of the "core" Spark API that we want to skip when + * finding the call site of a method. + */ + private val SPARK_CLASS_REGEX = """^spark(\.api\.java)?(\.rdd)?\.[A-Z]""".r + private[spark] class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String, val firstUserLine: Int, val firstUserClass: String) + /** * When called inside a class in the spark package, returns the name of the user code class * (outside the spark package) that called into Spark, as well as which Spark method they called. @@ -602,7 +609,7 @@ private object Utils extends Logging { for (el <- trace) { if (!finished) { - if (el.getClassName.startsWith("spark.") && !el.getClassName.startsWith("spark.examples.")) { + if (SPARK_CLASS_REGEX.findFirstIn(el.getClassName) != None) { lastSparkMethod = if (el.getMethodName == "") { // Spark method is a constructor; get its class name el.getClassName.substring(el.getClassName.lastIndexOf('.') + 1) -- cgit v1.2.3 From b91a218cea5a7ab4037675667922fc06bfec6fbf Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 14 Jul 2013 07:31:33 +0000 Subject: Cosmetic fixes to web UI --- core/src/main/scala/spark/deploy/master/ui/IndexPage.scala | 4 ++-- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index c6de2bafa3..7545ecf868 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -38,8 +38,8 @@ private[spark] class IndexPage(parent: MasterWebUI) {
        -
      • URL:{state.uri}
      • -
      • Workers:{state.workers.size}
      • +
      • URL: {state.uri}
      • +
      • Workers: {state.workers.size}
      • Cores: {state.workers.map(_.cores).sum} Total, {state.workers.map(_.coresUsed).sum} Used
      • Memory: diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 1e675ab2cb..7907ab3bc7 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -29,10 +29,10 @@ private[spark] class IndexPage(parent: JobProgressUI) { Stage Id Origin Submitted - Duration - Tasks: Complete/Total - Shuffle Activity - Stored RDD + Duration + Tasks: Complete/Total + Shuffle Activity + Stored RDD {rows.map(r => makeRow(r))} -- cgit v1.2.3 From c5c38d1987137a1dc5eb66dd1065735a542ef9b5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 14 Jul 2013 07:59:50 +0000 Subject: Some optimizations to loading phase of ALS --- .../scala/spark/mllib/recommendation/ALS.scala | 45 +++++++++++++++------- 1 file changed, 32 insertions(+), 13 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index 6c9fb2359c..dec3701ec0 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -2,6 +2,7 @@ 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 @@ -33,6 +34,12 @@ 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]. + */ +private[recommendation] case class Rating(user: Int, product: Int, rating: Double) + + /** * Alternating Least Squares matrix factorization. * @@ -126,13 +133,13 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l * 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[(Int, Int, Double)]): OutLinkBlock = { - val userIds = ratings.map(_._1).distinct.sorted + 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 ((u, p, r) <- ratings) { - shouldSend(userIdToPos(u))(p % numBlocks) = true + for (r <- ratings) { + shouldSend(userIdToPos(r.user))(r.product % numBlocks) = true } OutLinkBlock(userIds, shouldSend) } @@ -141,18 +148,28 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l * 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[(Int, Int, Double)]): InLinkBlock = { - val userIds = ratings.map(_._1).distinct.sorted + 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) { - val ratingsInBlock = ratings.filter(t => t._2 % numBlocks == productBlock) - val ratingsByProduct = ratingsInBlock.groupBy(_._2) // (p, Seq[(u, p, r)]) - .toArray - .sortBy(_._1) - .map{case (p, rs) => (rs.map(t => userIdToPos(t._1)), rs.map(_._3))} - ratingsForBlock(productBlock) = ratingsByProduct + // Create an array of (product, Seq(Rating)) ratings + val groupedRatings = blockRatings(productBlock).groupBy(_.product).toArray + // Sort them by user 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) } @@ -167,7 +184,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l { val grouped = ratings.partitionBy(new HashPartitioner(numBlocks)) val links = grouped.mapPartitionsWithIndex((blockId, elements) => { - val ratings = elements.map(_._2).toArray + val ratings = elements.map{case (k, t) => Rating(t._1, t._2, t._3)}.toArray val inLinkBlock = makeInLinkBlock(numBlocks, ratings) val outLinkBlock = makeOutLinkBlock(numBlocks, ratings) Iterator.single((blockId, (inLinkBlock, outLinkBlock))) @@ -373,6 +390,8 @@ object ALS { } val (master, ratingsFile, rank, iters, outputDir) = (args(0), args(1), args(2).toInt, args(3).toInt, args(4)) + System.setProperty("spark.serializer", "spark.KryoSerializer") + System.setProperty("spark.locality.wait", "10000") val sc = new SparkContext(master, "ALS") val ratings = sc.textFile(ratingsFile).map { line => val fields = line.split(',') -- cgit v1.2.3 From 931e4c96ef56302cc384086d1faa0f4ca3258e8b Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 14 Jul 2013 08:03:13 +0000 Subject: Fix a comment --- mllib/src/main/scala/spark/mllib/recommendation/ALS.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index dec3701ec0..21eb21276e 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -161,7 +161,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l for (productBlock <- 0 until numBlocks) { // Create an array of (product, Seq(Rating)) ratings val groupedRatings = blockRatings(productBlock).groupBy(_.product).toArray - // Sort them by user ID + // 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 } -- cgit v1.2.3 From 00556a94c98577d1536d0c2b4606a9933069a584 Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Sun, 14 Jul 2013 17:04:53 +0800 Subject: add spaces before curly braces and after for if conditions --- .../scala/spark/deploy/yarn/ApplicationMaster.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index 68bb36d316..6a0617cc06 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -130,7 +130,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e val t = new Thread { override def run() { var successed = false - try{ + try { // Copy var mainArgs: Array[String] = new Array[String](args.userArgs.size()) args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size()) @@ -140,9 +140,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // It need shutdown hook to set SUCCEEDED successed = true } finally { - if(successed){ + if (successed) { ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED) - }else{ + } else { ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.FAILED) } } @@ -190,7 +190,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e logInfo("All workers have launched.") // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout - if (userThread.isAlive){ + if (userThread.isAlive) { // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse. val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000) @@ -208,7 +208,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e val t = new Thread { override def run() { - while (userThread.isAlive){ + while (userThread.isAlive) { val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning if (missingWorkerCount > 0) { logInfo("Allocating " + missingWorkerCount + " containers to make up for (potentially ?) lost containers") @@ -250,7 +250,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e def finishApplicationMaster(status: FinalApplicationStatus) { synchronized { - if(isFinished){ + if (isFinished) { return } isFinished = true @@ -276,7 +276,7 @@ object ApplicationMaster { private val ALLOCATOR_LOOP_WAIT_COUNT = 30 def incrementAllocatorLoop(by: Int) { val count = yarnAllocatorLoop.getAndAdd(by) - if (count >= ALLOCATOR_LOOP_WAIT_COUNT){ + if (count >= ALLOCATOR_LOOP_WAIT_COUNT) { yarnAllocatorLoop.synchronized { // to wake threads off wait ... yarnAllocatorLoop.notifyAll() @@ -320,7 +320,7 @@ object ApplicationMaster { // Wait for initialization to complete and atleast 'some' nodes can get allocated yarnAllocatorLoop.synchronized { - while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT){ + while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT) { yarnAllocatorLoop.wait(1000L) } } -- cgit v1.2.3 From 4883586838b960c405af208262aed3ec4b078613 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 14 Jul 2013 10:37:26 -0700 Subject: Responding to Matei's review --- .../src/main/scala/spark/ui/env/EnvironmentUI.scala | 21 ++++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala index 8a3b330405..6b8b9f05bb 100644 --- a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala @@ -5,6 +5,7 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler import scala.collection.JavaConversions._ +import scala.util.Properties import spark.ui.JettyUtils._ import spark.ui.UIUtils.headerSparkPage @@ -21,19 +22,27 @@ private[spark] class EnvironmentUI(sc: SparkContext) { ) def envDetails(request: HttpServletRequest): Seq[Node] = { - val properties = System.getProperties.iterator.toSeq + val jvmInformation = Seq( + ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), + ("Java Home", Properties.javaHome), + ("Scala Version", Properties.versionString), + ("Scala Home", Properties.scalaHome) + ) + def jvmRow(kv: (String, String)) = {kv._1}{kv._2} + def jvmTable = UIUtils.listingTable(Seq("Name", "Value"), jvmRow, jvmInformation) + val properties = System.getProperties.iterator.toSeq val classPathProperty = properties .filter{case (k, v) => k.contains("java.class.path")} .headOption .getOrElse("", "") - val sparkProperties = properties.filter(_._1.contains("spark")) + val sparkProperties = properties.filter(_._1.startsWith("spark")) val otherProperties = properties.diff(sparkProperties :+ classPathProperty) val propertyHeaders = Seq("Name", "Value") def propertyRow(kv: (String, String)) = {kv._1}{kv._2} - val propertyTable = UIUtils.listingTable( - propertyHeaders, propertyRow, sparkProperties ++ otherProperties) + val sparkPropertyTable = UIUtils.listingTable(propertyHeaders, propertyRow, sparkProperties) + val otherPropertyTable = UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties) val classPathEntries = classPathProperty._2 .split(System.getProperty("path.separator", ":")) @@ -49,7 +58,9 @@ private[spark] class EnvironmentUI(sc: SparkContext) { val content = -

        System Properties

        {propertyTable} +

        Runtime Information

        {jvmTable} +

        Spark Properties

        {sparkPropertyTable} +

        System Properties

        {otherPropertyTable}

        Classpath Entries

        {classPathTable}
        -- cgit v1.2.3 From ed7fd501cf7ece730cbdee6c152b917cf6bfb16a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 15 Jul 2013 00:30:10 +0000 Subject: Make number of blocks in ALS configurable and lower the default --- mllib/src/main/scala/spark/mllib/recommendation/ALS.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index 21eb21276e..2abaf2f2dd 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -91,7 +91,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l */ def train(ratings: RDD[(Int, Int, Double)]): MatrixFactorizationModel = { val numBlocks = if (this.numBlocks == -1) { - math.max(ratings.context.defaultParallelism, ratings.partitions.size) + math.max(ratings.context.defaultParallelism, ratings.partitions.size / 2) } else { this.numBlocks } @@ -384,12 +384,13 @@ object ALS { } def main(args: Array[String]) { - if (args.length != 5) { - println("Usage: ALS ") + 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.locality.wait", "10000") val sc = new SparkContext(master, "ALS") @@ -397,7 +398,7 @@ object ALS { val fields = line.split(',') (fields(0).toInt, fields(1).toInt, fields(2).toDouble) } - val model = ALS.train(ratings, rank, iters) + 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(" ") } -- cgit v1.2.3 From d47c16f78d5cb935bd4022c9bed8376691371682 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 15 Jul 2013 01:55:54 +0000 Subject: Add an option to disable reference tracking in Kryo --- core/src/main/scala/spark/KryoSerializer.scala | 4 ++++ docs/configuration.md | 12 +++++++++++- 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala index d723ab7b1e..c7dbcc6fbc 100644 --- a/core/src/main/scala/spark/KryoSerializer.scala +++ b/core/src/main/scala/spark/KryoSerializer.scala @@ -210,6 +210,10 @@ class KryoSerializer extends spark.serializer.Serializer with Logging { val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] reg.registerClasses(kryo) } + + // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops + kryo.setReferences(System.getProperty("spark.kryo.referenceTracking", "true").toBoolean) + kryo } diff --git a/docs/configuration.md b/docs/configuration.md index 5a80510959..5c06897cae 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -197,9 +197,19 @@ Apart from these, the following properties are also available, and may be useful (e.g. map functions) reference large objects in the driver program. + + spark.kryo.referenceTracking + true + + Whether to track references to the same object when serializing data with Kryo, which is + necessary if your object graphs have loops and useful for efficiency if they contain multiple + copies of the same object. Can be disabled to improve performance if you know this is not the + case. + + spark.kryoserializer.buffer.mb - 32 + 2 Maximum object size to allow within Kryo (the library needs to create a buffer at least as large as the largest single object you'll serialize). Increase this if you get a "buffer limit -- cgit v1.2.3 From 4698a0d6886905ef21cbd52e108d0dcab3df12df Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 15 Jul 2013 02:54:11 +0000 Subject: Shuffle ratings in a more efficient way at start of ALS --- .../main/scala/spark/mllib/recommendation/ALS.scala | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index 2abaf2f2dd..4c18cbdc6b 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -6,8 +6,10 @@ 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} @@ -98,8 +100,8 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l val partitioner = new HashPartitioner(numBlocks) - val ratingsByUserBlock = ratings.map{ case (u, p, r) => (u % numBlocks, (u, p, r)) } - val ratingsByProductBlock = ratings.map{ case (u, p, r) => (p % numBlocks, (p, u, r)) } + val ratingsByUserBlock = ratings.map{ case (u, p, r) => (u % numBlocks, Rating(u, p, r)) } + val ratingsByProductBlock = ratings.map{ case (u, p, r) => (p % numBlocks, Rating(p, u, r)) } val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock) val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) @@ -179,12 +181,12 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l * 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, (Int, Int, Double))]) + 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{case (k, t) => Rating(t._1, t._2, t._3)}.toArray + val ratings = elements.map{_._2}.toArray val inLinkBlock = makeInLinkBlock(numBlocks, ratings) val outLinkBlock = makeOutLinkBlock(numBlocks, ratings) Iterator.single((blockId, (inLinkBlock, outLinkBlock))) @@ -383,6 +385,12 @@ object ALS { 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 []") @@ -392,6 +400,8 @@ object ALS { (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.locality.wait", "10000") val sc = new SparkContext(master, "ALS") val ratings = sc.textFile(ratingsFile).map { line => -- cgit v1.2.3 From b2aaa1199e7ecd8e1b2a9ddd8356b6393edafe6b Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 11:44:42 -0700 Subject: Adds app name in HTML page titles on job web UI: fixes SPARK-806 --- core/src/main/scala/spark/ui/UIUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala index 36d9c47245..fa46e2487d 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -31,7 +31,7 @@ private[spark] object UIUtils { - {title} + {sc.appName} - {title} -- cgit v1.2.3 From 0d78b6d9cd11fc12c546f25fa857ba8b285c062d Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 13:42:12 -0700 Subject: Links to job UI from standalone deploy cluster web UI: fixes SPARK-802 --- core/src/main/scala/spark/deploy/master/Master.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 4dd6c448a9..87a7791fbd 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -80,6 +80,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act logInfo("Registering app " + description.name) val app = addApplication(description, sender) logInfo("Registered app " + description.name + " with ID " + app.id) + logInfo("Started App web UI at " + description.appUiUrl) waitingApps += app context.watch(sender) // This doesn't work with remote actors but helps for testing sender ! RegisteredApplication(app.id) -- cgit v1.2.3 From 39557112501901da7f9b4be6159a5a0be5511b42 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 15:47:21 -0700 Subject: Added field to master UI with link to job UI --- core/src/main/scala/spark/deploy/master/ui/IndexPage.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index 7545ecf868..434f600e8e 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -26,8 +26,8 @@ private[spark] class IndexPage(parent: MasterWebUI) { val workers = state.workers.sortBy(_.id) val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers) - val appHeaders = Seq("ID", "Description", "Cores", "Memory per Node", "Submit Time", "User", - "State", "Duration") + val appHeaders = Seq("ID", "Job UI", "Description", "Cores", "Memory per Node", "Submit Time", + "User", "State", "Duration") val activeApps = state.activeApps.sortBy(_.startTime).reverse val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps) val completedApps = state.completedApps.sortBy(_.endTime).reverse @@ -103,6 +103,9 @@ private[spark] class IndexPage(parent: MasterWebUI) { {app.id} + + {app.appUiUrl} + {app.desc.name} {app.coresGranted} -- cgit v1.2.3 From fbf5aa761e40649d0488a8673d488882c8bdae48 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 15:50:03 -0700 Subject: Removed log message, added field in master UI to link to log UI --- core/src/main/scala/spark/deploy/master/Master.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 87a7791fbd..4dd6c448a9 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -80,7 +80,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act logInfo("Registering app " + description.name) val app = addApplication(description, sender) logInfo("Registered app " + description.name + " with ID " + app.id) - logInfo("Started App web UI at " + description.appUiUrl) waitingApps += app context.watch(sender) // This doesn't work with remote actors but helps for testing sender ! RegisteredApplication(app.id) -- cgit v1.2.3 From 6dc7c9bfb17cefdf162e86c8b52a4ffc8b59efaf Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 16:33:50 -0700 Subject: Removed job UI column, linked description to job UI --- core/src/main/scala/spark/deploy/master/ui/IndexPage.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index 434f600e8e..5e3c5e064f 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -26,8 +26,8 @@ private[spark] class IndexPage(parent: MasterWebUI) { val workers = state.workers.sortBy(_.id) val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers) - val appHeaders = Seq("ID", "Job UI", "Description", "Cores", "Memory per Node", "Submit Time", - "User", "State", "Duration") + val appHeaders = Seq("ID", "Description", "Cores", "Memory per Node", "Submit Time", "User", + "State", "Duration") val activeApps = state.activeApps.sortBy(_.startTime).reverse val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps) val completedApps = state.completedApps.sortBy(_.endTime).reverse @@ -104,9 +104,8 @@ private[spark] class IndexPage(parent: MasterWebUI) { {app.id} - {app.appUiUrl} + {app.desc.name} - {app.desc.name} {app.coresGranted} -- cgit v1.2.3 From a96b4ef761aa80310b194176f41a088c5bf6274a Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 15 Jul 2013 19:13:17 -0600 Subject: dding tgz option to make-distribution.sh --- make-distribution.sh | 24 +++++++++++++++++++++--- 1 file changed, 21 insertions(+), 3 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index feb13d52f9..ef3d2529d0 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Script to create a binary distribution for easy deploys of Spark. # The distribution directory defaults to dist/ but can be overridden below. @@ -6,6 +6,10 @@ # so it is completely self contained. # It does not contain source or *.class files. # +# Arguments +# (none): Creates dist/ directory +# tgz: Additionally creates spark-$VERSION-bin.tar.gz +# # Recommended deploy/testing procedure (standalone mode): # 1) Rsync / deploy the dist/ dir to one host # 2) cd to deploy dir; ./bin/start-master.sh @@ -19,8 +23,14 @@ DISTDIR="$FWDIR/dist" # Get version from SBT export TERM=dumb # Prevents color codes in SBT output -VERSION=$($FWDIR/sbt/sbt "show version" | tail -1 | cut -f 2) -echo "Making distribution for Spark $VERSION in $DISTDIR..." +VERSION=$($FWDIR/sbt/sbt "show version" | tail -1 | cut -f 2 | sed 's/^\([a-zA-Z0-9.-]*\).*/\1/') + +if [ "$1" == "tgz" ]; then + echo "Making spark-$VERSION-bin.tar.gz" +else + echo "Making distribution for Spark $VERSION in $DISTDIR..." +fi + # Build fat JAR $FWDIR/sbt/sbt "repl/assembly" @@ -37,3 +47,11 @@ cp $FWDIR/repl/target/*.jar "$DISTDIR/jars/" cp -r "$FWDIR/bin" "$DISTDIR" cp -r "$FWDIR/conf" "$DISTDIR" cp "$FWDIR/run" "$FWDIR/spark-shell" "$DISTDIR" + + +if [ "$1" == "tgz" ]; then + TARDIR="$FWDIR/spark-$VERSION" + cp -r $DISTDIR $TARDIR + tar -zcf spark-$VERSION-bin.tar.gz -C $FWDIR spark-$VERSION + rm -rf $TARDIR +fi -- cgit v1.2.3 From 90b0142985ca2c6c76ded1a5c073774308c1a727 Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 15 Jul 2013 19:13:39 -0600 Subject: adding files generated from make-distribution.sh to .gitignore --- .gitignore | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.gitignore b/.gitignore index ae39c52b11..00fbff6a2c 100644 --- a/.gitignore +++ b/.gitignore @@ -38,3 +38,5 @@ dependency-reduced-pom.xml .ensime_lucene checkpoint derby.log +dist/ +spark-*-bin.tar.gz -- cgit v1.2.3 From 69316603d6bf11ecf1ea3dab63df178bad835e2d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 15 Jul 2013 22:50:11 -0700 Subject: Throw a more meaningful message when runJob is called to launch tasks on non-existent partitions. --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 9 +++++++++ core/src/test/scala/spark/RDDSuite.scala | 6 ++++++ 2 files changed, 15 insertions(+) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 3d3b9ea011..8173ef709d 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -251,6 +251,15 @@ class DAGScheduler( if (partitions.size == 0) { return } + + // Check to make sure we are not launching a task on a partition that does not exist. + val maxPartitions = finalRdd.partitions.length + partitions.find(p => p >= maxPartitions).foreach { p => + throw new IllegalArgumentException( + "Attempting to access a non-existent partition: " + p + ". " + + "Total number of partitions: " + maxPartitions) + } + val (toSubmit, waiter) = prepareJob( finalRdd, func, partitions, callSite, allowLocal, resultHandler, properties) eventQueue.put(toSubmit) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index aa3ee5f5ee..7f7d4c8211 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -302,4 +302,10 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements") } } + + test("runJob on an invalid partition") { + intercept[IllegalArgumentException] { + sc.runJob(sc.parallelize(1 to 10, 2), {iter: Iterator[Int] => iter.size}, Seq(0, 1, 2), false) + } + } } -- cgit v1.2.3 From 2748e73eb9bb3f08b116c58ea404a6e76f9cbff9 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Jul 2013 16:08:46 +0530 Subject: Dependency upgrade Akka 2.0.3 -> 2.0.5 --- project/SparkBuild.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 178a056de8..2510a614e8 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -153,9 +153,9 @@ object SparkBuild extends Build { "org.ow2.asm" % "asm" % "4.0", "com.google.protobuf" % "protobuf-java" % "2.4.1", "de.javakaffee" % "kryo-serializers" % "0.22", - "com.typesafe.akka" % "akka-actor" % "2.0.3" excludeAll(excludeNetty), - "com.typesafe.akka" % "akka-remote" % "2.0.3" excludeAll(excludeNetty), - "com.typesafe.akka" % "akka-slf4j" % "2.0.3" excludeAll(excludeNetty), + "com.typesafe.akka" % "akka-actor" % "2.0.5" excludeAll(excludeNetty), + "com.typesafe.akka" % "akka-remote" % "2.0.5" excludeAll(excludeNetty), + "com.typesafe.akka" % "akka-slf4j" % "2.0.5" excludeAll(excludeNetty), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "net.liftweb" % "lift-json_2.9.2" % "2.5", @@ -235,7 +235,7 @@ object SparkBuild extends Build { "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty), "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty), - "com.typesafe.akka" % "akka-zeromq" % "2.0.3" excludeAll(excludeNetty) + "com.typesafe.akka" % "akka-zeromq" % "2.0.5" excludeAll(excludeNetty) ) ) ++ assemblySettings ++ extraAssemblySettings -- cgit v1.2.3 From f347cc3f659d4414a21de26feadcbe23a130e622 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 10:53:30 -0700 Subject: Fix deprecation warning and style issues --- core/src/test/scala/spark/FileServerSuite.scala | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/spark/FileServerSuite.scala index 9c24ca430d..c7855a7bd3 100644 --- a/core/src/test/scala/spark/FileServerSuite.scala +++ b/core/src/test/scala/spark/FileServerSuite.scala @@ -35,26 +35,26 @@ class FileServerSuite extends FunSuite with LocalSparkContext { val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { val path = SparkFiles.get("FileServerSuite.txt") - val in = new BufferedReader(new FileReader(path)) + val in = new BufferedReader(new FileReader(path)) val fileVal = in.readLine().toInt in.close() _ * fileVal + _ * fileVal - }.collect + }.collect() assert(result.toSet === Set((1,200), (2,300), (3,500))) } test("Distributing files locally using URL as input") { // addFile("file:///....") sc = new SparkContext("local[4]", "test") - sc.addFile((new File(tmpFile.toString)).toURL.toString) + sc.addFile(new File(tmpFile.toString).toURI.toString) val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { val path = SparkFiles.get("FileServerSuite.txt") - val in = new BufferedReader(new FileReader(path)) + val in = new BufferedReader(new FileReader(path)) val fileVal = in.readLine().toInt in.close() _ * fileVal + _ * fileVal - }.collect + }.collect() assert(result.toSet === Set((1,200), (2,300), (3,500))) } @@ -80,11 +80,11 @@ class FileServerSuite extends FunSuite with LocalSparkContext { val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { val path = SparkFiles.get("FileServerSuite.txt") - val in = new BufferedReader(new FileReader(path)) + val in = new BufferedReader(new FileReader(path)) val fileVal = in.readLine().toInt in.close() _ * fileVal + _ * fileVal - }.collect + }.collect() assert(result.toSet === Set((1,200), (2,300), (3,500))) } -- cgit v1.2.3 From 5c388808a81ebfb1fc23511882c18f9ae76ec509 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 11:01:14 -0700 Subject: SPARK-814: Result stages should be named after action --- .../main/scala/spark/scheduler/DAGScheduler.scala | 22 ++++++++++++++-------- core/src/main/scala/spark/scheduler/Stage.scala | 5 +++-- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 2 +- .../scala/spark/scheduler/JobLoggerSuite.scala | 4 ++-- 4 files changed, 20 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 3d3b9ea011..30a648f50b 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -150,7 +150,13 @@ class DAGScheduler( * as a result stage for the final RDD used directly in an action. The stage will also be given * the provided priority. */ - private def newStage(rdd: RDD[_], shuffleDep: Option[ShuffleDependency[_,_]], priority: Int): Stage = { + private def newStage( + rdd: RDD[_], + shuffleDep: Option[ShuffleDependency[_,_]], + priority: Int, + callSite: Option[String] = None) + : Stage = + { if (shuffleDep != None) { // Kind of ugly: need to register RDDs with the cache and map output tracker here // since we can't do it in the RDD constructor because # of partitions is unknown @@ -158,7 +164,7 @@ class DAGScheduler( mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.partitions.size) } val id = nextStageId.getAndIncrement() - val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority), priority) + val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority), priority, callSite) idToStage(id) = stage stageToInfos(stage) = StageInfo(stage) stage @@ -286,12 +292,12 @@ class DAGScheduler( event match { case JobSubmitted(finalRDD, func, partitions, allowLocal, callSite, listener, properties) => val runId = nextRunId.getAndIncrement() - val finalStage = newStage(finalRDD, None, runId) + val finalStage = newStage(finalRDD, None, runId, Some(callSite)) val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener, properties) clearCacheLocs() logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length + " output partitions (allowLocal=" + allowLocal + ")") - logInfo("Final stage: " + finalStage + " (" + finalStage.origin + ")") + logInfo("Final stage: " + finalStage + " (" + finalStage.name + ")") logInfo("Parents of final stage: " + finalStage.parents) logInfo("Missing parents: " + getMissingParentStages(finalStage)) if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { @@ -502,7 +508,7 @@ class DAGScheduler( case Some(t) => "%.03f".format((System.currentTimeMillis() - t) / 1000.0) case _ => "Unkown" } - logInfo("%s (%s) finished in %s s".format(stage, stage.origin, serviceTime)) + logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) stage.completionTime = Some(System.currentTimeMillis) val stageComp = StageCompleted(stageToInfos(stage)) sparkListeners.foreach{_.onStageCompleted(stageComp)} @@ -568,7 +574,7 @@ class DAGScheduler( if (stage.outputLocs.count(_ == Nil) != 0) { // Some tasks had failed; let's resubmit this stage // TODO: Lower-level scheduler should also deal with this - logInfo("Resubmitting " + stage + " (" + stage.origin + + logInfo("Resubmitting " + stage + " (" + stage.name + ") because some of its tasks had failed: " + stage.outputLocs.zipWithIndex.filter(_._1 == Nil).map(_._2).mkString(", ")) submitStage(stage) @@ -600,7 +606,7 @@ class DAGScheduler( running -= failedStage failed += failedStage // TODO: Cancel running tasks in the stage - logInfo("Marking " + failedStage + " (" + failedStage.origin + + logInfo("Marking " + failedStage + " (" + failedStage.name + ") for resubmision due to a fetch failure") // Mark the map whose fetch failed as broken in the map stage val mapStage = shuffleToMapStage(shuffleId) @@ -608,7 +614,7 @@ class DAGScheduler( mapStage.removeOutputLoc(mapId, bmAddress) mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) } - logInfo("The failed fetch was from " + mapStage + " (" + mapStage.origin + + logInfo("The failed fetch was from " + mapStage + " (" + mapStage.name + "); marking it for resubmission") failed += mapStage // Remember that a fetch failed now; this is used to resubmit the broken diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala index 539cf8233b..4937eb3b88 100644 --- a/core/src/main/scala/spark/scheduler/Stage.scala +++ b/core/src/main/scala/spark/scheduler/Stage.scala @@ -24,7 +24,8 @@ private[spark] class Stage( val rdd: RDD[_], val shuffleDep: Option[ShuffleDependency[_,_]], // Output shuffle if stage is a map stage val parents: List[Stage], - val priority: Int) + val priority: Int, + callSite: Option[String]) extends Logging { val isShuffleMap = shuffleDep != None @@ -85,7 +86,7 @@ private[spark] class Stage( return id } - def origin: String = rdd.origin + val name = callSite.getOrElse(rdd.origin) override def toString = "Stage " + id diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 7907ab3bc7..2df5f0192b 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -89,7 +89,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { {s.id} - {s.origin} + {s.name} {submissionTime} {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))} diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala index 699901f1a1..2474d744b8 100644 --- a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala @@ -37,8 +37,8 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers val parentRdd = makeRdd(4, Nil) val shuffleDep = new ShuffleDependency(parentRdd, null) val rootRdd = makeRdd(4, List(shuffleDep)) - val shuffleMapStage = new Stage(1, parentRdd, Some(shuffleDep), Nil, jobID) - val rootStage = new Stage(0, rootRdd, None, List(shuffleMapStage), jobID) + val shuffleMapStage = new Stage(1, parentRdd, Some(shuffleDep), Nil, jobID, None) + val rootStage = new Stage(0, rootRdd, None, List(shuffleMapStage), jobID, None) joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStage, 4)) joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getName) -- cgit v1.2.3 From 00a14deb6d9b6d89e0ddbe1540636622b1dc4d16 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 11:52:20 -0700 Subject: Update to latest Scala Maven plugin and allow Zinc external compiler --- pom.xml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 63dd80b5fe..6fa1f1d4f1 100644 --- a/pom.xml +++ b/pom.xml @@ -319,7 +319,7 @@ net.alchim31.maven scala-maven-plugin - 3.1.0 + 3.1.5 scala-compile-first @@ -346,12 +346,14 @@ ${scala.version} incremental + true -unchecked -optimise -Xms64m + -Xms1024m -Xmx1024m -XX:PermSize=${PermGen} -XX:MaxPermSize=${MaxPermGen} -- cgit v1.2.3 From af3c9d50424602f3e5af1055e83e9badef0a1632 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 17:21:33 -0700 Subject: Add Apache license headers and LICENSE and NOTICE files --- LICENSE | 229 ++++++++++++++++++--- NOTICE | 5 + bagel/pom.xml | 17 ++ bagel/src/main/scala/spark/bagel/Bagel.scala | 17 ++ .../scala/spark/bagel/examples/PageRankUtils.scala | 17 ++ .../spark/bagel/examples/WikipediaPageRank.scala | 17 ++ .../examples/WikipediaPageRankStandalone.scala | 17 ++ bagel/src/test/resources/log4j.properties | 19 +- bagel/src/test/scala/bagel/BagelSuite.scala | 17 ++ bin/slaves.sh | 5 +- bin/spark-config.sh | 19 +- bin/spark-daemon.sh | 5 +- bin/spark-daemons.sh | 17 ++ bin/start-all.sh | 17 ++ bin/start-master.sh | 17 ++ bin/start-slave.sh | 17 ++ bin/start-slaves.sh | 17 ++ bin/stop-all.sh | 17 ++ bin/stop-master.sh | 17 ++ bin/stop-slaves.sh | 17 ++ core/pom.xml | 17 ++ .../apache/hadoop/mapred/HadoopMapRedUtil.scala | 17 ++ .../hadoop/mapreduce/HadoopMapReduceUtil.scala | 17 ++ .../scala/spark/deploy/SparkHadoopUtil.scala | 17 ++ .../apache/hadoop/mapred/HadoopMapRedUtil.scala | 17 ++ .../hadoop/mapreduce/HadoopMapReduceUtil.scala | 17 ++ .../scala/spark/deploy/SparkHadoopUtil.scala | 17 ++ .../spark/deploy/yarn/ApplicationMaster.scala | 17 ++ .../deploy/yarn/ApplicationMasterArguments.scala | 17 ++ .../scala/spark/deploy/yarn/Client.scala | 17 ++ .../scala/spark/deploy/yarn/ClientArguments.scala | 17 ++ .../scala/spark/deploy/yarn/WorkerRunnable.scala | 17 ++ .../spark/deploy/yarn/YarnAllocationHandler.scala | 17 ++ .../scheduler/cluster/YarnClusterScheduler.scala | 17 ++ .../apache/hadoop/mapred/HadoopMapRedUtil.scala | 17 ++ .../hadoop/mapreduce/HadoopMapReduceUtil.scala | 17 ++ .../scala/spark/deploy/SparkHadoopUtil.scala | 17 ++ .../main/java/spark/network/netty/FileClient.java | 17 ++ .../netty/FileClientChannelInitializer.java | 17 ++ .../spark/network/netty/FileClientHandler.java | 17 ++ .../main/java/spark/network/netty/FileServer.java | 17 ++ .../netty/FileServerChannelInitializer.java | 17 ++ .../spark/network/netty/FileServerHandler.java | 17 ++ .../java/spark/network/netty/PathResolver.java | 17 ++ core/src/main/scala/spark/Accumulators.scala | 17 ++ core/src/main/scala/spark/Aggregator.scala | 17 ++ .../scala/spark/BlockStoreShuffleFetcher.scala | 17 ++ core/src/main/scala/spark/Cache.scala | 17 ++ core/src/main/scala/spark/CacheManager.scala | 17 ++ core/src/main/scala/spark/ClosureCleaner.scala | 17 ++ core/src/main/scala/spark/Dependency.scala | 17 ++ core/src/main/scala/spark/DoubleRDDFunctions.scala | 17 ++ .../main/scala/spark/FetchFailedException.scala | 17 ++ core/src/main/scala/spark/HadoopWriter.scala | 17 ++ core/src/main/scala/spark/HttpFileServer.scala | 19 +- core/src/main/scala/spark/HttpServer.scala | 17 ++ core/src/main/scala/spark/JavaSerializer.scala | 17 ++ core/src/main/scala/spark/KryoSerializer.scala | 17 ++ core/src/main/scala/spark/Logging.scala | 17 ++ core/src/main/scala/spark/MapOutputTracker.scala | 17 ++ core/src/main/scala/spark/PairRDDFunctions.scala | 17 ++ core/src/main/scala/spark/Partition.scala | 17 ++ core/src/main/scala/spark/Partitioner.scala | 17 ++ core/src/main/scala/spark/RDD.scala | 17 ++ core/src/main/scala/spark/RDDCheckpointData.scala | 17 ++ .../scala/spark/SequenceFileRDDFunctions.scala | 17 ++ .../main/scala/spark/SerializableWritable.scala | 17 ++ core/src/main/scala/spark/ShuffleFetcher.scala | 17 ++ core/src/main/scala/spark/SizeEstimator.scala | 17 ++ core/src/main/scala/spark/SoftReferenceCache.scala | 17 ++ core/src/main/scala/spark/SparkContext.scala | 17 ++ core/src/main/scala/spark/SparkEnv.scala | 17 ++ core/src/main/scala/spark/SparkException.scala | 17 ++ core/src/main/scala/spark/SparkFiles.java | 17 ++ core/src/main/scala/spark/TaskContext.scala | 17 ++ core/src/main/scala/spark/TaskEndReason.scala | 17 ++ core/src/main/scala/spark/TaskState.scala | 17 ++ core/src/main/scala/spark/Utils.scala | 17 ++ .../main/scala/spark/api/java/JavaDoubleRDD.scala | 17 ++ .../main/scala/spark/api/java/JavaPairRDD.scala | 17 ++ core/src/main/scala/spark/api/java/JavaRDD.scala | 17 ++ .../main/scala/spark/api/java/JavaRDDLike.scala | 17 ++ .../scala/spark/api/java/JavaSparkContext.scala | 17 ++ .../java/JavaSparkContextVarargsWorkaround.java | 17 ++ .../main/scala/spark/api/java/StorageLevels.java | 17 ++ .../api/java/function/DoubleFlatMapFunction.java | 17 ++ .../spark/api/java/function/DoubleFunction.java | 17 ++ .../spark/api/java/function/FlatMapFunction.scala | 17 ++ .../spark/api/java/function/FlatMapFunction2.scala | 17 ++ .../scala/spark/api/java/function/Function.java | 17 ++ .../scala/spark/api/java/function/Function2.java | 17 ++ .../api/java/function/PairFlatMapFunction.java | 17 ++ .../spark/api/java/function/PairFunction.java | 17 ++ .../spark/api/java/function/VoidFunction.scala | 19 +- .../spark/api/java/function/WrappedFunction1.scala | 17 ++ .../spark/api/java/function/WrappedFunction2.scala | 17 ++ .../scala/spark/api/python/PythonPartitioner.scala | 17 ++ .../main/scala/spark/api/python/PythonRDD.scala | 17 ++ .../spark/api/python/PythonWorkerFactory.scala | 17 ++ .../spark/broadcast/BitTorrentBroadcast.scala | 17 ++ .../src/main/scala/spark/broadcast/Broadcast.scala | 17 ++ .../scala/spark/broadcast/BroadcastFactory.scala | 17 ++ .../main/scala/spark/broadcast/HttpBroadcast.scala | 17 ++ .../main/scala/spark/broadcast/MultiTracker.scala | 17 ++ .../main/scala/spark/broadcast/SourceInfo.scala | 17 ++ .../main/scala/spark/broadcast/TreeBroadcast.scala | 17 ++ .../spark/deploy/ApplicationDescription.scala | 17 ++ core/src/main/scala/spark/deploy/Command.scala | 17 ++ .../main/scala/spark/deploy/DeployMessage.scala | 17 ++ .../main/scala/spark/deploy/ExecutorState.scala | 17 ++ .../src/main/scala/spark/deploy/JsonProtocol.scala | 19 +- .../scala/spark/deploy/LocalSparkCluster.scala | 17 ++ core/src/main/scala/spark/deploy/WebUI.scala | 17 ++ .../main/scala/spark/deploy/client/Client.scala | 17 ++ .../scala/spark/deploy/client/ClientListener.scala | 17 ++ .../scala/spark/deploy/client/TestClient.scala | 17 ++ .../scala/spark/deploy/client/TestExecutor.scala | 17 ++ .../spark/deploy/master/ApplicationInfo.scala | 17 ++ .../spark/deploy/master/ApplicationState.scala | 17 ++ .../scala/spark/deploy/master/ExecutorInfo.scala | 17 ++ .../main/scala/spark/deploy/master/Master.scala | 17 ++ .../spark/deploy/master/MasterArguments.scala | 17 ++ .../scala/spark/deploy/master/WorkerInfo.scala | 17 ++ .../scala/spark/deploy/master/WorkerState.scala | 17 ++ .../spark/deploy/master/ui/ApplicationPage.scala | 17 ++ .../scala/spark/deploy/master/ui/IndexPage.scala | 17 ++ .../scala/spark/deploy/master/ui/MasterWebUI.scala | 19 +- .../scala/spark/deploy/worker/ExecutorRunner.scala | 17 ++ .../main/scala/spark/deploy/worker/Worker.scala | 17 ++ .../spark/deploy/worker/WorkerArguments.scala | 17 ++ .../scala/spark/deploy/worker/ui/IndexPage.scala | 17 ++ .../scala/spark/deploy/worker/ui/WorkerWebUI.scala | 17 ++ core/src/main/scala/spark/executor/Executor.scala | 17 ++ .../scala/spark/executor/ExecutorBackend.scala | 17 ++ .../scala/spark/executor/ExecutorExitCode.scala | 17 ++ .../spark/executor/ExecutorURLClassLoader.scala | 17 ++ .../spark/executor/MesosExecutorBackend.scala | 17 ++ .../spark/executor/StandaloneExecutorBackend.scala | 17 ++ .../main/scala/spark/executor/TaskMetrics.scala | 17 ++ .../main/scala/spark/network/BufferMessage.scala | 19 +- core/src/main/scala/spark/network/Connection.scala | 17 ++ .../scala/spark/network/ConnectionManager.scala | 17 ++ .../scala/spark/network/ConnectionManagerId.scala | 17 ++ .../spark/network/ConnectionManagerTest.scala | 17 ++ core/src/main/scala/spark/network/Message.scala | 17 ++ .../main/scala/spark/network/MessageChunk.scala | 17 ++ .../scala/spark/network/MessageChunkHeader.scala | 17 ++ .../main/scala/spark/network/ReceiverTest.scala | 17 ++ core/src/main/scala/spark/network/SenderTest.scala | 17 ++ .../scala/spark/network/netty/FileHeader.scala | 17 ++ .../scala/spark/network/netty/ShuffleCopier.scala | 17 ++ .../scala/spark/network/netty/ShuffleSender.scala | 17 ++ core/src/main/scala/spark/package.scala | 17 ++ .../spark/partial/ApproximateActionListener.scala | 17 ++ .../scala/spark/partial/ApproximateEvaluator.scala | 17 ++ .../main/scala/spark/partial/BoundedDouble.scala | 17 ++ .../main/scala/spark/partial/CountEvaluator.scala | 17 ++ .../spark/partial/GroupedCountEvaluator.scala | 17 ++ .../scala/spark/partial/GroupedMeanEvaluator.scala | 17 ++ .../scala/spark/partial/GroupedSumEvaluator.scala | 17 ++ .../main/scala/spark/partial/MeanEvaluator.scala | 17 ++ .../main/scala/spark/partial/PartialResult.scala | 17 ++ .../main/scala/spark/partial/StudentTCacher.scala | 17 ++ .../main/scala/spark/partial/SumEvaluator.scala | 17 ++ core/src/main/scala/spark/rdd/BlockRDD.scala | 17 ++ core/src/main/scala/spark/rdd/CartesianRDD.scala | 17 ++ core/src/main/scala/spark/rdd/CheckpointRDD.scala | 17 ++ core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 17 ++ core/src/main/scala/spark/rdd/CoalescedRDD.scala | 17 ++ core/src/main/scala/spark/rdd/EmptyRDD.scala | 17 ++ core/src/main/scala/spark/rdd/FilteredRDD.scala | 17 ++ core/src/main/scala/spark/rdd/FlatMappedRDD.scala | 17 ++ core/src/main/scala/spark/rdd/GlommedRDD.scala | 17 ++ core/src/main/scala/spark/rdd/HadoopRDD.scala | 17 ++ core/src/main/scala/spark/rdd/JdbcRDD.scala | 17 ++ .../main/scala/spark/rdd/MapPartitionsRDD.scala | 17 ++ .../spark/rdd/MapPartitionsWithIndexRDD.scala | 17 ++ core/src/main/scala/spark/rdd/MappedRDD.scala | 17 ++ core/src/main/scala/spark/rdd/NewHadoopRDD.scala | 17 ++ .../scala/spark/rdd/ParallelCollectionRDD.scala | 17 ++ .../main/scala/spark/rdd/PartitionPruningRDD.scala | 17 ++ core/src/main/scala/spark/rdd/PipedRDD.scala | 17 ++ core/src/main/scala/spark/rdd/SampledRDD.scala | 17 ++ core/src/main/scala/spark/rdd/ShuffledRDD.scala | 17 ++ core/src/main/scala/spark/rdd/SubtractedRDD.scala | 17 ++ core/src/main/scala/spark/rdd/UnionRDD.scala | 17 ++ .../main/scala/spark/rdd/ZippedPartitionsRDD.scala | 17 ++ core/src/main/scala/spark/rdd/ZippedRDD.scala | 17 ++ .../src/main/scala/spark/scheduler/ActiveJob.scala | 17 ++ .../main/scala/spark/scheduler/DAGScheduler.scala | 17 ++ .../scala/spark/scheduler/DAGSchedulerEvent.scala | 17 ++ .../scala/spark/scheduler/InputFormatInfo.scala | 17 ++ .../main/scala/spark/scheduler/JobListener.scala | 17 ++ .../src/main/scala/spark/scheduler/JobLogger.scala | 17 ++ .../src/main/scala/spark/scheduler/JobResult.scala | 17 ++ .../src/main/scala/spark/scheduler/JobWaiter.scala | 17 ++ .../src/main/scala/spark/scheduler/MapStatus.scala | 17 ++ .../main/scala/spark/scheduler/ResultTask.scala | 17 ++ .../scala/spark/scheduler/ShuffleMapTask.scala | 17 ++ .../main/scala/spark/scheduler/SparkListener.scala | 17 ++ .../src/main/scala/spark/scheduler/SplitInfo.scala | 17 ++ core/src/main/scala/spark/scheduler/Stage.scala | 17 ++ .../src/main/scala/spark/scheduler/StageInfo.scala | 19 +- core/src/main/scala/spark/scheduler/Task.scala | 17 ++ .../main/scala/spark/scheduler/TaskResult.scala | 17 ++ .../main/scala/spark/scheduler/TaskScheduler.scala | 17 ++ .../spark/scheduler/TaskSchedulerListener.scala | 17 ++ core/src/main/scala/spark/scheduler/TaskSet.scala | 17 ++ .../spark/scheduler/cluster/ClusterScheduler.scala | 17 ++ .../scheduler/cluster/ClusterTaskSetManager.scala | 17 ++ .../scheduler/cluster/ExecutorLossReason.scala | 17 ++ .../main/scala/spark/scheduler/cluster/Pool.scala | 17 ++ .../spark/scheduler/cluster/Schedulable.scala | 17 ++ .../scheduler/cluster/SchedulableBuilder.scala | 17 ++ .../spark/scheduler/cluster/SchedulerBackend.scala | 17 ++ .../scheduler/cluster/SchedulingAlgorithm.scala | 17 ++ .../spark/scheduler/cluster/SchedulingMode.scala | 17 ++ .../cluster/SparkDeploySchedulerBackend.scala | 17 ++ .../cluster/StandaloneClusterMessage.scala | 17 ++ .../cluster/StandaloneSchedulerBackend.scala | 17 ++ .../spark/scheduler/cluster/TaskDescription.scala | 17 ++ .../scala/spark/scheduler/cluster/TaskInfo.scala | 17 ++ .../spark/scheduler/cluster/TaskSetManager.scala | 17 ++ .../spark/scheduler/cluster/WorkerOffer.scala | 17 ++ .../spark/scheduler/local/LocalScheduler.scala | 17 ++ .../scheduler/local/LocalTaskSetManager.scala | 17 ++ .../mesos/CoarseMesosSchedulerBackend.scala | 17 ++ .../scheduler/mesos/MesosSchedulerBackend.scala | 17 ++ .../main/scala/spark/serializer/Serializer.scala | 17 ++ .../scala/spark/serializer/SerializerManager.scala | 17 ++ .../main/scala/spark/storage/BlockException.scala | 17 ++ .../scala/spark/storage/BlockFetchTracker.scala | 17 ++ .../scala/spark/storage/BlockFetcherIterator.scala | 17 ++ .../main/scala/spark/storage/BlockManager.scala | 17 ++ .../main/scala/spark/storage/BlockManagerId.scala | 17 ++ .../scala/spark/storage/BlockManagerMaster.scala | 17 ++ .../spark/storage/BlockManagerMasterActor.scala | 17 ++ .../scala/spark/storage/BlockManagerMessages.scala | 17 ++ .../spark/storage/BlockManagerSlaveActor.scala | 17 ++ .../scala/spark/storage/BlockManagerWorker.scala | 17 ++ .../main/scala/spark/storage/BlockMessage.scala | 17 ++ .../scala/spark/storage/BlockMessageArray.scala | 17 ++ .../scala/spark/storage/BlockObjectWriter.scala | 17 ++ core/src/main/scala/spark/storage/BlockStore.scala | 17 ++ core/src/main/scala/spark/storage/DiskStore.scala | 17 ++ .../src/main/scala/spark/storage/MemoryStore.scala | 17 ++ core/src/main/scala/spark/storage/PutResult.scala | 17 ++ .../scala/spark/storage/ShuffleBlockManager.scala | 17 ++ .../main/scala/spark/storage/StorageLevel.scala | 17 ++ .../main/scala/spark/storage/StorageUtils.scala | 17 ++ .../main/scala/spark/storage/ThreadingTest.scala | 17 ++ core/src/main/scala/spark/ui/JettyUtils.scala | 17 ++ core/src/main/scala/spark/ui/Page.scala | 19 +- core/src/main/scala/spark/ui/SparkUI.scala | 17 ++ core/src/main/scala/spark/ui/UIUtils.scala | 19 +- .../main/scala/spark/ui/UIWorkloadGenerator.scala | 17 ++ .../main/scala/spark/ui/env/EnvironmentUI.scala | 17 ++ core/src/main/scala/spark/ui/jobs/IndexPage.scala | 17 ++ .../main/scala/spark/ui/jobs/JobProgressUI.scala | 19 +- core/src/main/scala/spark/ui/jobs/StagePage.scala | 17 ++ .../scala/spark/ui/storage/BlockManagerUI.scala | 17 ++ .../main/scala/spark/ui/storage/IndexPage.scala | 17 ++ core/src/main/scala/spark/ui/storage/RDDPage.scala | 17 ++ core/src/main/scala/spark/util/AkkaUtils.scala | 17 ++ .../scala/spark/util/BoundedPriorityQueue.scala | 17 ++ .../scala/spark/util/ByteBufferInputStream.scala | 17 ++ .../main/scala/spark/util/CompletionIterator.scala | 19 +- core/src/main/scala/spark/util/Distribution.scala | 19 +- core/src/main/scala/spark/util/IdGenerator.scala | 17 ++ core/src/main/scala/spark/util/IntParam.scala | 17 ++ core/src/main/scala/spark/util/MemoryParam.scala | 17 ++ .../main/scala/spark/util/MetadataCleaner.scala | 17 ++ core/src/main/scala/spark/util/NextIterator.scala | 19 +- .../scala/spark/util/RateLimitedOutputStream.scala | 17 ++ .../main/scala/spark/util/SerializableBuffer.scala | 17 ++ core/src/main/scala/spark/util/StatCounter.scala | 17 ++ .../main/scala/spark/util/TimeStampedHashMap.scala | 17 ++ .../main/scala/spark/util/TimeStampedHashSet.scala | 17 ++ core/src/main/scala/spark/util/Vector.scala | 17 ++ core/src/test/resources/fairscheduler.xml | 17 ++ core/src/test/resources/log4j.properties | 19 +- core/src/test/scala/spark/AccumulatorSuite.scala | 17 ++ core/src/test/scala/spark/BroadcastSuite.scala | 17 ++ core/src/test/scala/spark/CheckpointSuite.scala | 17 ++ .../src/test/scala/spark/ClosureCleanerSuite.scala | 17 ++ core/src/test/scala/spark/DistributedSuite.scala | 17 ++ core/src/test/scala/spark/DriverSuite.scala | 17 ++ core/src/test/scala/spark/FailureSuite.scala | 17 ++ core/src/test/scala/spark/FileServerSuite.scala | 17 ++ core/src/test/scala/spark/FileSuite.scala | 17 ++ core/src/test/scala/spark/JavaAPISuite.java | 17 ++ .../src/test/scala/spark/KryoSerializerSuite.scala | 17 ++ core/src/test/scala/spark/LocalSparkContext.scala | 17 ++ .../test/scala/spark/MapOutputTrackerSuite.scala | 17 ++ .../test/scala/spark/PairRDDFunctionsSuite.scala | 17 ++ core/src/test/scala/spark/PartitioningSuite.scala | 17 ++ core/src/test/scala/spark/PipedRDDSuite.scala | 17 ++ core/src/test/scala/spark/RDDSuite.scala | 17 ++ core/src/test/scala/spark/SharedSparkContext.scala | 17 ++ core/src/test/scala/spark/ShuffleNettySuite.scala | 17 ++ core/src/test/scala/spark/ShuffleSuite.scala | 17 ++ core/src/test/scala/spark/SizeEstimatorSuite.scala | 17 ++ core/src/test/scala/spark/SortingSuite.scala | 17 ++ core/src/test/scala/spark/ThreadingSuite.scala | 17 ++ core/src/test/scala/spark/UnpersistSuite.scala | 17 ++ core/src/test/scala/spark/UtilsSuite.scala | 17 ++ .../test/scala/spark/ZippedPartitionsSuite.scala | 17 ++ core/src/test/scala/spark/rdd/JdbcRDDSuite.scala | 17 ++ .../spark/rdd/ParallelCollectionSplitSuite.scala | 17 ++ .../spark/scheduler/ClusterSchedulerSuite.scala | 17 ++ .../scala/spark/scheduler/DAGSchedulerSuite.scala | 17 ++ .../scala/spark/scheduler/JobLoggerSuite.scala | 17 ++ .../spark/scheduler/LocalSchedulerSuite.scala | 17 ++ .../scala/spark/scheduler/SparkListenerSuite.scala | 17 ++ .../scala/spark/scheduler/TaskContextSuite.scala | 17 ++ .../scala/spark/storage/BlockManagerSuite.scala | 17 ++ core/src/test/scala/spark/ui/UISuite.scala | 17 ++ .../test/scala/spark/util/DistributionSuite.scala | 17 ++ .../test/scala/spark/util/NextIteratorSuite.scala | 17 ++ .../spark/util/RateLimitedOutputStreamSuite.scala | 17 ++ docs/_plugins/copy_api_dirs.rb | 17 ++ ec2/spark-ec2 | 2 + ec2/spark_ec2.py | 2 + examples/pom.xml | 17 ++ .../src/main/java/spark/examples/JavaHdfsLR.java | 17 ++ .../src/main/java/spark/examples/JavaKMeans.java | 17 ++ .../src/main/java/spark/examples/JavaLogQuery.java | 17 ++ .../src/main/java/spark/examples/JavaSparkPi.java | 17 ++ examples/src/main/java/spark/examples/JavaTC.java | 17 ++ .../main/java/spark/examples/JavaWordCount.java | 17 ++ .../streaming/examples/JavaFlumeEventCount.java | 17 ++ .../streaming/examples/JavaNetworkWordCount.java | 17 ++ .../spark/streaming/examples/JavaQueueStream.java | 17 ++ .../main/scala/spark/examples/BroadcastTest.scala | 17 ++ .../main/scala/spark/examples/CassandraTest.scala | 17 ++ .../spark/examples/ExceptionHandlingTest.scala | 17 ++ .../main/scala/spark/examples/GroupByTest.scala | 17 ++ .../src/main/scala/spark/examples/HBaseTest.scala | 19 +- .../src/main/scala/spark/examples/HdfsTest.scala | 17 ++ .../src/main/scala/spark/examples/LocalALS.scala | 17 ++ .../main/scala/spark/examples/LocalFileLR.scala | 17 ++ .../main/scala/spark/examples/LocalKMeans.scala | 17 ++ .../src/main/scala/spark/examples/LocalLR.scala | 17 ++ .../src/main/scala/spark/examples/LocalPi.scala | 17 ++ .../src/main/scala/spark/examples/LogQuery.scala | 17 ++ .../scala/spark/examples/MultiBroadcastTest.scala | 17 ++ .../spark/examples/SimpleSkewedGroupByTest.scala | 17 ++ .../scala/spark/examples/SkewedGroupByTest.scala | 17 ++ .../src/main/scala/spark/examples/SparkALS.scala | 17 ++ .../main/scala/spark/examples/SparkHdfsLR.scala | 17 ++ .../main/scala/spark/examples/SparkKMeans.scala | 17 ++ .../src/main/scala/spark/examples/SparkLR.scala | 17 ++ .../src/main/scala/spark/examples/SparkPi.scala | 17 ++ .../src/main/scala/spark/examples/SparkTC.scala | 17 ++ .../spark/streaming/examples/ActorWordCount.scala | 17 ++ .../spark/streaming/examples/FlumeEventCount.scala | 17 ++ .../spark/streaming/examples/HdfsWordCount.scala | 17 ++ .../spark/streaming/examples/KafkaWordCount.scala | 17 ++ .../streaming/examples/NetworkWordCount.scala | 17 ++ .../spark/streaming/examples/QueueStream.scala | 17 ++ .../spark/streaming/examples/RawNetworkGrep.scala | 17 ++ .../examples/StatefulNetworkWordCount.scala | 17 ++ .../streaming/examples/TwitterAlgebirdCMS.scala | 17 ++ .../streaming/examples/TwitterAlgebirdHLL.scala | 17 ++ .../streaming/examples/TwitterPopularTags.scala | 17 ++ .../spark/streaming/examples/ZeroMQWordCount.scala | 17 ++ .../examples/clickstream/PageViewGenerator.scala | 17 ++ .../examples/clickstream/PageViewStream.scala | 17 ++ make-distribution.sh | 19 ++ .../main/scala/spark/mllib/clustering/KMeans.scala | 17 ++ .../scala/spark/mllib/clustering/KMeansModel.scala | 17 ++ .../scala/spark/mllib/clustering/LocalKMeans.scala | 17 ++ .../scala/spark/mllib/optimization/Gradient.scala | 17 ++ .../spark/mllib/optimization/GradientDescent.scala | 17 ++ .../scala/spark/mllib/optimization/Updater.scala | 17 ++ .../scala/spark/mllib/recommendation/ALS.scala | 17 ++ .../recommendation/MatrixFactorizationModel.scala | 17 ++ .../mllib/regression/LogisticRegression.scala | 17 ++ .../regression/LogisticRegressionGenerator.scala | 17 ++ .../scala/spark/mllib/regression/Regression.scala | 17 ++ .../spark/mllib/regression/RidgeRegression.scala | 17 ++ .../regression/RidgeRegressionGenerator.scala | 17 ++ .../src/main/scala/spark/mllib/util/MLUtils.scala | 17 ++ mllib/src/test/resources/log4j.properties | 19 +- .../scala/spark/mllib/clustering/KMeansSuite.scala | 17 ++ .../spark/mllib/recommendation/ALSSuite.scala | 17 ++ .../mllib/regression/LogisticRegressionSuite.scala | 17 ++ .../mllib/regression/RidgeRegressionSuite.scala | 17 ++ pom.xml | 17 ++ project/SparkBuild.scala | 16 ++ project/build.properties | 17 ++ pyspark | 17 ++ python/epydoc.conf | 17 ++ python/examples/als.py | 17 ++ python/examples/kmeans.py | 17 ++ python/examples/logistic_regression.py | 17 ++ python/examples/pi.py | 17 ++ python/examples/transitive_closure.py | 17 ++ python/examples/wordcount.py | 17 ++ python/pyspark/accumulators.py | 17 ++ python/pyspark/broadcast.py | 17 ++ python/pyspark/context.py | 17 ++ python/pyspark/daemon.py | 17 ++ python/pyspark/files.py | 17 ++ python/pyspark/java_gateway.py | 17 ++ python/pyspark/rdd.py | 17 ++ python/pyspark/serializers.py | 17 ++ python/pyspark/shell.py | 17 ++ python/pyspark/tests.py | 17 ++ python/pyspark/worker.py | 17 ++ python/run-tests | 20 +- repl-bin/pom.xml | 17 ++ repl-bin/src/deb/bin/run | 19 +- repl-bin/src/deb/bin/spark-executor | 17 ++ repl-bin/src/deb/bin/spark-shell | 17 ++ repl/pom.xml | 17 ++ .../scala/spark/repl/ExecutorClassLoader.scala | 17 ++ repl/src/main/scala/spark/repl/Main.scala | 17 ++ repl/src/test/resources/log4j.properties | 19 +- repl/src/test/scala/spark/repl/ReplSuite.scala | 17 ++ run | 17 ++ run.cmd | 18 ++ run2.cmd | 17 ++ sbt/sbt | 20 ++ sbt/sbt.cmd | 20 ++ spark-executor | 18 ++ spark-shell | 19 ++ spark-shell.cmd | 18 ++ streaming/pom.xml | 17 ++ .../main/scala/spark/streaming/Checkpoint.scala | 17 ++ .../src/main/scala/spark/streaming/DStream.scala | 17 ++ .../spark/streaming/DStreamCheckpointData.scala | 17 ++ .../main/scala/spark/streaming/DStreamGraph.scala | 17 ++ .../src/main/scala/spark/streaming/Duration.scala | 17 ++ .../src/main/scala/spark/streaming/Interval.scala | 17 ++ streaming/src/main/scala/spark/streaming/Job.scala | 17 ++ .../main/scala/spark/streaming/JobManager.scala | 17 ++ .../spark/streaming/NetworkInputTracker.scala | 17 ++ .../spark/streaming/PairDStreamFunctions.scala | 17 ++ .../src/main/scala/spark/streaming/Scheduler.scala | 17 ++ .../scala/spark/streaming/StreamingContext.scala | 17 ++ .../src/main/scala/spark/streaming/Time.scala | 19 +- .../spark/streaming/api/java/JavaDStream.scala | 19 +- .../spark/streaming/api/java/JavaDStreamLike.scala | 19 +- .../spark/streaming/api/java/JavaPairDStream.scala | 17 ++ .../streaming/api/java/JavaStreamingContext.scala | 17 ++ .../spark/streaming/dstream/CoGroupedDStream.scala | 17 ++ .../streaming/dstream/ConstantInputDStream.scala | 19 +- .../spark/streaming/dstream/FileInputDStream.scala | 17 ++ .../spark/streaming/dstream/FilteredDStream.scala | 17 ++ .../streaming/dstream/FlatMapValuedDStream.scala | 17 ++ .../streaming/dstream/FlatMappedDStream.scala | 17 ++ .../streaming/dstream/FlumeInputDStream.scala | 17 ++ .../spark/streaming/dstream/ForEachDStream.scala | 17 ++ .../spark/streaming/dstream/GlommedDStream.scala | 17 ++ .../spark/streaming/dstream/InputDStream.scala | 17 ++ .../streaming/dstream/KafkaInputDStream.scala | 17 ++ .../streaming/dstream/MapPartitionedDStream.scala | 17 ++ .../spark/streaming/dstream/MapValuedDStream.scala | 17 ++ .../spark/streaming/dstream/MappedDStream.scala | 17 ++ .../streaming/dstream/NetworkInputDStream.scala | 17 ++ .../streaming/dstream/PluggableInputDStream.scala | 17 ++ .../streaming/dstream/QueueInputDStream.scala | 17 ++ .../spark/streaming/dstream/RawInputDStream.scala | 17 ++ .../streaming/dstream/ReducedWindowedDStream.scala | 17 ++ .../spark/streaming/dstream/ShuffledDStream.scala | 17 ++ .../streaming/dstream/SocketInputDStream.scala | 17 ++ .../spark/streaming/dstream/StateDStream.scala | 17 ++ .../streaming/dstream/TransformedDStream.scala | 17 ++ .../streaming/dstream/TwitterInputDStream.scala | 17 ++ .../spark/streaming/dstream/UnionDStream.scala | 17 ++ .../spark/streaming/dstream/WindowedDStream.scala | 17 ++ .../spark/streaming/receivers/ActorReceiver.scala | 17 ++ .../spark/streaming/receivers/ZeroMQReceiver.scala | 17 ++ .../main/scala/spark/streaming/util/Clock.scala | 17 ++ .../spark/streaming/util/MasterFailureTest.scala | 17 ++ .../scala/spark/streaming/util/RawTextHelper.scala | 17 ++ .../scala/spark/streaming/util/RawTextSender.scala | 17 ++ .../spark/streaming/util/RecurringTimer.scala | 17 ++ .../test/java/spark/streaming/JavaAPISuite.java | 17 ++ .../test/java/spark/streaming/JavaTestUtils.scala | 19 +- streaming/src/test/resources/log4j.properties | 19 +- .../spark/streaming/BasicOperationsSuite.scala | 17 ++ .../scala/spark/streaming/CheckpointSuite.scala | 17 ++ .../test/scala/spark/streaming/FailureSuite.scala | 17 ++ .../scala/spark/streaming/InputStreamsSuite.scala | 17 ++ .../test/scala/spark/streaming/TestSuiteBase.scala | 17 ++ .../spark/streaming/WindowOperationsSuite.scala | 17 ++ 488 files changed, 8448 insertions(+), 59 deletions(-) create mode 100644 NOTICE diff --git a/LICENSE b/LICENSE index d17afa1fc6..d645695673 100644 --- a/LICENSE +++ b/LICENSE @@ -1,27 +1,202 @@ -Copyright (c) 2010, Regents of the University of California. -All rights reserved. - -Redistribution and use in source and binary forms, with or without -modification, are permitted provided that the following conditions -are met: - * Redistributions of source code must retain the above copyright - notice, this list of conditions and the following disclaimer. - * Redistributions in binary form must reproduce the above copyright - notice, this list of conditions and the following disclaimer in the - documentation and/or other materials provided with the distribution. - * Neither the name of the University of California, Berkeley nor the - names of its contributors may be used to endorse or promote - products derived from this software without specific prior written - permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED -TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR -PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF -LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING -NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/NOTICE b/NOTICE new file mode 100644 index 0000000000..7cbb114b2a --- /dev/null +++ b/NOTICE @@ -0,0 +1,5 @@ +Apache Spark +Copyright 2013 The Apache Software Foundation. + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). diff --git a/bagel/pom.xml b/bagel/pom.xml index b83a0ef6c0..60bbc49e6c 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 diff --git a/bagel/src/main/scala/spark/bagel/Bagel.scala b/bagel/src/main/scala/spark/bagel/Bagel.scala index 5ecdd7d004..80c8d53d2b 100644 --- a/bagel/src/main/scala/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/spark/bagel/Bagel.scala @@ -1,3 +1,20 @@ +/* + * 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.bagel import spark._ diff --git a/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala b/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala index b97d786ed4..de65e27fe0 100644 --- a/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala +++ b/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala @@ -1,3 +1,20 @@ +/* + * 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.bagel.examples import spark._ diff --git a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala index bc32663e0f..a0c5ac9c18 100644 --- a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala +++ b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala @@ -1,3 +1,20 @@ +/* + * 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.bagel.examples import spark._ diff --git a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala index 9d9d80d809..3c54a85f42 100644 --- a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala +++ b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala @@ -1,3 +1,20 @@ +/* + * 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.bagel.examples import spark._ diff --git a/bagel/src/test/resources/log4j.properties b/bagel/src/test/resources/log4j.properties index 83d05cab2f..5cdcf35b23 100644 --- a/bagel/src/test/resources/log4j.properties +++ b/bagel/src/test/resources/log4j.properties @@ -1,4 +1,21 @@ -# Set everything to be logged to the file bagel/target/unit-tests.log +# +# 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. +# + +# Set everything to be logged to the file bagel/target/unit-tests.log log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false diff --git a/bagel/src/test/scala/bagel/BagelSuite.scala b/bagel/src/test/scala/bagel/BagelSuite.scala index a09c978068..ef2d57fbd0 100644 --- a/bagel/src/test/scala/bagel/BagelSuite.scala +++ b/bagel/src/test/scala/bagel/BagelSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.bagel import org.scalatest.{FunSuite, Assertions, BeforeAndAfter} diff --git a/bin/slaves.sh b/bin/slaves.sh index 030581e058..c8fb5ca473 100755 --- a/bin/slaves.sh +++ b/bin/slaves.sh @@ -1,7 +1,5 @@ #!/usr/bin/env bash -# This Spark deploy script is a modified version of the Apache Hadoop deploy -# script, available under the Apache 2 license: # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -10,13 +8,14 @@ # (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 +# 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. +# # Run a shell command on all slave hosts. # diff --git a/bin/spark-config.sh b/bin/spark-config.sh index d4b6558866..cd2c7b7b0d 100755 --- a/bin/spark-config.sh +++ b/bin/spark-config.sh @@ -1,3 +1,20 @@ +# +# 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. +# + # included in all the spark scripts with source command # should not be executable directly # also should not be passed any arguments, since we need original $* @@ -16,4 +33,4 @@ this="$config_bin/$script" export SPARK_PREFIX=`dirname "$this"`/.. export SPARK_HOME=${SPARK_PREFIX} -export SPARK_CONF_DIR="$SPARK_HOME/conf" \ No newline at end of file +export SPARK_CONF_DIR="$SPARK_HOME/conf" diff --git a/bin/spark-daemon.sh b/bin/spark-daemon.sh index 8ee3ec481f..a5b88ca785 100755 --- a/bin/spark-daemon.sh +++ b/bin/spark-daemon.sh @@ -1,7 +1,5 @@ #!/usr/bin/env bash -# This Spark deploy script is a modified version of the Apache Hadoop deploy -# script, available under the Apache 2 license: # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -10,13 +8,14 @@ # (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 +# 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. +# # Runs a Spark command as a daemon. # diff --git a/bin/spark-daemons.sh b/bin/spark-daemons.sh index 0619097e4d..354eb905a1 100755 --- a/bin/spark-daemons.sh +++ b/bin/spark-daemons.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# 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. +# + # Run a Spark command on all slave hosts. usage="Usage: spark-daemons.sh [--config confdir] [--hosts hostlistfile] [start|stop] command instance-number args..." diff --git a/bin/start-all.sh b/bin/start-all.sh index b9891ad2f6..0182f1ab24 100755 --- a/bin/start-all.sh +++ b/bin/start-all.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# 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. +# + # Start all spark daemons. # Starts the master on this node. # Starts a worker on each node specified in conf/slaves diff --git a/bin/start-master.sh b/bin/start-master.sh index 83a3e1f3dc..2288fb19d7 100755 --- a/bin/start-master.sh +++ b/bin/start-master.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# 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. +# + # Starts the master on the machine this script is executed on. bin=`dirname "$0"` diff --git a/bin/start-slave.sh b/bin/start-slave.sh index 1082c09eb1..d6db16882d 100755 --- a/bin/start-slave.sh +++ b/bin/start-slave.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash + +# +# 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. +# + # Usage: start-slave.sh # where is like "spark://localhost:7077" diff --git a/bin/start-slaves.sh b/bin/start-slaves.sh index 4e05224190..dad7c3df76 100755 --- a/bin/start-slaves.sh +++ b/bin/start-slaves.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# 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. +# + bin=`dirname "$0"` bin=`cd "$bin"; pwd` diff --git a/bin/stop-all.sh b/bin/stop-all.sh index d352f6f631..a043ac0095 100755 --- a/bin/stop-all.sh +++ b/bin/stop-all.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# 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. +# + # Start all spark daemons. # Run this on the master nde diff --git a/bin/stop-master.sh b/bin/stop-master.sh index 172ee5891d..31a610bf9d 100755 --- a/bin/stop-master.sh +++ b/bin/stop-master.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# 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. +# + # Starts the master on the machine this script is executed on. bin=`dirname "$0"` diff --git a/bin/stop-slaves.sh b/bin/stop-slaves.sh index fbfc594472..8e056f23d4 100755 --- a/bin/stop-slaves.sh +++ b/bin/stop-slaves.sh @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# 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. +# + # Starts the master on the machine this script is executed on. bin=`dirname "$0"` diff --git a/core/pom.xml b/core/pom.xml index dbb2da9a9c..6329b2fbd8 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 diff --git a/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala index f286f2cf9c..25386b2796 100644 --- a/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala +++ b/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala @@ -1,3 +1,20 @@ +/* + * 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.hadoop.mapred trait HadoopMapRedUtil { diff --git a/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala index 264d421d14..b1002e0cac 100644 --- a/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala +++ b/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala @@ -1,3 +1,20 @@ +/* + * 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.hadoop.mapreduce import org.apache.hadoop.conf.Configuration diff --git a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala index f1c86de4cc..df55be1254 100644 --- a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.JobConf diff --git a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala index 875c0a220b..0f972b7a0b 100644 --- a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala +++ b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala @@ -1,4 +1,21 @@ +/* + * 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.hadoop.mapred import org.apache.hadoop.mapreduce.TaskType diff --git a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala index 8bc6fb6dea..1a7cdf4788 100644 --- a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala +++ b/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala @@ -1,3 +1,20 @@ +/* + * 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.hadoop.mapreduce import org.apache.hadoop.conf.Configuration diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala index 301a57fffa..6122fdced0 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy import collection.mutable.HashMap diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala index 6a0617cc06..1b06169739 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.yarn import java.net.Socket diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala index 1b00208511..8de44b1f66 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.yarn import spark.util.IntParam diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala index 514c17f241..8bcbfc2735 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.yarn import java.net.{InetSocketAddress, URI} diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala index 07e7edea36..67aff03781 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.yarn import spark.util.MemoryParam diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala index cc6f3344a1..f458f2f6a1 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.yarn import java.net.URI diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala index 61dd72a651..b0af8baf08 100644 --- a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.yarn import spark.{Logging, Utils} diff --git a/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala b/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala index ed732d36bf..307d96111c 100644 --- a/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala +++ b/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.cluster import spark._ diff --git a/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala index a0652d7fc7..4b3d84670c 100644 --- a/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala +++ b/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala @@ -1,3 +1,20 @@ +/* + * 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.hadoop.mapred trait HadoopMapRedUtil { diff --git a/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala index 7fdbe322fd..aa3b1ed3a5 100644 --- a/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala +++ b/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala @@ -1,3 +1,20 @@ +/* + * 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.hadoop.mapreduce import org.apache.hadoop.conf.Configuration diff --git a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala index f1c86de4cc..df55be1254 100644 --- a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.JobConf diff --git a/core/src/main/java/spark/network/netty/FileClient.java b/core/src/main/java/spark/network/netty/FileClient.java index a4bb4bc701..0625a6d502 100644 --- a/core/src/main/java/spark/network/netty/FileClient.java +++ b/core/src/main/java/spark/network/netty/FileClient.java @@ -1,3 +1,20 @@ +/* + * 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.network.netty; import io.netty.bootstrap.Bootstrap; diff --git a/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java b/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java index af25baf641..05ad4b61d7 100644 --- a/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java +++ b/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java @@ -1,3 +1,20 @@ +/* + * 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.network.netty; import io.netty.buffer.BufType; diff --git a/core/src/main/java/spark/network/netty/FileClientHandler.java b/core/src/main/java/spark/network/netty/FileClientHandler.java index 9fc9449827..e8cd9801f6 100644 --- a/core/src/main/java/spark/network/netty/FileClientHandler.java +++ b/core/src/main/java/spark/network/netty/FileClientHandler.java @@ -1,3 +1,20 @@ +/* + * 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.network.netty; import io.netty.buffer.ByteBuf; diff --git a/core/src/main/java/spark/network/netty/FileServer.java b/core/src/main/java/spark/network/netty/FileServer.java index dd3a557ae5..9f009a61d5 100644 --- a/core/src/main/java/spark/network/netty/FileServer.java +++ b/core/src/main/java/spark/network/netty/FileServer.java @@ -1,3 +1,20 @@ +/* + * 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.network.netty; import java.net.InetSocketAddress; diff --git a/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java b/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java index 8f1f5c65cd..50c57a81a3 100644 --- a/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java +++ b/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java @@ -1,3 +1,20 @@ +/* + * 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.network.netty; import io.netty.channel.ChannelInitializer; diff --git a/core/src/main/java/spark/network/netty/FileServerHandler.java b/core/src/main/java/spark/network/netty/FileServerHandler.java index a78eddb1b5..176ba8da49 100644 --- a/core/src/main/java/spark/network/netty/FileServerHandler.java +++ b/core/src/main/java/spark/network/netty/FileServerHandler.java @@ -1,3 +1,20 @@ +/* + * 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.network.netty; import java.io.File; diff --git a/core/src/main/java/spark/network/netty/PathResolver.java b/core/src/main/java/spark/network/netty/PathResolver.java index 302411672c..f446c55b19 100755 --- a/core/src/main/java/spark/network/netty/PathResolver.java +++ b/core/src/main/java/spark/network/netty/PathResolver.java @@ -1,3 +1,20 @@ +/* + * 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.network.netty; diff --git a/core/src/main/scala/spark/Accumulators.scala b/core/src/main/scala/spark/Accumulators.scala index 57c6df35be..6ff92ce833 100644 --- a/core/src/main/scala/spark/Accumulators.scala +++ b/core/src/main/scala/spark/Accumulators.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.io._ diff --git a/core/src/main/scala/spark/Aggregator.scala b/core/src/main/scala/spark/Aggregator.scala index df8ce9c054..136b4da61e 100644 --- a/core/src/main/scala/spark/Aggregator.scala +++ b/core/src/main/scala/spark/Aggregator.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.util.{HashMap => JHashMap} diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index 3239f4c385..8f6953b1f5 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -1,3 +1,20 @@ +/* + * 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 import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/Cache.scala b/core/src/main/scala/spark/Cache.scala index 20d677a854..b0c83ce59d 100644 --- a/core/src/main/scala/spark/Cache.scala +++ b/core/src/main/scala/spark/Cache.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.util.concurrent.atomic.AtomicInteger diff --git a/core/src/main/scala/spark/CacheManager.scala b/core/src/main/scala/spark/CacheManager.scala index f7a2b7e802..81314805a9 100644 --- a/core/src/main/scala/spark/CacheManager.scala +++ b/core/src/main/scala/spark/CacheManager.scala @@ -1,3 +1,20 @@ +/* + * 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 import scala.collection.mutable.{ArrayBuffer, HashSet} diff --git a/core/src/main/scala/spark/ClosureCleaner.scala b/core/src/main/scala/spark/ClosureCleaner.scala index d5e7132ff9..8b39241095 100644 --- a/core/src/main/scala/spark/ClosureCleaner.scala +++ b/core/src/main/scala/spark/ClosureCleaner.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.lang.reflect.Field diff --git a/core/src/main/scala/spark/Dependency.scala b/core/src/main/scala/spark/Dependency.scala index 2af44aa383..d17e70a4fa 100644 --- a/core/src/main/scala/spark/Dependency.scala +++ b/core/src/main/scala/spark/Dependency.scala @@ -1,3 +1,20 @@ +/* + * 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 /** diff --git a/core/src/main/scala/spark/DoubleRDDFunctions.scala b/core/src/main/scala/spark/DoubleRDDFunctions.scala index 178d31a73b..93ef097702 100644 --- a/core/src/main/scala/spark/DoubleRDDFunctions.scala +++ b/core/src/main/scala/spark/DoubleRDDFunctions.scala @@ -1,3 +1,20 @@ +/* + * 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 import spark.partial.BoundedDouble diff --git a/core/src/main/scala/spark/FetchFailedException.scala b/core/src/main/scala/spark/FetchFailedException.scala index 40b0193f19..a2dae6cae9 100644 --- a/core/src/main/scala/spark/FetchFailedException.scala +++ b/core/src/main/scala/spark/FetchFailedException.scala @@ -1,3 +1,20 @@ +/* + * 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 import spark.storage.BlockManagerId diff --git a/core/src/main/scala/spark/HadoopWriter.scala b/core/src/main/scala/spark/HadoopWriter.scala index 5e8396edb9..b1fe0075a3 100644 --- a/core/src/main/scala/spark/HadoopWriter.scala +++ b/core/src/main/scala/spark/HadoopWriter.scala @@ -1,3 +1,20 @@ +/* + * 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.hadoop.mapred import org.apache.hadoop.fs.FileSystem diff --git a/core/src/main/scala/spark/HttpFileServer.scala b/core/src/main/scala/spark/HttpFileServer.scala index 00901d95e2..a13a7a2859 100644 --- a/core/src/main/scala/spark/HttpFileServer.scala +++ b/core/src/main/scala/spark/HttpFileServer.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.io.{File} @@ -42,4 +59,4 @@ private[spark] class HttpFileServer extends Logging { return dir + "/" + file.getName } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/HttpServer.scala b/core/src/main/scala/spark/HttpServer.scala index 4e0507c080..c9dffbc631 100644 --- a/core/src/main/scala/spark/HttpServer.scala +++ b/core/src/main/scala/spark/HttpServer.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.io.File diff --git a/core/src/main/scala/spark/JavaSerializer.scala b/core/src/main/scala/spark/JavaSerializer.scala index b04a27d073..04c5f44e6b 100644 --- a/core/src/main/scala/spark/JavaSerializer.scala +++ b/core/src/main/scala/spark/JavaSerializer.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.io._ diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala index c7dbcc6fbc..ee37da7948 100644 --- a/core/src/main/scala/spark/KryoSerializer.scala +++ b/core/src/main/scala/spark/KryoSerializer.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.io._ diff --git a/core/src/main/scala/spark/Logging.scala b/core/src/main/scala/spark/Logging.scala index 0fc8c31463..79b0362830 100644 --- a/core/src/main/scala/spark/Logging.scala +++ b/core/src/main/scala/spark/Logging.scala @@ -1,3 +1,20 @@ +/* + * 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 import org.slf4j.Logger diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala index fde597ffd1..2c417e31db 100644 --- a/core/src/main/scala/spark/MapOutputTracker.scala +++ b/core/src/main/scala/spark/MapOutputTracker.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.io._ diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 8b313c645f..6b0cc2fbf1 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/Partition.scala b/core/src/main/scala/spark/Partition.scala index e384308ef6..2a4edcec98 100644 --- a/core/src/main/scala/spark/Partition.scala +++ b/core/src/main/scala/spark/Partition.scala @@ -1,3 +1,20 @@ +/* + * 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 /** diff --git a/core/src/main/scala/spark/Partitioner.scala b/core/src/main/scala/spark/Partitioner.scala index 6f8cd17c88..660af70d52 100644 --- a/core/src/main/scala/spark/Partitioner.scala +++ b/core/src/main/scala/spark/Partitioner.scala @@ -1,3 +1,20 @@ +/* + * 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 /** diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 8aa77266bc..ca7cdd622a 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.util.Random diff --git a/core/src/main/scala/spark/RDDCheckpointData.scala b/core/src/main/scala/spark/RDDCheckpointData.scala index 57e0405fb4..b615f820eb 100644 --- a/core/src/main/scala/spark/RDDCheckpointData.scala +++ b/core/src/main/scala/spark/RDDCheckpointData.scala @@ -1,3 +1,20 @@ +/* + * 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 import org.apache.hadoop.fs.Path diff --git a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala index 2911f9036e..9f30b7f22f 100644 --- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.io.EOFException diff --git a/core/src/main/scala/spark/SerializableWritable.scala b/core/src/main/scala/spark/SerializableWritable.scala index 8306fbf570..0236611ef9 100644 --- a/core/src/main/scala/spark/SerializableWritable.scala +++ b/core/src/main/scala/spark/SerializableWritable.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.io._ diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala index 9513a00126..dcced035e7 100644 --- a/core/src/main/scala/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/spark/ShuffleFetcher.scala @@ -1,3 +1,20 @@ +/* + * 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 import spark.executor.TaskMetrics diff --git a/core/src/main/scala/spark/SizeEstimator.scala b/core/src/main/scala/spark/SizeEstimator.scala index f8a4c4e489..6cc57566d7 100644 --- a/core/src/main/scala/spark/SizeEstimator.scala +++ b/core/src/main/scala/spark/SizeEstimator.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.lang.reflect.Field diff --git a/core/src/main/scala/spark/SoftReferenceCache.scala b/core/src/main/scala/spark/SoftReferenceCache.scala index 3dd0a4b1f9..f41a379582 100644 --- a/core/src/main/scala/spark/SoftReferenceCache.scala +++ b/core/src/main/scala/spark/SoftReferenceCache.scala @@ -1,3 +1,20 @@ +/* + * 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 import com.google.common.collect.MapMaker diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 43e6af2351..46b9935cb7 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.io._ diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index ec59b4f48f..f2bdc11bdb 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -1,3 +1,20 @@ +/* + * 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 import collection.mutable diff --git a/core/src/main/scala/spark/SparkException.scala b/core/src/main/scala/spark/SparkException.scala index aa7a16d7dd..b7045eea63 100644 --- a/core/src/main/scala/spark/SparkException.scala +++ b/core/src/main/scala/spark/SparkException.scala @@ -1,3 +1,20 @@ +/* + * 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 class SparkException(message: String, cause: Throwable) diff --git a/core/src/main/scala/spark/SparkFiles.java b/core/src/main/scala/spark/SparkFiles.java index 566aec622c..f9b3f7965e 100644 --- a/core/src/main/scala/spark/SparkFiles.java +++ b/core/src/main/scala/spark/SparkFiles.java @@ -1,3 +1,20 @@ +/* + * 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; import java.io.File; diff --git a/core/src/main/scala/spark/TaskContext.scala b/core/src/main/scala/spark/TaskContext.scala index dd0609026a..b79f4ca813 100644 --- a/core/src/main/scala/spark/TaskContext.scala +++ b/core/src/main/scala/spark/TaskContext.scala @@ -1,3 +1,20 @@ +/* + * 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 import executor.TaskMetrics diff --git a/core/src/main/scala/spark/TaskEndReason.scala b/core/src/main/scala/spark/TaskEndReason.scala index bb75ec208c..3ad665da34 100644 --- a/core/src/main/scala/spark/TaskEndReason.scala +++ b/core/src/main/scala/spark/TaskEndReason.scala @@ -1,3 +1,20 @@ +/* + * 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 import spark.executor.TaskMetrics diff --git a/core/src/main/scala/spark/TaskState.scala b/core/src/main/scala/spark/TaskState.scala index 78eb33a628..9df7d8277b 100644 --- a/core/src/main/scala/spark/TaskState.scala +++ b/core/src/main/scala/spark/TaskState.scala @@ -1,3 +1,20 @@ +/* + * 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 import org.apache.mesos.Protos.{TaskState => MesosTaskState} diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index a36186bf8a..e6a96a5ec1 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.io._ diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala index 16692c0440..392556f261 100644 --- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.api.java import spark.RDD diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index 76051597b6..ccc511dc5f 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.api.java import java.util.{List => JList} diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala index 626b499454..c0bf2cf568 100644 --- a/core/src/main/scala/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.api.java import spark._ diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index 27f40ecdfd..21b5abf053 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -1,3 +1,20 @@ +/* + * 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.api.java import java.util.{List => JList, Comparator} diff --git a/core/src/main/scala/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/spark/api/java/JavaSparkContext.scala index 5f18b1e15b..fe182e7ab6 100644 --- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/spark/api/java/JavaSparkContext.scala @@ -1,3 +1,20 @@ +/* + * 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.api.java import java.util.{Map => JMap} diff --git a/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java b/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java index 97344e73da..42b1de01b1 100644 --- a/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java +++ b/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java @@ -1,3 +1,20 @@ +/* + * 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.api.java; import java.util.Arrays; diff --git a/core/src/main/scala/spark/api/java/StorageLevels.java b/core/src/main/scala/spark/api/java/StorageLevels.java index 5e5845ac3a..f385636e83 100644 --- a/core/src/main/scala/spark/api/java/StorageLevels.java +++ b/core/src/main/scala/spark/api/java/StorageLevels.java @@ -1,3 +1,20 @@ +/* + * 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.api.java; import spark.storage.StorageLevel; diff --git a/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java b/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java index 3a8192be3a..8bc88d757f 100644 --- a/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java +++ b/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java @@ -1,3 +1,20 @@ +/* + * 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.api.java.function; diff --git a/core/src/main/scala/spark/api/java/function/DoubleFunction.java b/core/src/main/scala/spark/api/java/function/DoubleFunction.java index c6ef76d088..1aa1e5dae0 100644 --- a/core/src/main/scala/spark/api/java/function/DoubleFunction.java +++ b/core/src/main/scala/spark/api/java/function/DoubleFunction.java @@ -1,3 +1,20 @@ +/* + * 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.api.java.function; diff --git a/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala b/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala index e027cdacd3..9eb0cfe3f9 100644 --- a/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala +++ b/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala @@ -1,3 +1,20 @@ +/* + * 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.api.java.function /** diff --git a/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala b/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala index 6044043add..dda98710c2 100644 --- a/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala +++ b/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala @@ -1,3 +1,20 @@ +/* + * 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.api.java.function /** diff --git a/core/src/main/scala/spark/api/java/function/Function.java b/core/src/main/scala/spark/api/java/function/Function.java index dae8295f21..2a2ea0aacf 100644 --- a/core/src/main/scala/spark/api/java/function/Function.java +++ b/core/src/main/scala/spark/api/java/function/Function.java @@ -1,3 +1,20 @@ +/* + * 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.api.java.function; import scala.reflect.ClassManifest; diff --git a/core/src/main/scala/spark/api/java/function/Function2.java b/core/src/main/scala/spark/api/java/function/Function2.java index 69bf12c8c9..952d31ece4 100644 --- a/core/src/main/scala/spark/api/java/function/Function2.java +++ b/core/src/main/scala/spark/api/java/function/Function2.java @@ -1,3 +1,20 @@ +/* + * 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.api.java.function; import scala.reflect.ClassManifest; diff --git a/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java b/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java index b3cc4df6aa..4aad602da3 100644 --- a/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java +++ b/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java @@ -1,3 +1,20 @@ +/* + * 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.api.java.function; import scala.Tuple2; diff --git a/core/src/main/scala/spark/api/java/function/PairFunction.java b/core/src/main/scala/spark/api/java/function/PairFunction.java index 9fc6df4b88..ccfe64ecf1 100644 --- a/core/src/main/scala/spark/api/java/function/PairFunction.java +++ b/core/src/main/scala/spark/api/java/function/PairFunction.java @@ -1,3 +1,20 @@ +/* + * 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.api.java.function; import scala.Tuple2; diff --git a/core/src/main/scala/spark/api/java/function/VoidFunction.scala b/core/src/main/scala/spark/api/java/function/VoidFunction.scala index b0096cf2bf..f6fc0b0f7d 100644 --- a/core/src/main/scala/spark/api/java/function/VoidFunction.scala +++ b/core/src/main/scala/spark/api/java/function/VoidFunction.scala @@ -1,3 +1,20 @@ +/* + * 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.api.java.function /** @@ -13,4 +30,4 @@ abstract class VoidFunction[T] extends Serializable { // return Unit), so it is implicitly converted to a Function1[T, Unit]: object VoidFunction { implicit def toFunction[T](f: VoidFunction[T]) : Function1[T, Unit] = ((x : T) => f.call(x)) -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala b/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala index 923f5cdf4f..1758a38c4e 100644 --- a/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala +++ b/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala @@ -1,3 +1,20 @@ +/* + * 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.api.java.function import scala.runtime.AbstractFunction1 diff --git a/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala b/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala index 2c6e9b1571..b093567d2c 100644 --- a/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala +++ b/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala @@ -1,3 +1,20 @@ +/* + * 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.api.java.function import scala.runtime.AbstractFunction2 diff --git a/core/src/main/scala/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/spark/api/python/PythonPartitioner.scala index d618c098c2..31a719fbff 100644 --- a/core/src/main/scala/spark/api/python/PythonPartitioner.scala +++ b/core/src/main/scala/spark/api/python/PythonPartitioner.scala @@ -1,3 +1,20 @@ +/* + * 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.api.python import spark.Partitioner diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 31d8ea89d4..af10822dbd 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.api.python import java.io._ diff --git a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala index 85d1dfeac8..078ad45ce8 100644 --- a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala @@ -1,3 +1,20 @@ +/* + * 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.api.python import java.io.{DataInputStream, IOException} diff --git a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala index adcb2d2415..6f7d385379 100644 --- a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala +++ b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala @@ -1,3 +1,20 @@ +/* + * 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.broadcast import java.io._ diff --git a/core/src/main/scala/spark/broadcast/Broadcast.scala b/core/src/main/scala/spark/broadcast/Broadcast.scala index 415bde5d67..aba56a60ca 100644 --- a/core/src/main/scala/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/spark/broadcast/Broadcast.scala @@ -1,3 +1,20 @@ +/* + * 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.broadcast import java.io._ diff --git a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/spark/broadcast/BroadcastFactory.scala index 5c6184c3c7..d33d95c7d9 100644 --- a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/spark/broadcast/BroadcastFactory.scala @@ -1,3 +1,20 @@ +/* + * 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.broadcast /** diff --git a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala index 7e30b8f7d2..c565876950 100644 --- a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala @@ -1,3 +1,20 @@ +/* + * 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.broadcast import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} diff --git a/core/src/main/scala/spark/broadcast/MultiTracker.scala b/core/src/main/scala/spark/broadcast/MultiTracker.scala index 3fd77af73f..7855d44e9b 100644 --- a/core/src/main/scala/spark/broadcast/MultiTracker.scala +++ b/core/src/main/scala/spark/broadcast/MultiTracker.scala @@ -1,3 +1,20 @@ +/* + * 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.broadcast import java.io._ diff --git a/core/src/main/scala/spark/broadcast/SourceInfo.scala b/core/src/main/scala/spark/broadcast/SourceInfo.scala index c79bb93c38..b17ae63b5c 100644 --- a/core/src/main/scala/spark/broadcast/SourceInfo.scala +++ b/core/src/main/scala/spark/broadcast/SourceInfo.scala @@ -1,3 +1,20 @@ +/* + * 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.broadcast import java.util.BitSet diff --git a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala index c55c476117..ea1e9a12c1 100644 --- a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala +++ b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala @@ -1,3 +1,20 @@ +/* + * 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.broadcast import java.io._ diff --git a/core/src/main/scala/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/spark/deploy/ApplicationDescription.scala index 02193c7008..a8b22fbef8 100644 --- a/core/src/main/scala/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/spark/deploy/ApplicationDescription.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy private[spark] class ApplicationDescription( diff --git a/core/src/main/scala/spark/deploy/Command.scala b/core/src/main/scala/spark/deploy/Command.scala index 577101e3c3..bad629e965 100644 --- a/core/src/main/scala/spark/deploy/Command.scala +++ b/core/src/main/scala/spark/deploy/Command.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy import scala.collection.Map diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala index 51274acb1e..e1f8aff6f5 100644 --- a/core/src/main/scala/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/spark/deploy/DeployMessage.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy import spark.deploy.ExecutorState.ExecutorState diff --git a/core/src/main/scala/spark/deploy/ExecutorState.scala b/core/src/main/scala/spark/deploy/ExecutorState.scala index 5dc0c54552..08c9a3b725 100644 --- a/core/src/main/scala/spark/deploy/ExecutorState.scala +++ b/core/src/main/scala/spark/deploy/ExecutorState.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy private[spark] object ExecutorState diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index 335aaeb769..64f89623e1 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy import master.{ApplicationInfo, WorkerInfo} @@ -62,4 +79,4 @@ private[spark] object JsonProtocol { ("executors" -> obj.executors.toList.map(writeExecutorRunner)) ~ ("finishedexecutors" -> obj.finishedExecutors.toList.map(writeExecutorRunner)) } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala index 939f26b6f4..6b8e9f27af 100644 --- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} diff --git a/core/src/main/scala/spark/deploy/WebUI.scala b/core/src/main/scala/spark/deploy/WebUI.scala index 844c4142c7..8ea7792ef4 100644 --- a/core/src/main/scala/spark/deploy/WebUI.scala +++ b/core/src/main/scala/spark/deploy/WebUI.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy import java.text.SimpleDateFormat diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index 4af44f9c16..edefa0292d 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.client import spark.deploy._ diff --git a/core/src/main/scala/spark/deploy/client/ClientListener.scala b/core/src/main/scala/spark/deploy/client/ClientListener.scala index e8c4083f9d..064024455e 100644 --- a/core/src/main/scala/spark/deploy/client/ClientListener.scala +++ b/core/src/main/scala/spark/deploy/client/ClientListener.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.client /** diff --git a/core/src/main/scala/spark/deploy/client/TestClient.scala b/core/src/main/scala/spark/deploy/client/TestClient.scala index f195082808..4f4daa141a 100644 --- a/core/src/main/scala/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/spark/deploy/client/TestClient.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.client import spark.util.AkkaUtils diff --git a/core/src/main/scala/spark/deploy/client/TestExecutor.scala b/core/src/main/scala/spark/deploy/client/TestExecutor.scala index 0e46db2272..8a22b6b89f 100644 --- a/core/src/main/scala/spark/deploy/client/TestExecutor.scala +++ b/core/src/main/scala/spark/deploy/client/TestExecutor.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.client private[spark] object TestExecutor { diff --git a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala index 785c16e2be..15ff919738 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.master import spark.deploy.ApplicationDescription diff --git a/core/src/main/scala/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/spark/deploy/master/ApplicationState.scala index 15016b388d..94f0ad8bae 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationState.scala +++ b/core/src/main/scala/spark/deploy/master/ApplicationState.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.master private[spark] object ApplicationState diff --git a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala b/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala index 48e6055fb5..99b60f7d09 100644 --- a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala +++ b/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.master import spark.deploy.ExecutorState diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 4dd6c448a9..e5a7a87e2e 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.master import akka.actor._ diff --git a/core/src/main/scala/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/spark/deploy/master/MasterArguments.scala index 3d28ecabb4..d0ec3d5ea0 100644 --- a/core/src/main/scala/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/spark/deploy/master/MasterArguments.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.master import spark.util.IntParam diff --git a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala index 0c08c5f417..4135cfeb28 100644 --- a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.master import akka.actor.ActorRef diff --git a/core/src/main/scala/spark/deploy/master/WorkerState.scala b/core/src/main/scala/spark/deploy/master/WorkerState.scala index 0bf35014c8..3e50b7748d 100644 --- a/core/src/main/scala/spark/deploy/master/WorkerState.scala +++ b/core/src/main/scala/spark/deploy/master/WorkerState.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.master private[spark] object WorkerState extends Enumeration("ALIVE", "DEAD", "DECOMMISSIONED") { diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala index 8553377d8f..32264af393 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.master.ui import akka.dispatch.Await diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala index 5e3c5e064f..b05197c1b9 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.master.ui import akka.dispatch.Await 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 82bc6961e2..04b32c7968 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.master.ui import akka.actor.ActorRef @@ -56,4 +73,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" -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 5d3d54c65e..8b51ff1c3a 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.worker import java.io._ diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index f20ea42d7f..0bd88ea253 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.worker import scala.collection.mutable.{ArrayBuffer, HashMap} diff --git a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala index 2b96611ee3..9fcd3260ca 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.worker import spark.util.IntParam diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala index c515f2e238..7548a26c2e 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.worker.ui import akka.dispatch.Await diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index ccd55c1ce4..61d4cd6d99 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.worker.ui import akka.actor.ActorRef diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 8360547a74..2e81151882 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -1,3 +1,20 @@ +/* + * 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.executor import java.io.{File, FileOutputStream} diff --git a/core/src/main/scala/spark/executor/ExecutorBackend.scala b/core/src/main/scala/spark/executor/ExecutorBackend.scala index e97e509700..33a6f8a824 100644 --- a/core/src/main/scala/spark/executor/ExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/ExecutorBackend.scala @@ -1,3 +1,20 @@ +/* + * 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.executor import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/executor/ExecutorExitCode.scala b/core/src/main/scala/spark/executor/ExecutorExitCode.scala index fd76029cb3..64b9fb88f8 100644 --- a/core/src/main/scala/spark/executor/ExecutorExitCode.scala +++ b/core/src/main/scala/spark/executor/ExecutorExitCode.scala @@ -1,3 +1,20 @@ +/* + * 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.executor /** diff --git a/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala b/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala index 5beb4d049e..09d12fb65b 100644 --- a/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala +++ b/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala @@ -1,3 +1,20 @@ +/* + * 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.executor import java.net.{URLClassLoader, URL} diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala index 10f3531df0..4961c42fad 100644 --- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala @@ -1,3 +1,20 @@ +/* + * 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.executor import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index ebe2ac68d8..f4003da732 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -1,3 +1,20 @@ +/* + * 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.executor import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index 1dc13754f9..3151627839 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -1,3 +1,20 @@ +/* + * 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.executor class TaskMetrics extends Serializable { diff --git a/core/src/main/scala/spark/network/BufferMessage.scala b/core/src/main/scala/spark/network/BufferMessage.scala index 7b0e489a6c..e566aeac13 100644 --- a/core/src/main/scala/spark/network/BufferMessage.scala +++ b/core/src/main/scala/spark/network/BufferMessage.scala @@ -1,3 +1,20 @@ +/* + * 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.network import java.nio.ByteBuffer @@ -91,4 +108,4 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: "BufferMessage(id = " + id + ", size = " + size + ")" } } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/network/Connection.scala b/core/src/main/scala/spark/network/Connection.scala index 6e28f677a3..b66c00b58c 100644 --- a/core/src/main/scala/spark/network/Connection.scala +++ b/core/src/main/scala/spark/network/Connection.scala @@ -1,3 +1,20 @@ +/* + * 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.network import spark._ diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 624a094856..6c4e7dc03e 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -1,3 +1,20 @@ +/* + * 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.network import spark._ diff --git a/core/src/main/scala/spark/network/ConnectionManagerId.scala b/core/src/main/scala/spark/network/ConnectionManagerId.scala index b554e84251..9d5c518293 100644 --- a/core/src/main/scala/spark/network/ConnectionManagerId.scala +++ b/core/src/main/scala/spark/network/ConnectionManagerId.scala @@ -1,3 +1,20 @@ +/* + * 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.network import java.net.InetSocketAddress diff --git a/core/src/main/scala/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/spark/network/ConnectionManagerTest.scala index 533e4610f3..9e3827aaf5 100644 --- a/core/src/main/scala/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/spark/network/ConnectionManagerTest.scala @@ -1,3 +1,20 @@ +/* + * 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.network import spark._ diff --git a/core/src/main/scala/spark/network/Message.scala b/core/src/main/scala/spark/network/Message.scala index d4f03610eb..a25457ea35 100644 --- a/core/src/main/scala/spark/network/Message.scala +++ b/core/src/main/scala/spark/network/Message.scala @@ -1,3 +1,20 @@ +/* + * 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.network import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/network/MessageChunk.scala b/core/src/main/scala/spark/network/MessageChunk.scala index aaf9204d0e..784db5ab62 100644 --- a/core/src/main/scala/spark/network/MessageChunk.scala +++ b/core/src/main/scala/spark/network/MessageChunk.scala @@ -1,3 +1,20 @@ +/* + * 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.network import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/network/MessageChunkHeader.scala b/core/src/main/scala/spark/network/MessageChunkHeader.scala index 3693d509d6..18d0cbcc14 100644 --- a/core/src/main/scala/spark/network/MessageChunkHeader.scala +++ b/core/src/main/scala/spark/network/MessageChunkHeader.scala @@ -1,3 +1,20 @@ +/* + * 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.network import java.net.InetAddress diff --git a/core/src/main/scala/spark/network/ReceiverTest.scala b/core/src/main/scala/spark/network/ReceiverTest.scala index a174d5f403..2bbc736f40 100644 --- a/core/src/main/scala/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/spark/network/ReceiverTest.scala @@ -1,3 +1,20 @@ +/* + * 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.network import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/network/SenderTest.scala b/core/src/main/scala/spark/network/SenderTest.scala index a4ff69e4d2..542c54c36b 100644 --- a/core/src/main/scala/spark/network/SenderTest.scala +++ b/core/src/main/scala/spark/network/SenderTest.scala @@ -1,3 +1,20 @@ +/* + * 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.network import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/network/netty/FileHeader.scala b/core/src/main/scala/spark/network/netty/FileHeader.scala index aed4254234..bf46d32aa3 100644 --- a/core/src/main/scala/spark/network/netty/FileHeader.scala +++ b/core/src/main/scala/spark/network/netty/FileHeader.scala @@ -1,3 +1,20 @@ +/* + * 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.network.netty import io.netty.buffer._ diff --git a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala index 8d5194a737..b01f6369f6 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/spark/network/netty/ShuffleCopier.scala @@ -1,3 +1,20 @@ +/* + * 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.network.netty import java.util.concurrent.Executors diff --git a/core/src/main/scala/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/spark/network/netty/ShuffleSender.scala index d6fa4b1e80..cdf88b03a0 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleSender.scala +++ b/core/src/main/scala/spark/network/netty/ShuffleSender.scala @@ -1,3 +1,20 @@ +/* + * 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.network.netty import java.io.File diff --git a/core/src/main/scala/spark/package.scala b/core/src/main/scala/spark/package.scala index 389ec4da3e..b244bfbf06 100644 --- a/core/src/main/scala/spark/package.scala +++ b/core/src/main/scala/spark/package.scala @@ -1,3 +1,20 @@ +/* + * 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. + */ + /** * Core Spark functionality. [[spark.SparkContext]] serves as the main entry point to Spark, while * [[spark.RDD]] is the data type representing a distributed collection, and provides most diff --git a/core/src/main/scala/spark/partial/ApproximateActionListener.scala b/core/src/main/scala/spark/partial/ApproximateActionListener.scala index de2dce161a..691d939150 100644 --- a/core/src/main/scala/spark/partial/ApproximateActionListener.scala +++ b/core/src/main/scala/spark/partial/ApproximateActionListener.scala @@ -1,3 +1,20 @@ +/* + * 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.partial import spark._ diff --git a/core/src/main/scala/spark/partial/ApproximateEvaluator.scala b/core/src/main/scala/spark/partial/ApproximateEvaluator.scala index 75713b2eaa..5eae144dfb 100644 --- a/core/src/main/scala/spark/partial/ApproximateEvaluator.scala +++ b/core/src/main/scala/spark/partial/ApproximateEvaluator.scala @@ -1,3 +1,20 @@ +/* + * 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.partial /** diff --git a/core/src/main/scala/spark/partial/BoundedDouble.scala b/core/src/main/scala/spark/partial/BoundedDouble.scala index 463c33d6e2..8bdbe6c012 100644 --- a/core/src/main/scala/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/spark/partial/BoundedDouble.scala @@ -1,3 +1,20 @@ +/* + * 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.partial /** diff --git a/core/src/main/scala/spark/partial/CountEvaluator.scala b/core/src/main/scala/spark/partial/CountEvaluator.scala index daf2c5170c..6aa92094eb 100644 --- a/core/src/main/scala/spark/partial/CountEvaluator.scala +++ b/core/src/main/scala/spark/partial/CountEvaluator.scala @@ -1,3 +1,20 @@ +/* + * 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.partial import cern.jet.stat.Probability diff --git a/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala b/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala index 01fbb8a11b..ebe2e5a1e3 100644 --- a/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala +++ b/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala @@ -1,3 +1,20 @@ +/* + * 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.partial import java.util.{HashMap => JHashMap} diff --git a/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala b/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala index c622df5220..2dadbbd5fb 100644 --- a/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala +++ b/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala @@ -1,3 +1,20 @@ +/* + * 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.partial import java.util.{HashMap => JHashMap} diff --git a/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala b/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala index 20fa55cff2..ae2b63f7cb 100644 --- a/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala +++ b/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala @@ -1,3 +1,20 @@ +/* + * 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.partial import java.util.{HashMap => JHashMap} diff --git a/core/src/main/scala/spark/partial/MeanEvaluator.scala b/core/src/main/scala/spark/partial/MeanEvaluator.scala index 762c85400d..5ddcad7075 100644 --- a/core/src/main/scala/spark/partial/MeanEvaluator.scala +++ b/core/src/main/scala/spark/partial/MeanEvaluator.scala @@ -1,3 +1,20 @@ +/* + * 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.partial import cern.jet.stat.Probability diff --git a/core/src/main/scala/spark/partial/PartialResult.scala b/core/src/main/scala/spark/partial/PartialResult.scala index 200ed4ea1e..922a9f9bc6 100644 --- a/core/src/main/scala/spark/partial/PartialResult.scala +++ b/core/src/main/scala/spark/partial/PartialResult.scala @@ -1,3 +1,20 @@ +/* + * 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.partial class PartialResult[R](initialVal: R, isFinal: Boolean) { diff --git a/core/src/main/scala/spark/partial/StudentTCacher.scala b/core/src/main/scala/spark/partial/StudentTCacher.scala index 443abba5cd..f3bb987d46 100644 --- a/core/src/main/scala/spark/partial/StudentTCacher.scala +++ b/core/src/main/scala/spark/partial/StudentTCacher.scala @@ -1,3 +1,20 @@ +/* + * 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.partial import cern.jet.stat.Probability diff --git a/core/src/main/scala/spark/partial/SumEvaluator.scala b/core/src/main/scala/spark/partial/SumEvaluator.scala index 58fb60f441..4083abef03 100644 --- a/core/src/main/scala/spark/partial/SumEvaluator.scala +++ b/core/src/main/scala/spark/partial/SumEvaluator.scala @@ -1,3 +1,20 @@ +/* + * 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.partial import cern.jet.stat.Probability diff --git a/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/spark/rdd/BlockRDD.scala index 719d4bf03e..0ebb722d73 100644 --- a/core/src/main/scala/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/spark/rdd/BlockRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/CartesianRDD.scala b/core/src/main/scala/spark/rdd/CartesianRDD.scala index 38600b8be4..150e5bca29 100644 --- a/core/src/main/scala/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/spark/rdd/CartesianRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import java.io.{ObjectOutputStream, IOException} diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 377b1bdbe0..6794e0e201 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import spark._ diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 8966f9f86e..c540cd36eb 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import java.io.{ObjectOutputStream, IOException} diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 6d862c0c28..2b5bf18541 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import spark.{Dependency, OneToOneDependency, NarrowDependency, RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/EmptyRDD.scala b/core/src/main/scala/spark/rdd/EmptyRDD.scala index e4dd3a7fa7..d7d4db5d30 100644 --- a/core/src/main/scala/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/spark/rdd/EmptyRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/FilteredRDD.scala b/core/src/main/scala/spark/rdd/FilteredRDD.scala index c84ec39d21..783508cfd1 100644 --- a/core/src/main/scala/spark/rdd/FilteredRDD.scala +++ b/core/src/main/scala/spark/rdd/FilteredRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import spark.{OneToOneDependency, RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala index 8ebc778925..ed75eac3ff 100644 --- a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala +++ b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import spark.{RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/GlommedRDD.scala b/core/src/main/scala/spark/rdd/GlommedRDD.scala index e16c7ba881..1573f8a289 100644 --- a/core/src/main/scala/spark/rdd/GlommedRDD.scala +++ b/core/src/main/scala/spark/rdd/GlommedRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import spark.{RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index 07c103503c..d0fdeb741e 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import java.io.EOFException diff --git a/core/src/main/scala/spark/rdd/JdbcRDD.scala b/core/src/main/scala/spark/rdd/JdbcRDD.scala index a50f407737..59132437d2 100644 --- a/core/src/main/scala/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/spark/rdd/JdbcRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import java.sql.{Connection, ResultSet} diff --git a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala index d283c5b2bb..af8f0a112f 100644 --- a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import spark.{RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala index afb7504ba1..3b4e9518fd 100644 --- a/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala +++ b/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import spark.{RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/MappedRDD.scala b/core/src/main/scala/spark/rdd/MappedRDD.scala index af07311b6d..8b411dd85d 100644 --- a/core/src/main/scala/spark/rdd/MappedRDD.scala +++ b/core/src/main/scala/spark/rdd/MappedRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import spark.{RDD, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index 901d01ef30..17fe805fd4 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import java.text.SimpleDateFormat diff --git a/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala index 07585a88ce..16ba0c26f8 100644 --- a/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import scala.collection.immutable.NumericRange diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index 41ff62dd22..191cfde565 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import spark.{NarrowDependency, RDD, SparkEnv, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala index c0baf43d43..2cefdc78b0 100644 --- a/core/src/main/scala/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/spark/rdd/PipedRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import java.io.PrintWriter diff --git a/core/src/main/scala/spark/rdd/SampledRDD.scala b/core/src/main/scala/spark/rdd/SampledRDD.scala index 243673f151..574c9b141d 100644 --- a/core/src/main/scala/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/spark/rdd/SampledRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import java.util.Random diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala index c7d1926b83..0137f80953 100644 --- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import spark.{Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala index 8a9efc5da2..0402b9f250 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import java.util.{HashMap => JHashMap} diff --git a/core/src/main/scala/spark/rdd/UnionRDD.scala b/core/src/main/scala/spark/rdd/UnionRDD.scala index 2c52a67e22..2776826f18 100644 --- a/core/src/main/scala/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/spark/rdd/UnionRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala index b234428ab2..6a4fa13ad6 100644 --- a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala index f728e93d24..b1c43b3195 100644 --- a/core/src/main/scala/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} diff --git a/core/src/main/scala/spark/scheduler/ActiveJob.scala b/core/src/main/scala/spark/scheduler/ActiveJob.scala index 105eaecb22..71cc94edb6 100644 --- a/core/src/main/scala/spark/scheduler/ActiveJob.scala +++ b/core/src/main/scala/spark/scheduler/ActiveJob.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import spark.TaskContext diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 64ed91f5a0..29e879aa42 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import cluster.TaskInfo diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index acad915f13..506c87f65b 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import java.util.Properties diff --git a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala index 17d0ea4f80..65f8c3200e 100644 --- a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import spark.Logging diff --git a/core/src/main/scala/spark/scheduler/JobListener.scala b/core/src/main/scala/spark/scheduler/JobListener.scala index f46b9d551d..af108b8fec 100644 --- a/core/src/main/scala/spark/scheduler/JobListener.scala +++ b/core/src/main/scala/spark/scheduler/JobListener.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler /** diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index 6a9d52f356..85b5ddd4a8 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import java.io.PrintWriter diff --git a/core/src/main/scala/spark/scheduler/JobResult.scala b/core/src/main/scala/spark/scheduler/JobResult.scala index a0fdf391e6..a61b335152 100644 --- a/core/src/main/scala/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/spark/scheduler/JobResult.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler /** diff --git a/core/src/main/scala/spark/scheduler/JobWaiter.scala b/core/src/main/scala/spark/scheduler/JobWaiter.scala index 6ff2e29434..69cd161c1f 100644 --- a/core/src/main/scala/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/spark/scheduler/JobWaiter.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/scheduler/MapStatus.scala b/core/src/main/scala/spark/scheduler/MapStatus.scala index 203abb917b..2f6a68ee85 100644 --- a/core/src/main/scala/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/spark/scheduler/MapStatus.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import spark.storage.BlockManagerId diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala index 83166bce22..361b1e6b91 100644 --- a/core/src/main/scala/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/spark/scheduler/ResultTask.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import spark._ diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index 95647389c3..1c25605f75 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import java.io._ diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 8de3aa91a4..4fb1c5d42d 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import java.util.Properties diff --git a/core/src/main/scala/spark/scheduler/SplitInfo.scala b/core/src/main/scala/spark/scheduler/SplitInfo.scala index 6abfb7a1f7..4e3661ec5d 100644 --- a/core/src/main/scala/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/spark/scheduler/SplitInfo.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala index 4937eb3b88..5428daeb94 100644 --- a/core/src/main/scala/spark/scheduler/Stage.scala +++ b/core/src/main/scala/spark/scheduler/Stage.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import java.net.URI diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala index 8d83ff10c4..c4026f995a 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/spark/scheduler/StageInfo.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import spark.scheduler.cluster.TaskInfo @@ -9,4 +26,4 @@ case class StageInfo( val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = mutable.Buffer[(TaskInfo, TaskMetrics)]() ) { override def toString = stage.rdd.toString -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/scheduler/Task.scala b/core/src/main/scala/spark/scheduler/Task.scala index a6462c6968..50768d43e0 100644 --- a/core/src/main/scala/spark/scheduler/Task.scala +++ b/core/src/main/scala/spark/scheduler/Task.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import spark.serializer.SerializerInstance diff --git a/core/src/main/scala/spark/scheduler/TaskResult.scala b/core/src/main/scala/spark/scheduler/TaskResult.scala index 6de0aa7adf..dc0621ea7b 100644 --- a/core/src/main/scala/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/spark/scheduler/TaskResult.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import java.io._ diff --git a/core/src/main/scala/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/spark/scheduler/TaskScheduler.scala index 7787b54762..5188308006 100644 --- a/core/src/main/scala/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/spark/scheduler/TaskScheduler.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler /** diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala index b75d3736cf..245e7ccb52 100644 --- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala +++ b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import spark.scheduler.cluster.TaskInfo diff --git a/core/src/main/scala/spark/scheduler/TaskSet.scala b/core/src/main/scala/spark/scheduler/TaskSet.scala index e4b5fcaedb..dc3550dd0b 100644 --- a/core/src/main/scala/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/spark/scheduler/TaskSet.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import java.util.Properties diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 3a0c29b27f..7c10074dc7 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.cluster import java.lang.{Boolean => JBoolean} diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 327d6797ae..3d06520675 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.cluster import java.util.{HashMap => JHashMap, NoSuchElementException, Arrays} diff --git a/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala b/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala index 8bf838209f..8825f2dd24 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.cluster import spark.executor.ExecutorExitCode diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/spark/scheduler/cluster/Pool.scala index 941ba7a3f1..83708f07e1 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Pool.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Pool.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.cluster import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala index 2dd9c0564f..f557b142c4 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.cluster import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala index 18cc15c2a5..95554023c0 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.cluster import java.io.{File, FileInputStream, FileOutputStream} diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala index 8844057a5c..4431744ec3 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.cluster import spark.{SparkContext, Utils} diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala index f33310a34a..69e0ac2a6b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.cluster /** diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala index 6e0c6793e0..4b3e3e50e1 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.cluster object SchedulingMode extends Enumeration("FAIR","FIFO"){ diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 170ede0f44..55d6c0a47e 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.cluster import spark.{Utils, Logging, SparkContext} diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala index 3335294844..ac9e5ef94d 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.cluster import spark.TaskState.TaskState diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 004592a540..03a64e0192 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.cluster import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala index b41e951be9..761fdf6919 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.cluster import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index 718f26bfbd..a1ebd48b01 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.cluster import spark.Utils diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index b4dd75d90f..07c3ddcc7e 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.cluster import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala index c47824315c..06d1203f70 100644 --- a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala +++ b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.cluster /** diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index b000e328e6..1f73cb99a7 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.local import java.io.File diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index f12fec41d5..e662ad6709 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.local import java.io.File diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index f4a2994b6d..7bc6040544 100644 --- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.mesos import com.google.protobuf.ByteString diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala index e83368b98d..75b8268b55 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler.mesos import com.google.protobuf.ByteString diff --git a/core/src/main/scala/spark/serializer/Serializer.scala b/core/src/main/scala/spark/serializer/Serializer.scala index 2ad73b711d..dc94d42bb6 100644 --- a/core/src/main/scala/spark/serializer/Serializer.scala +++ b/core/src/main/scala/spark/serializer/Serializer.scala @@ -1,3 +1,20 @@ +/* + * 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.serializer import java.io.{EOFException, InputStream, OutputStream} diff --git a/core/src/main/scala/spark/serializer/SerializerManager.scala b/core/src/main/scala/spark/serializer/SerializerManager.scala index 60b2aac797..b7b24705a2 100644 --- a/core/src/main/scala/spark/serializer/SerializerManager.scala +++ b/core/src/main/scala/spark/serializer/SerializerManager.scala @@ -1,3 +1,20 @@ +/* + * 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.serializer import java.util.concurrent.ConcurrentHashMap diff --git a/core/src/main/scala/spark/storage/BlockException.scala b/core/src/main/scala/spark/storage/BlockException.scala index f275d476df..8ebfaf3cbf 100644 --- a/core/src/main/scala/spark/storage/BlockException.scala +++ b/core/src/main/scala/spark/storage/BlockException.scala @@ -1,3 +1,20 @@ +/* + * 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.storage private[spark] diff --git a/core/src/main/scala/spark/storage/BlockFetchTracker.scala b/core/src/main/scala/spark/storage/BlockFetchTracker.scala index 0718156b1b..265e554ad8 100644 --- a/core/src/main/scala/spark/storage/BlockFetchTracker.scala +++ b/core/src/main/scala/spark/storage/BlockFetchTracker.scala @@ -1,3 +1,20 @@ +/* + * 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.storage private[spark] trait BlockFetchTracker { diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index bec876213e..1965c5bc19 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 9b39d3aadf..e4ffa57ad2 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import java.io.{InputStream, OutputStream} diff --git a/core/src/main/scala/spark/storage/BlockManagerId.scala b/core/src/main/scala/spark/storage/BlockManagerId.scala index 1e557d6148..b36a6176c0 100644 --- a/core/src/main/scala/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/spark/storage/BlockManagerId.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 58888b1ebb..3186f7c85b 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import akka.actor.ActorRef diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index 0d4384ba1f..244000d952 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import java.util.{HashMap => JHashMap} diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/spark/storage/BlockManagerMessages.scala index 0010726c8d..01de4ccb8f 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMessages.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import java.io.{Externalizable, ObjectInput, ObjectOutput} diff --git a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala index b264d1deb5..45cffad810 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import akka.actor.Actor diff --git a/core/src/main/scala/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/spark/storage/BlockManagerWorker.scala index 3057ade233..39064bce92 100644 --- a/core/src/main/scala/spark/storage/BlockManagerWorker.scala +++ b/core/src/main/scala/spark/storage/BlockManagerWorker.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/BlockMessage.scala b/core/src/main/scala/spark/storage/BlockMessage.scala index 30d7500e01..ab72dbb62b 100644 --- a/core/src/main/scala/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/spark/storage/BlockMessage.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/BlockMessageArray.scala b/core/src/main/scala/spark/storage/BlockMessageArray.scala index ee0c5ff9a2..b0229d6124 100644 --- a/core/src/main/scala/spark/storage/BlockMessageArray.scala +++ b/core/src/main/scala/spark/storage/BlockMessageArray.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/spark/storage/BlockObjectWriter.scala index 42e2b07d5c..01ed6e8c1f 100644 --- a/core/src/main/scala/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/spark/storage/BlockObjectWriter.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/BlockStore.scala b/core/src/main/scala/spark/storage/BlockStore.scala index 8188d3595e..c8db0022b0 100644 --- a/core/src/main/scala/spark/storage/BlockStore.scala +++ b/core/src/main/scala/spark/storage/BlockStore.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index da859eebcb..3495d653bd 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import java.io.{File, FileOutputStream, OutputStream, RandomAccessFile} diff --git a/core/src/main/scala/spark/storage/MemoryStore.scala b/core/src/main/scala/spark/storage/MemoryStore.scala index eba5ee507f..b5a86b85a7 100644 --- a/core/src/main/scala/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/spark/storage/MemoryStore.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import java.util.LinkedHashMap diff --git a/core/src/main/scala/spark/storage/PutResult.scala b/core/src/main/scala/spark/storage/PutResult.scala index 76f236057b..3a0974fe15 100644 --- a/core/src/main/scala/spark/storage/PutResult.scala +++ b/core/src/main/scala/spark/storage/PutResult.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala index 44638e0c2d..8a7a6f9ed3 100644 --- a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/spark/storage/ShuffleBlockManager.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import spark.serializer.Serializer diff --git a/core/src/main/scala/spark/storage/StorageLevel.scala b/core/src/main/scala/spark/storage/StorageLevel.scala index cc0c354e7e..f52650988c 100644 --- a/core/src/main/scala/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/spark/storage/StorageLevel.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index aca16f533a..2aeed4ea3c 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import spark.{Utils, SparkContext} diff --git a/core/src/main/scala/spark/storage/ThreadingTest.scala b/core/src/main/scala/spark/storage/ThreadingTest.scala index 5c406e68cb..b3ab1ff4b4 100644 --- a/core/src/main/scala/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/spark/storage/ThreadingTest.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import akka.actor._ diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index bc6f9c10d5..ca6088ad93 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -1,3 +1,20 @@ +/* + * 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.ui import annotation.tailrec diff --git a/core/src/main/scala/spark/ui/Page.scala b/core/src/main/scala/spark/ui/Page.scala index ed8f91842c..a31e750d06 100644 --- a/core/src/main/scala/spark/ui/Page.scala +++ b/core/src/main/scala/spark/ui/Page.scala @@ -1,3 +1,20 @@ +/* + * 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.ui -private[spark] object Page extends Enumeration { val Storage, Jobs, Environment = Value } \ No newline at end of file +private[spark] object Page extends Enumeration { val Storage, Jobs, Environment = Value } diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 874e5ba8ec..9396f22063 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -1,3 +1,20 @@ +/* + * 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.ui import javax.servlet.http.HttpServletRequest diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala index fa46e2487d..b1d11954dd 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -1,3 +1,20 @@ +/* + * 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.ui import scala.xml.Node @@ -115,4 +132,4 @@ private[spark] object UIUtils { } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala index 8bbc6ce88e..a80e2d7002 100644 --- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala @@ -1,3 +1,20 @@ +/* + * 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.ui import scala.util.Random diff --git a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala index 6b8b9f05bb..5ae7935ed4 100644 --- a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala @@ -1,3 +1,20 @@ +/* + * 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.ui.env import javax.servlet.http.HttpServletRequest diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 2df5f0192b..f31af3cda6 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -1,3 +1,20 @@ +/* + * 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.ui.jobs import java.util.Date diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 84730cc091..44dcf82d11 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -1,3 +1,20 @@ +/* + * 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.ui.jobs import akka.util.Duration @@ -124,4 +141,4 @@ private[spark] class JobProgressListener extends SparkListener { } return false // No tasks have finished for this stage } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 51b82b6a8c..292966f23a 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -1,3 +1,20 @@ +/* + * 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.ui.jobs import java.util.Date diff --git a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala index 3d67888e2c..49ed069c75 100644 --- a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala @@ -1,3 +1,20 @@ +/* + * 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.ui.storage import akka.util.Duration diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index d284134391..4e0360d19a 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -1,3 +1,20 @@ +/* + * 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.ui.storage import javax.servlet.http.HttpServletRequest diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 428db6fa95..003be54ad8 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -1,3 +1,20 @@ +/* + * 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.ui.storage import javax.servlet.http.HttpServletRequest diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index 18fc9673bf..9233277bdb 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -1,3 +1,20 @@ +/* + * 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.util import akka.actor.{ActorSystem, ExtendedActorSystem} diff --git a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala index 4bc5db8bb7..0575497f5d 100644 --- a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/spark/util/BoundedPriorityQueue.scala @@ -1,3 +1,20 @@ +/* + * 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.util import java.io.Serializable diff --git a/core/src/main/scala/spark/util/ByteBufferInputStream.scala b/core/src/main/scala/spark/util/ByteBufferInputStream.scala index d7e67497fe..47a28e2f76 100644 --- a/core/src/main/scala/spark/util/ByteBufferInputStream.scala +++ b/core/src/main/scala/spark/util/ByteBufferInputStream.scala @@ -1,3 +1,20 @@ +/* + * 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.util import java.io.InputStream diff --git a/core/src/main/scala/spark/util/CompletionIterator.scala b/core/src/main/scala/spark/util/CompletionIterator.scala index 8139183780..210450892b 100644 --- a/core/src/main/scala/spark/util/CompletionIterator.scala +++ b/core/src/main/scala/spark/util/CompletionIterator.scala @@ -1,3 +1,20 @@ +/* + * 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.util /** @@ -22,4 +39,4 @@ object CompletionIterator { def completion() = completionFunction } } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/util/Distribution.scala b/core/src/main/scala/spark/util/Distribution.scala index 24738b4307..5d4d7a6c50 100644 --- a/core/src/main/scala/spark/util/Distribution.scala +++ b/core/src/main/scala/spark/util/Distribution.scala @@ -1,3 +1,20 @@ +/* + * 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.util import java.io.PrintStream @@ -62,4 +79,4 @@ object Distribution { quantiles.foreach{q => out.print(q + "\t")} out.println } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/util/IdGenerator.scala b/core/src/main/scala/spark/util/IdGenerator.scala index b6e309fe1a..3422280559 100644 --- a/core/src/main/scala/spark/util/IdGenerator.scala +++ b/core/src/main/scala/spark/util/IdGenerator.scala @@ -1,3 +1,20 @@ +/* + * 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.util import java.util.concurrent.atomic.AtomicInteger diff --git a/core/src/main/scala/spark/util/IntParam.scala b/core/src/main/scala/spark/util/IntParam.scala index 0427646747..daf0d58fa2 100644 --- a/core/src/main/scala/spark/util/IntParam.scala +++ b/core/src/main/scala/spark/util/IntParam.scala @@ -1,3 +1,20 @@ +/* + * 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.util /** diff --git a/core/src/main/scala/spark/util/MemoryParam.scala b/core/src/main/scala/spark/util/MemoryParam.scala index 3726738842..298562323a 100644 --- a/core/src/main/scala/spark/util/MemoryParam.scala +++ b/core/src/main/scala/spark/util/MemoryParam.scala @@ -1,3 +1,20 @@ +/* + * 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.util import spark.Utils diff --git a/core/src/main/scala/spark/util/MetadataCleaner.scala b/core/src/main/scala/spark/util/MetadataCleaner.scala index dafa906712..92909e0959 100644 --- a/core/src/main/scala/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/spark/util/MetadataCleaner.scala @@ -1,3 +1,20 @@ +/* + * 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.util import java.util.concurrent.{TimeUnit, ScheduledFuture, Executors} diff --git a/core/src/main/scala/spark/util/NextIterator.scala b/core/src/main/scala/spark/util/NextIterator.scala index 48b5018ddd..22163ece8d 100644 --- a/core/src/main/scala/spark/util/NextIterator.scala +++ b/core/src/main/scala/spark/util/NextIterator.scala @@ -1,3 +1,20 @@ +/* + * 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.util /** Provides a basic/boilerplate Iterator implementation. */ @@ -68,4 +85,4 @@ private[spark] abstract class NextIterator[U] extends Iterator[U] { gotNext = false nextValue } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/util/RateLimitedOutputStream.scala b/core/src/main/scala/spark/util/RateLimitedOutputStream.scala index e3f00ea8c7..00f782bbe7 100644 --- a/core/src/main/scala/spark/util/RateLimitedOutputStream.scala +++ b/core/src/main/scala/spark/util/RateLimitedOutputStream.scala @@ -1,3 +1,20 @@ +/* + * 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.util import scala.annotation.tailrec diff --git a/core/src/main/scala/spark/util/SerializableBuffer.scala b/core/src/main/scala/spark/util/SerializableBuffer.scala index 09d588fe1c..7e6842628a 100644 --- a/core/src/main/scala/spark/util/SerializableBuffer.scala +++ b/core/src/main/scala/spark/util/SerializableBuffer.scala @@ -1,3 +1,20 @@ +/* + * 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.util import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/util/StatCounter.scala b/core/src/main/scala/spark/util/StatCounter.scala index 2b980340b7..76358d4151 100644 --- a/core/src/main/scala/spark/util/StatCounter.scala +++ b/core/src/main/scala/spark/util/StatCounter.scala @@ -1,3 +1,20 @@ +/* + * 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.util /** diff --git a/core/src/main/scala/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/spark/util/TimeStampedHashMap.scala index 92dfaa6e6f..cc7909194a 100644 --- a/core/src/main/scala/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/spark/util/TimeStampedHashMap.scala @@ -1,3 +1,20 @@ +/* + * 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.util import java.util.concurrent.ConcurrentHashMap diff --git a/core/src/main/scala/spark/util/TimeStampedHashSet.scala b/core/src/main/scala/spark/util/TimeStampedHashSet.scala index 5f1cc93752..41e3fd8cba 100644 --- a/core/src/main/scala/spark/util/TimeStampedHashSet.scala +++ b/core/src/main/scala/spark/util/TimeStampedHashSet.scala @@ -1,3 +1,20 @@ +/* + * 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.util import scala.collection.mutable.Set diff --git a/core/src/main/scala/spark/util/Vector.scala b/core/src/main/scala/spark/util/Vector.scala index 835822edb2..ed49386f18 100644 --- a/core/src/main/scala/spark/util/Vector.scala +++ b/core/src/main/scala/spark/util/Vector.scala @@ -1,3 +1,20 @@ +/* + * 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.util class Vector(val elements: Array[Double]) extends Serializable { diff --git a/core/src/test/resources/fairscheduler.xml b/core/src/test/resources/fairscheduler.xml index 6e573b1883..996ffb1864 100644 --- a/core/src/test/resources/fairscheduler.xml +++ b/core/src/test/resources/fairscheduler.xml @@ -1,4 +1,21 @@ + + 2 diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index d05cf3dec1..f6fef03689 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -1,4 +1,21 @@ -# Set everything to be logged to the file core/target/unit-tests.log +# +# 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. +# + +# Set everything to be logged to the file core/target/unit-tests.log log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false diff --git a/core/src/test/scala/spark/AccumulatorSuite.scala b/core/src/test/scala/spark/AccumulatorSuite.scala index ac8ae7d308..0af175f316 100644 --- a/core/src/test/scala/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/spark/AccumulatorSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/BroadcastSuite.scala b/core/src/test/scala/spark/BroadcastSuite.scala index 362a31fb0d..785721ece8 100644 --- a/core/src/test/scala/spark/BroadcastSuite.scala +++ b/core/src/test/scala/spark/BroadcastSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala index 28a7b21b92..a84c89e3c9 100644 --- a/core/src/test/scala/spark/CheckpointSuite.scala +++ b/core/src/test/scala/spark/CheckpointSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/ClosureCleanerSuite.scala b/core/src/test/scala/spark/ClosureCleanerSuite.scala index b2d0dd4627..7d2831e19c 100644 --- a/core/src/test/scala/spark/ClosureCleanerSuite.scala +++ b/core/src/test/scala/spark/ClosureCleanerSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.io.NotSerializableException diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala index 0024ede828..e11efe459c 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/spark/DistributedSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import network.ConnectionManagerId diff --git a/core/src/test/scala/spark/DriverSuite.scala b/core/src/test/scala/spark/DriverSuite.scala index 31c3dd75fb..ed16b9d8ef 100644 --- a/core/src/test/scala/spark/DriverSuite.scala +++ b/core/src/test/scala/spark/DriverSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.io.File diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/spark/FailureSuite.scala index 8c1445a465..6c847b8fef 100644 --- a/core/src/test/scala/spark/FailureSuite.scala +++ b/core/src/test/scala/spark/FailureSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/spark/FileServerSuite.scala index c7855a7bd3..242ae971f8 100644 --- a/core/src/test/scala/spark/FileServerSuite.scala +++ b/core/src/test/scala/spark/FileServerSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import com.google.common.io.Files diff --git a/core/src/test/scala/spark/FileSuite.scala b/core/src/test/scala/spark/FileSuite.scala index e61ff7793d..1e2c257c4b 100644 --- a/core/src/test/scala/spark/FileSuite.scala +++ b/core/src/test/scala/spark/FileSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.io.{FileWriter, PrintWriter, File} diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index d306124fca..aaf03e683b 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -1,3 +1,20 @@ +/* + * 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; import java.io.File; diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/spark/KryoSerializerSuite.scala index 327e2ff848..c3323dcbb3 100644 --- a/core/src/test/scala/spark/KryoSerializerSuite.scala +++ b/core/src/test/scala/spark/KryoSerializerSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import scala.collection.mutable diff --git a/core/src/test/scala/spark/LocalSparkContext.scala b/core/src/test/scala/spark/LocalSparkContext.scala index bd184222ed..ddc212d290 100644 --- a/core/src/test/scala/spark/LocalSparkContext.scala +++ b/core/src/test/scala/spark/LocalSparkContext.scala @@ -1,3 +1,20 @@ +/* + * 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 import org.scalatest.Suite diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index 6e585e1c3a..ce6cec0451 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/PairRDDFunctionsSuite.scala b/core/src/test/scala/spark/PairRDDFunctionsSuite.scala index 682d2745bf..b102eaf4e6 100644 --- a/core/src/test/scala/spark/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/spark/PairRDDFunctionsSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import scala.collection.mutable.ArrayBuffer diff --git a/core/src/test/scala/spark/PartitioningSuite.scala b/core/src/test/scala/spark/PartitioningSuite.scala index 99e433e3bd..b1e0b2b4d0 100644 --- a/core/src/test/scala/spark/PartitioningSuite.scala +++ b/core/src/test/scala/spark/PartitioningSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/spark/PipedRDDSuite.scala index d263bb00e9..35c04710a3 100644 --- a/core/src/test/scala/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/spark/PipedRDDSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 7f7d4c8211..cbddf4e523 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import scala.collection.mutable.HashMap diff --git a/core/src/test/scala/spark/SharedSparkContext.scala b/core/src/test/scala/spark/SharedSparkContext.scala index 1da79f9824..70c24515be 100644 --- a/core/src/test/scala/spark/SharedSparkContext.scala +++ b/core/src/test/scala/spark/SharedSparkContext.scala @@ -1,3 +1,20 @@ +/* + * 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 import org.scalatest.Suite diff --git a/core/src/test/scala/spark/ShuffleNettySuite.scala b/core/src/test/scala/spark/ShuffleNettySuite.scala index bfaffa953e..6bad6c1d13 100644 --- a/core/src/test/scala/spark/ShuffleNettySuite.scala +++ b/core/src/test/scala/spark/ShuffleNettySuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import org.scalatest.BeforeAndAfterAll diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 950218fa28..3a56c26bef 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import scala.collection.mutable.ArrayBuffer diff --git a/core/src/test/scala/spark/SizeEstimatorSuite.scala b/core/src/test/scala/spark/SizeEstimatorSuite.scala index b5c8525f91..1ef812dfbd 100644 --- a/core/src/test/scala/spark/SizeEstimatorSuite.scala +++ b/core/src/test/scala/spark/SizeEstimatorSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/SortingSuite.scala b/core/src/test/scala/spark/SortingSuite.scala index f7bf207c68..b933c4aab8 100644 --- a/core/src/test/scala/spark/SortingSuite.scala +++ b/core/src/test/scala/spark/SortingSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/ThreadingSuite.scala b/core/src/test/scala/spark/ThreadingSuite.scala index ff315b6693..f2acd0bd3c 100644 --- a/core/src/test/scala/spark/ThreadingSuite.scala +++ b/core/src/test/scala/spark/ThreadingSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import java.util.concurrent.Semaphore diff --git a/core/src/test/scala/spark/UnpersistSuite.scala b/core/src/test/scala/spark/UnpersistSuite.scala index 94776e7572..93977d16f4 100644 --- a/core/src/test/scala/spark/UnpersistSuite.scala +++ b/core/src/test/scala/spark/UnpersistSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala index 1e1260f606..31c3b25c50 100644 --- a/core/src/test/scala/spark/UtilsSuite.scala +++ b/core/src/test/scala/spark/UtilsSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import com.google.common.base.Charsets diff --git a/core/src/test/scala/spark/ZippedPartitionsSuite.scala b/core/src/test/scala/spark/ZippedPartitionsSuite.scala index 96cb295f45..5e6d7b09d8 100644 --- a/core/src/test/scala/spark/ZippedPartitionsSuite.scala +++ b/core/src/test/scala/spark/ZippedPartitionsSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import scala.collection.immutable.NumericRange diff --git a/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala index 6afb0fa9bc..dc8ca941c1 100644 --- a/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala +++ b/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import org.scalatest.{ BeforeAndAfter, FunSuite } diff --git a/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala b/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala index d27a2538e4..d1276d541f 100644 --- a/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala +++ b/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.rdd import scala.collection.immutable.NumericRange diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala index 8e1ad27e14..8f81d0b6ee 100644 --- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 30e6fef950..f802b66cf1 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import scala.collection.mutable.{Map, HashMap} diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala index 2474d744b8..0f855c38da 100644 --- a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import java.util.Properties diff --git a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala index 8bd813fd14..14bb58731b 100644 --- a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala index 48aa67c543..392d67d67b 100644 --- a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala index 647bcaf860..95a6eee2fc 100644 --- a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.scheduler import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala index b9d5f9668e..b719d65342 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import java.nio.ByteBuffer diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index fc0c160720..56c1fed6ad 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -1,3 +1,20 @@ +/* + * 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.ui import scala.util.{Failure, Success, Try} diff --git a/core/src/test/scala/spark/util/DistributionSuite.scala b/core/src/test/scala/spark/util/DistributionSuite.scala index cc6249b1dd..6578b55e82 100644 --- a/core/src/test/scala/spark/util/DistributionSuite.scala +++ b/core/src/test/scala/spark/util/DistributionSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.util import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/util/NextIteratorSuite.scala b/core/src/test/scala/spark/util/NextIteratorSuite.scala index ed5b36da73..fdbd43d941 100644 --- a/core/src/test/scala/spark/util/NextIteratorSuite.scala +++ b/core/src/test/scala/spark/util/NextIteratorSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.util import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala b/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala index 794063fb6d..4c0044202f 100644 --- a/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala +++ b/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.util import org.scalatest.FunSuite diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index c10ae595de..45ef4bba82 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -1,3 +1,20 @@ +# +# 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. +# + require 'fileutils' include FileUtils diff --git a/ec2/spark-ec2 b/ec2/spark-ec2 index 2714f19ba3..454057aa0d 100755 --- a/ec2/spark-ec2 +++ b/ec2/spark-ec2 @@ -1,5 +1,6 @@ #!/bin/sh +# # 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 @@ -15,6 +16,7 @@ # 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. +# cd "`dirname $0`" PYTHONPATH="./third_party/boto-2.4.1.zip/boto-2.4.1:$PYTHONPATH" python ./spark_ec2.py $@ diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 7affe6fffc..2ec3c007fb 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -1,6 +1,7 @@ #!/usr/bin/env python # -*- coding: utf-8 -*- +# # 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 @@ -16,6 +17,7 @@ # 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. +# from __future__ import with_statement diff --git a/examples/pom.xml b/examples/pom.xml index 1976765c3d..7a8d08fade 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 diff --git a/examples/src/main/java/spark/examples/JavaHdfsLR.java b/examples/src/main/java/spark/examples/JavaHdfsLR.java index 8b0a9b6808..9485e0cfa9 100644 --- a/examples/src/main/java/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/spark/examples/JavaHdfsLR.java @@ -1,3 +1,20 @@ +/* + * 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.examples; import spark.api.java.JavaRDD; diff --git a/examples/src/main/java/spark/examples/JavaKMeans.java b/examples/src/main/java/spark/examples/JavaKMeans.java index 626034eb0d..2d34776177 100644 --- a/examples/src/main/java/spark/examples/JavaKMeans.java +++ b/examples/src/main/java/spark/examples/JavaKMeans.java @@ -1,3 +1,20 @@ +/* + * 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.examples; import scala.Tuple2; diff --git a/examples/src/main/java/spark/examples/JavaLogQuery.java b/examples/src/main/java/spark/examples/JavaLogQuery.java index 6b22e7120c..d22684d980 100644 --- a/examples/src/main/java/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/spark/examples/JavaLogQuery.java @@ -1,3 +1,20 @@ +/* + * 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.examples; import com.google.common.collect.Lists; diff --git a/examples/src/main/java/spark/examples/JavaSparkPi.java b/examples/src/main/java/spark/examples/JavaSparkPi.java index a15a967de8..d5f42fbb38 100644 --- a/examples/src/main/java/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/spark/examples/JavaSparkPi.java @@ -1,3 +1,20 @@ +/* + * 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.examples; import spark.api.java.JavaRDD; diff --git a/examples/src/main/java/spark/examples/JavaTC.java b/examples/src/main/java/spark/examples/JavaTC.java index b319bdab44..559d7f9e53 100644 --- a/examples/src/main/java/spark/examples/JavaTC.java +++ b/examples/src/main/java/spark/examples/JavaTC.java @@ -1,3 +1,20 @@ +/* + * 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.examples; import scala.Tuple2; diff --git a/examples/src/main/java/spark/examples/JavaWordCount.java b/examples/src/main/java/spark/examples/JavaWordCount.java index 9d4c7a252d..1af370c1c3 100644 --- a/examples/src/main/java/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/spark/examples/JavaWordCount.java @@ -1,3 +1,20 @@ +/* + * 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.examples; import scala.Tuple2; diff --git a/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java index e24c6ddaa7..096a9ae219 100644 --- a/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java @@ -1,3 +1,20 @@ +/* + * 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.streaming.examples; import spark.api.java.function.Function; diff --git a/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java index 3e57580fd4..c54d3f3d59 100644 --- a/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java @@ -1,3 +1,20 @@ +/* + * 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.streaming.examples; import com.google.common.collect.Lists; diff --git a/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java index 15b82c8da1..1f4a991542 100644 --- a/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java +++ b/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java @@ -1,3 +1,20 @@ +/* + * 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.streaming.examples; import com.google.common.collect.Lists; diff --git a/examples/src/main/scala/spark/examples/BroadcastTest.scala b/examples/src/main/scala/spark/examples/BroadcastTest.scala index ba59be1687..911490cb6c 100644 --- a/examples/src/main/scala/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/spark/examples/BroadcastTest.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/CassandraTest.scala b/examples/src/main/scala/spark/examples/CassandraTest.scala index 0fe1833e83..104bfd5204 100644 --- a/examples/src/main/scala/spark/examples/CassandraTest.scala +++ b/examples/src/main/scala/spark/examples/CassandraTest.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import org.apache.hadoop.mapreduce.Job diff --git a/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala index 21a90f2e5a..67ddaec8d2 100644 --- a/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/GroupByTest.scala b/examples/src/main/scala/spark/examples/GroupByTest.scala index a6603653f1..5cee413615 100644 --- a/examples/src/main/scala/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/spark/examples/GroupByTest.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/HBaseTest.scala b/examples/src/main/scala/spark/examples/HBaseTest.scala index 6e910154d4..4dd6c243ac 100644 --- a/examples/src/main/scala/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/spark/examples/HBaseTest.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import spark._ @@ -32,4 +49,4 @@ object HBaseTest { System.exit(0) } -} \ No newline at end of file +} diff --git a/examples/src/main/scala/spark/examples/HdfsTest.scala b/examples/src/main/scala/spark/examples/HdfsTest.scala index dd61c467f7..23258336e2 100644 --- a/examples/src/main/scala/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/spark/examples/HdfsTest.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import spark._ diff --git a/examples/src/main/scala/spark/examples/LocalALS.scala b/examples/src/main/scala/spark/examples/LocalALS.scala index 2de810e062..7a449a9d72 100644 --- a/examples/src/main/scala/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/spark/examples/LocalALS.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import scala.math.sqrt diff --git a/examples/src/main/scala/spark/examples/LocalFileLR.scala b/examples/src/main/scala/spark/examples/LocalFileLR.scala index f958ef9f72..c1f8d32aa8 100644 --- a/examples/src/main/scala/spark/examples/LocalFileLR.scala +++ b/examples/src/main/scala/spark/examples/LocalFileLR.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import java.util.Random diff --git a/examples/src/main/scala/spark/examples/LocalKMeans.scala b/examples/src/main/scala/spark/examples/LocalKMeans.scala index 4849f216fb..0a0bc6f476 100644 --- a/examples/src/main/scala/spark/examples/LocalKMeans.scala +++ b/examples/src/main/scala/spark/examples/LocalKMeans.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import java.util.Random diff --git a/examples/src/main/scala/spark/examples/LocalLR.scala b/examples/src/main/scala/spark/examples/LocalLR.scala index cd73f553d6..ab99bf1fbe 100644 --- a/examples/src/main/scala/spark/examples/LocalLR.scala +++ b/examples/src/main/scala/spark/examples/LocalLR.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import java.util.Random diff --git a/examples/src/main/scala/spark/examples/LocalPi.scala b/examples/src/main/scala/spark/examples/LocalPi.scala index 9457472f2d..ccd69695df 100644 --- a/examples/src/main/scala/spark/examples/LocalPi.scala +++ b/examples/src/main/scala/spark/examples/LocalPi.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import scala.math.random diff --git a/examples/src/main/scala/spark/examples/LogQuery.scala b/examples/src/main/scala/spark/examples/LogQuery.scala index 6497596d35..e815ececf7 100644 --- a/examples/src/main/scala/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/spark/examples/LogQuery.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala index a0aaf60918..d0b1cf06e5 100644 --- a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala index 461b84a2c6..d197bbaf7c 100644 --- a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala index 435675f9de..4641b82444 100644 --- a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import spark.SparkContext diff --git a/examples/src/main/scala/spark/examples/SparkALS.scala b/examples/src/main/scala/spark/examples/SparkALS.scala index 8fb3b0fb2a..ba0dfd8f9b 100644 --- a/examples/src/main/scala/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/spark/examples/SparkALS.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import scala.math.sqrt diff --git a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala index 3d080a0257..ef6e09a8e8 100644 --- a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import java.util.Random diff --git a/examples/src/main/scala/spark/examples/SparkKMeans.scala b/examples/src/main/scala/spark/examples/SparkKMeans.scala index 4161c59fea..38ed3b149a 100644 --- a/examples/src/main/scala/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/spark/examples/SparkKMeans.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import java.util.Random diff --git a/examples/src/main/scala/spark/examples/SparkLR.scala b/examples/src/main/scala/spark/examples/SparkLR.scala index 2f41aeb376..52a0d69744 100644 --- a/examples/src/main/scala/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/spark/examples/SparkLR.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import java.util.Random diff --git a/examples/src/main/scala/spark/examples/SparkPi.scala b/examples/src/main/scala/spark/examples/SparkPi.scala index f598d2ff9c..00560ac9d1 100644 --- a/examples/src/main/scala/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/spark/examples/SparkPi.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import scala.math.random diff --git a/examples/src/main/scala/spark/examples/SparkTC.scala b/examples/src/main/scala/spark/examples/SparkTC.scala index 911ae8f168..bf988a953b 100644 --- a/examples/src/main/scala/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/spark/examples/SparkTC.scala @@ -1,3 +1,20 @@ +/* + * 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.examples import spark._ diff --git a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala index 3b847fe603..f97174aeae 100644 --- a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.examples import scala.collection.mutable.LinkedList diff --git a/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala index 39c76fd98a..3ab4fc2c37 100644 --- a/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.examples import spark.util.IntParam diff --git a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala index 9389f8a38d..f5baec242d 100644 --- a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.examples import spark.streaming.{Seconds, StreamingContext} diff --git a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala index 9202e65e09..4929703ba2 100644 --- a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.examples import java.util.Properties diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala index 704540c2bf..150fb5eb9c 100644 --- a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.examples import spark.streaming.{Seconds, StreamingContext} diff --git a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala index f450e21040..da36c8c23c 100644 --- a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.examples import spark.RDD diff --git a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala index 175281e095..7fb680bcc3 100644 --- a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala +++ b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.examples import spark.util.IntParam diff --git a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala index 51c3c9f9b4..33ab324732 100644 --- a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.examples import spark.streaming._ diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala index 528778ed72..8770abd57e 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.examples import spark.streaming.{Seconds, StreamingContext} diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala index 896e9fd8af..cba5c986be 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.examples import spark.streaming.{Seconds, StreamingContext} diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala index 65f0b6d352..682b99f75e 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.examples import spark.streaming.{Seconds, StreamingContext} diff --git a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala index 74d0d338b7..e264fae609 100644 --- a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.examples import akka.actor.ActorSystem diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala index 4c6e08bc74..375d5c9d22 100644 --- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.examples.clickstream import java.net.{InetAddress,ServerSocket,Socket,SocketException} diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala index e226a4a73a..a24342bebf 100644 --- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.examples.clickstream import spark.streaming.{Seconds, StreamingContext} diff --git a/make-distribution.sh b/make-distribution.sh index feb13d52f9..0116215163 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -1,4 +1,22 @@ #!/bin/bash + +# +# 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. +# + # # Script to create a binary distribution for easy deploys of Spark. # The distribution directory defaults to dist/ but can be overridden below. @@ -12,6 +30,7 @@ # 3) Verify master is up by visiting web page, ie http://master-ip:8080. Note the spark:// URL. # 4) ./bin/start-slave.sh 1 <> # 5) MASTER="spark://my-master-ip:7077" ./spark-shell +# # Figure out where the Spark framework is installed FWDIR="$(cd `dirname $0`; pwd)" diff --git a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala index b0e141ff32..d875d6de50 100644 --- a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala @@ -1,3 +1,20 @@ +/* + * 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 diff --git a/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala index 4fd0646160..b8f80e80cd 100644 --- a/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala @@ -1,3 +1,20 @@ +/* + * 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 diff --git a/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala index e12b3be251..89fe7d7e85 100644 --- a/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala +++ b/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala @@ -1,3 +1,20 @@ +/* + * 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 diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala index 90b0999a5e..2fb0c8136f 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala @@ -1,3 +1,20 @@ +/* + * 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 diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index eff853f379..e1b73bc25e 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -1,3 +1,20 @@ +/* + * 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} diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index ea80bfcbfd..b864fd4634 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -1,3 +1,20 @@ +/* + * 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 diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index 4c18cbdc6b..7da96397a6 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -1,3 +1,20 @@ +/* + * 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} diff --git a/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala index fb812a6dbe..38637b3dd1 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -1,3 +1,20 @@ +/* + * 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 diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index e4db7bb9b7..bb294c2257 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -1,3 +1,20 @@ +/* + * 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} diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala index 6e7c023bac..8094d22405 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala @@ -1,3 +1,20 @@ +/* + * 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 diff --git a/mllib/src/main/scala/spark/mllib/regression/Regression.scala b/mllib/src/main/scala/spark/mllib/regression/Regression.scala index f79974c191..645204ddf3 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Regression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/Regression.scala @@ -1,3 +1,20 @@ +/* + * 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 diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index 5f813df402..7c7f912b43 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -1,3 +1,20 @@ +/* + * 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} diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala index b83f505d8e..c2260ae286 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala @@ -1,3 +1,20 @@ +/* + * 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 diff --git a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala index 08a031dded..b5e564df6d 100644 --- a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala @@ -1,3 +1,20 @@ +/* + * 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} diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties index 390c92763c..a112e0b506 100644 --- a/mllib/src/test/resources/log4j.properties +++ b/mllib/src/test/resources/log4j.properties @@ -1,4 +1,21 @@ -# Set everything to be logged to the file core/target/unit-tests.log +# +# 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. +# + +# Set everything to be logged to the file core/target/unit-tests.log log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false diff --git a/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala index cb096f39a9..bebade9afb 100644 --- a/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 diff --git a/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala index 2ada9ae76b..f98590b8d9 100644 --- a/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 diff --git a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala index 04d3400cb4..bc9bfd054f 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 diff --git a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala index df41dbbdff..3c588c6162 100644 --- a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 diff --git a/pom.xml b/pom.xml index 6fa1f1d4f1..4b48072c6e 100644 --- a/pom.xml +++ b/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 org.spark-project diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 178a056de8..7a4d4c4575 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -1,3 +1,19 @@ +/* + * 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. + */ import sbt._ import sbt.Classpaths.publishTask diff --git a/project/build.properties b/project/build.properties index 9b860e23c5..08e17131f6 100644 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1,18 @@ +# +# 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. +# + sbt.version=0.12.3 diff --git a/pyspark b/pyspark index d662e90287..37a355462e 100755 --- a/pyspark +++ b/pyspark @@ -1,5 +1,22 @@ #!/usr/bin/env bash +# +# 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. +# + # Figure out where the Scala framework is installed FWDIR="$(cd `dirname $0`; pwd)" diff --git a/python/epydoc.conf b/python/epydoc.conf index 45102cd9fe..d5d5aa5454 100644 --- a/python/epydoc.conf +++ b/python/epydoc.conf @@ -1,5 +1,22 @@ [epydoc] # Epydoc section marker (required by ConfigParser) +# +# 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. +# + # Information about the project. name: PySpark url: http://spark-project.org diff --git a/python/examples/als.py b/python/examples/als.py index 010f80097f..f2b2eee64c 100755 --- a/python/examples/als.py +++ b/python/examples/als.py @@ -1,3 +1,20 @@ +# +# 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. +# + """ This example requires numpy (http://www.numpy.org/) """ diff --git a/python/examples/kmeans.py b/python/examples/kmeans.py index 72cf9f88c6..c670556f2b 100644 --- a/python/examples/kmeans.py +++ b/python/examples/kmeans.py @@ -1,3 +1,20 @@ +# +# 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. +# + """ This example requires numpy (http://www.numpy.org/) """ diff --git a/python/examples/logistic_regression.py b/python/examples/logistic_regression.py index f13698a86f..54d227d0d3 100755 --- a/python/examples/logistic_regression.py +++ b/python/examples/logistic_regression.py @@ -1,3 +1,20 @@ +# +# 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. +# + """ This example requires numpy (http://www.numpy.org/) """ diff --git a/python/examples/pi.py b/python/examples/pi.py index 127cba029b..33c026e824 100644 --- a/python/examples/pi.py +++ b/python/examples/pi.py @@ -1,3 +1,20 @@ +# +# 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. +# + import sys from random import random from operator import add diff --git a/python/examples/transitive_closure.py b/python/examples/transitive_closure.py index 7f85a1008e..40be3b5000 100644 --- a/python/examples/transitive_closure.py +++ b/python/examples/transitive_closure.py @@ -1,3 +1,20 @@ +# +# 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. +# + import sys from random import Random diff --git a/python/examples/wordcount.py b/python/examples/wordcount.py index 857160624b..41c846ba79 100644 --- a/python/examples/wordcount.py +++ b/python/examples/wordcount.py @@ -1,3 +1,20 @@ +# +# 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. +# + import sys from operator import add diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index 3e9d7d36da..d367f91967 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -1,3 +1,20 @@ +# +# 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. +# + """ >>> from pyspark.context import SparkContext >>> sc = SparkContext('local', 'test') diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index def810dd46..dfdaba274f 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -1,3 +1,20 @@ +# +# 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. +# + """ >>> from pyspark.context import SparkContext >>> sc = SparkContext('local', 'test') diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 657fe6f989..2f741cb345 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -1,3 +1,20 @@ +# +# 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. +# + import os import shutil import sys diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 78c9457b84..eb18ec08c9 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -1,3 +1,20 @@ +# +# 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. +# + import os import signal import socket diff --git a/python/pyspark/files.py b/python/pyspark/files.py index 001b7a28b6..89bcbcfe06 100644 --- a/python/pyspark/files.py +++ b/python/pyspark/files.py @@ -1,3 +1,20 @@ +# +# 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. +# + import os diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 2329e536cc..e503fb7621 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -1,3 +1,20 @@ +# +# 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. +# + import os import sys from subprocess import Popen, PIPE diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index a9fec17a9d..c6a6b24c5a 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1,3 +1,20 @@ +# +# 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. +# + from base64 import standard_b64encode as b64enc import copy from collections import defaultdict diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 5a95144983..fecacd1241 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -1,3 +1,20 @@ +# +# 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. +# + import struct import cPickle diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 54ff1bf8e7..cc8cd9e3c4 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -1,3 +1,20 @@ +# +# 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. +# + """ An interactive shell. diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 1e34d47365..dfd841b10a 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -1,3 +1,20 @@ +# +# 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. +# + """ Unit tests for PySpark; additional tests are implemented as doctests in individual modules. diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 379bbfd4c2..75d692beeb 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -1,3 +1,20 @@ +# +# 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. +# + """ Worker that receives input from Piped RDD. """ diff --git a/python/run-tests b/python/run-tests index a3a9ff5dcb..1ee947d414 100755 --- a/python/run-tests +++ b/python/run-tests @@ -1,6 +1,24 @@ #!/usr/bin/env bash -# Figure out where the Scala framework is installed +# +# 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. +# + + +# Figure out where the Spark framework is installed FWDIR="$(cd `dirname $0`; cd ../; pwd)" FAILED=0 diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 7a7280313e..7c4e722cc1 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 diff --git a/repl-bin/src/deb/bin/run b/repl-bin/src/deb/bin/run index c54c9e97a0..0e5100e00b 100755 --- a/repl-bin/src/deb/bin/run +++ b/repl-bin/src/deb/bin/run @@ -1,6 +1,23 @@ #!/bin/bash -SCALA_VERSION=2.9.2 +# +# 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. +# + +SCALA_VERSION=2.9.3 # Figure out where the Scala framework is installed FWDIR="$(cd `dirname $0`; pwd)" diff --git a/repl-bin/src/deb/bin/spark-executor b/repl-bin/src/deb/bin/spark-executor index 47b9cccdfe..d3c1c23413 100755 --- a/repl-bin/src/deb/bin/spark-executor +++ b/repl-bin/src/deb/bin/spark-executor @@ -1,5 +1,22 @@ #!/bin/bash +# +# 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. +# + FWDIR="$(cd `dirname $0`; pwd)" echo "Running spark-executor with framework dir = $FWDIR" exec $FWDIR/run spark.executor.MesosExecutorBackend diff --git a/repl-bin/src/deb/bin/spark-shell b/repl-bin/src/deb/bin/spark-shell index 219c66eb0b..8b258543db 100755 --- a/repl-bin/src/deb/bin/spark-shell +++ b/repl-bin/src/deb/bin/spark-shell @@ -1,4 +1,21 @@ #!/bin/bash +# +# 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. +# + FWDIR="$(cd `dirname $0`; pwd)" exec $FWDIR/run spark.repl.Main "$@" diff --git a/repl/pom.xml b/repl/pom.xml index 92a2020b48..7d8da03254 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 diff --git a/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala index 0e9aa863b5..274bc585db 100644 --- a/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala @@ -1,3 +1,20 @@ +/* + * 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.repl import java.io.{ByteArrayOutputStream, InputStream} diff --git a/repl/src/main/scala/spark/repl/Main.scala b/repl/src/main/scala/spark/repl/Main.scala index 58809ab646..d824d62fd1 100644 --- a/repl/src/main/scala/spark/repl/Main.scala +++ b/repl/src/main/scala/spark/repl/Main.scala @@ -1,3 +1,20 @@ +/* + * 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.repl import scala.collection.mutable.Set diff --git a/repl/src/test/resources/log4j.properties b/repl/src/test/resources/log4j.properties index cfb1a390e6..a6d33e69d2 100644 --- a/repl/src/test/resources/log4j.properties +++ b/repl/src/test/resources/log4j.properties @@ -1,4 +1,21 @@ -# Set everything to be logged to the repl/target/unit-tests.log +# +# 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. +# + +# Set everything to be logged to the repl/target/unit-tests.log log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false diff --git a/repl/src/test/scala/spark/repl/ReplSuite.scala b/repl/src/test/scala/spark/repl/ReplSuite.scala index f46e6d8be4..80ae605558 100644 --- a/repl/src/test/scala/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/spark/repl/ReplSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.repl import java.io._ diff --git a/run b/run index 6b5bc01a51..72ee76d722 100755 --- a/run +++ b/run @@ -1,5 +1,22 @@ #!/bin/bash +# +# 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. +# + # Figure out where the Scala framework is installed FWDIR="$(cd `dirname $0`; pwd)" diff --git a/run.cmd b/run.cmd index cc5605f8a9..c91764e617 100644 --- a/run.cmd +++ b/run.cmd @@ -1,2 +1,20 @@ @echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + cmd /V /E /C %~dp0run2.cmd %* diff --git a/run2.cmd b/run2.cmd index a9c4df180f..dc5e50931e 100644 --- a/run2.cmd +++ b/run2.cmd @@ -1,5 +1,22 @@ @echo off +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + set SCALA_VERSION=2.9.3 rem Figure out where the Spark framework is installed diff --git a/sbt/sbt b/sbt/sbt index 523fbb346b..8c9a42824f 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -1,8 +1,28 @@ #!/bin/bash + +# +# 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. +# + EXTRA_ARGS="" if [ "$MESOS_HOME" != "" ]; then EXTRA_ARGS="-Djava.library.path=$MESOS_HOME/lib/java" fi + export SPARK_HOME=$(cd "$(dirname $0)/.."; pwd) export SPARK_TESTING=1 # To put test classes on classpath + java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=128m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" diff --git a/sbt/sbt.cmd b/sbt/sbt.cmd index ce3ae70174..56a16d77f2 100644 --- a/sbt/sbt.cmd +++ b/sbt/sbt.cmd @@ -1,5 +1,25 @@ @echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + set EXTRA_ARGS= if not "%MESOS_HOME%x"=="x" set EXTRA_ARGS=-Djava.library.path=%MESOS_HOME%\lib\java + set SPARK_HOME=%~dp0.. + java -Xmx1200M -XX:MaxPermSize=200m %EXTRA_ARGS% -jar %SPARK_HOME%\sbt\sbt-launch-0.11.3-2.jar "%*" diff --git a/spark-executor b/spark-executor index b66c374ca8..feccbf5cc2 100755 --- a/spark-executor +++ b/spark-executor @@ -1,4 +1,22 @@ #!/bin/sh + +# +# 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. +# + FWDIR="`dirname $0`" echo "Running spark-executor with framework dir = $FWDIR" exec $FWDIR/run spark.executor.MesosExecutorBackend diff --git a/spark-shell b/spark-shell index a8e72143fb..5371fc540a 100755 --- a/spark-shell +++ b/spark-shell @@ -1,4 +1,22 @@ #!/bin/bash --posix + +# +# 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. +# + # # Shell script for starting the Spark Shell REPL # Note that it will set MASTER to spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT} @@ -6,6 +24,7 @@ # Options: # -c Set the number of cores for REPL to use # + FWDIR="`dirname $0`" for o in "$@"; do diff --git a/spark-shell.cmd b/spark-shell.cmd index 34697d52d7..ec65eabb74 100644 --- a/spark-shell.cmd +++ b/spark-shell.cmd @@ -1,4 +1,22 @@ @echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + set FWDIR=%~dp0 set SPARK_LAUNCH_WITH_SCALA=1 cmd /V /E /C %FWDIR%run2.cmd spark.repl.Main %* diff --git a/streaming/pom.xml b/streaming/pom.xml index 2fb5bbdeb5..7e6b06d772 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala index 450e48d66e..1e4c1e3742 100644 --- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming import spark.{Logging, Utils} diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala index 9be7926a4a..684d3abb56 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/spark/streaming/DStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming import spark.streaming.dstream._ diff --git a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala index 6b0fade7c6..399ca1c63d 100644 --- a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming import org.apache.hadoop.fs.Path diff --git a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala index 3b331956f5..c09a332d44 100644 --- a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming import dstream.InputDStream diff --git a/streaming/src/main/scala/spark/streaming/Duration.scala b/streaming/src/main/scala/spark/streaming/Duration.scala index c2135195d8..12a14e233d 100644 --- a/streaming/src/main/scala/spark/streaming/Duration.scala +++ b/streaming/src/main/scala/spark/streaming/Duration.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming import spark.Utils diff --git a/streaming/src/main/scala/spark/streaming/Interval.scala b/streaming/src/main/scala/spark/streaming/Interval.scala index 6a8b81760e..b30cd969e9 100644 --- a/streaming/src/main/scala/spark/streaming/Interval.scala +++ b/streaming/src/main/scala/spark/streaming/Interval.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming private[streaming] diff --git a/streaming/src/main/scala/spark/streaming/Job.scala b/streaming/src/main/scala/spark/streaming/Job.scala index 67bd8388bc..ceb3f92b65 100644 --- a/streaming/src/main/scala/spark/streaming/Job.scala +++ b/streaming/src/main/scala/spark/streaming/Job.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming import java.util.concurrent.atomic.AtomicLong diff --git a/streaming/src/main/scala/spark/streaming/JobManager.scala b/streaming/src/main/scala/spark/streaming/JobManager.scala index 7696c4a592..a31230689f 100644 --- a/streaming/src/main/scala/spark/streaming/JobManager.scala +++ b/streaming/src/main/scala/spark/streaming/JobManager.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming import spark.Logging diff --git a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala index b159d26c02..d4cf2e568c 100644 --- a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala +++ b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming import spark.streaming.dstream.{NetworkInputDStream, NetworkReceiver} diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala index 8d0a83d439..47bf07bee1 100644 --- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming import spark.streaming.StreamingContext._ diff --git a/streaming/src/main/scala/spark/streaming/Scheduler.scala b/streaming/src/main/scala/spark/streaming/Scheduler.scala index 1c4b22a898..252cc2a303 100644 --- a/streaming/src/main/scala/spark/streaming/Scheduler.scala +++ b/streaming/src/main/scala/spark/streaming/Scheduler.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming import util.{ManualClock, RecurringTimer, Clock} diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 36b841af8f..ffd656227d 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming import akka.actor.Props diff --git a/streaming/src/main/scala/spark/streaming/Time.scala b/streaming/src/main/scala/spark/streaming/Time.scala index f14decf08b..ad5eab9dd2 100644 --- a/streaming/src/main/scala/spark/streaming/Time.scala +++ b/streaming/src/main/scala/spark/streaming/Time.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming /** @@ -52,4 +69,4 @@ case class Time(private val millis: Long) { object Time { val ordering = Ordering.by((time: Time) => time.millis) -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala index 4d93f0a5f7..7dcb1d713d 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.api.java import spark.streaming.{Duration, Time, DStream} @@ -82,4 +99,4 @@ class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassM object JavaDStream { implicit def fromDStream[T: ClassManifest](dstream: DStream[T]): JavaDStream[T] = new JavaDStream[T](dstream) -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala index 548809a359..3ab5c1fdde 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.api.java import java.util.{List => JList} @@ -296,4 +313,4 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T def checkpoint(interval: Duration) = { dstream.checkpoint(interval) } -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala index 30240cad98..ccd15563b0 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.api.java import java.util.{List => JList} diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala index ed7b789d98..b7720ad0ea 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.api.java import spark.streaming._ diff --git a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala index 4ef4bb7de1..99553d295d 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.{RDD, Partitioner} diff --git a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala index 41c3af4694..095137092a 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.RDD @@ -16,4 +33,4 @@ class ConstantInputDStream[T: ClassManifest](ssc_ : StreamingContext, rdd: RDD[T override def compute(validTime: Time): Option[RDD[T]] = { Some(rdd) } -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala index 41b9bd9461..de0536125d 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.RDD diff --git a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala index e993164f99..9d8c5c3175 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala index cabd34f5f2..78d7117f0f 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala index a69af60589..d13bebb10f 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala index c9644b3a83..4906f503c2 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.streaming.StreamingContext diff --git a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala index ee69ea5177..7df537eb56 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.RDD diff --git a/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala index b589cbd4d5..06fda6fe8e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala index 3c5d43a609..4dbdec459d 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.streaming.{Time, Duration, StreamingContext, DStream} diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index 55d2957be4..6ee588af15 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.Logging diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala index 848afecfad..af41a1b9ac 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala index 6055aa6a05..8d8a6161c6 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala index 20818a0cab..3fda84a38a 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala index 122a529bb7..344b41c4d0 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.streaming.{Time, StreamingContext, AddBlocks, RegisterReceiver, DeregisterReceiver} diff --git a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala index 3c2a81947b..33f7cd063f 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.streaming.StreamingContext diff --git a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala index da224ad6f7..b269061b73 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.RDD diff --git a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala index 1b2fa56779..236f74f575 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.Logging diff --git a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala index 343b6915e7..96260501ab 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.streaming.StreamingContext._ diff --git a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala index 1f9548bfb8..83b57b27f7 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.{RDD, Partitioner} diff --git a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala index 1408af0afa..5877b10e0e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.streaming.StreamingContext diff --git a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala index db62955036..4b46613d5e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.RDD diff --git a/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala index 99660d9dee..e7fbc5bbcf 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.RDD diff --git a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala index ff7a58be45..f09a8b9f90 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark._ diff --git a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala index 00bad5da34..3eaa9a7e7f 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala index cbf0c88108..fd24d61730 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.dstream import spark.RDD diff --git a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala index 036c95a860..2d9937eab8 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.receivers import akka.actor.{ Actor, PoisonPill, Props, SupervisorStrategy } diff --git a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala index 5533c3cf1e..22d554e7e4 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala +++ b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.receivers import akka.actor.Actor diff --git a/streaming/src/main/scala/spark/streaming/util/Clock.scala b/streaming/src/main/scala/spark/streaming/util/Clock.scala index 974651f9f6..d9ac722df5 100644 --- a/streaming/src/main/scala/spark/streaming/util/Clock.scala +++ b/streaming/src/main/scala/spark/streaming/util/Clock.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.util private[streaming] diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala index 426a9b6f71..8ce5d8daf5 100644 --- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.util import spark.{Logging, RDD} diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala index 03749d4a94..bf04120293 100644 --- a/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala +++ b/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.util import spark.SparkContext diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala index bd0b0e74c1..5cc6ad9dee 100644 --- a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala +++ b/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.util import java.nio.ByteBuffer diff --git a/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala index 8e10276deb..7ecc44236d 100644 --- a/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala +++ b/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming.util private[streaming] diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java index 4cf10582a9..3b93790baa 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -1,3 +1,20 @@ +/* + * 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.streaming; import com.google.common.base.Optional; diff --git a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/spark/streaming/JavaTestUtils.scala index 64a7e7cbf9..f9d25db8da 100644 --- a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala +++ b/streaming/src/test/java/spark/streaming/JavaTestUtils.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming import collection.mutable.{SynchronizedBuffer, ArrayBuffer} @@ -64,4 +81,4 @@ object JavaTestUtils extends JavaTestBase { object JavaCheckpointTestUtils extends JavaTestBase { override def actuallyWait = true -} \ No newline at end of file +} diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties index 59c445e63f..063529a9cb 100644 --- a/streaming/src/test/resources/log4j.properties +++ b/streaming/src/test/resources/log4j.properties @@ -1,4 +1,21 @@ -# Set everything to be logged to the file streaming/target/unit-tests.log +# +# 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. +# + +# Set everything to be logged to the file streaming/target/unit-tests.log log4j.rootCategory=INFO, file # log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala index 565089a853..67e3e0cd30 100644 --- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming import spark.streaming.StreamingContext._ diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala index 607dea77ec..8c639648f0 100644 --- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming import dstream.FileInputDStream diff --git a/streaming/src/test/scala/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/spark/streaming/FailureSuite.scala index a5fa7ab92d..7fc649fe27 100644 --- a/streaming/src/test/scala/spark/streaming/FailureSuite.scala +++ b/streaming/src/test/scala/spark/streaming/FailureSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming import spark.Logging diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala index b024fc9dcc..1c5419b16d 100644 --- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming import akka.actor.Actor diff --git a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala index ad6aa79d10..cb34b5a7cc 100644 --- a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming import spark.streaming.dstream.{InputDStream, ForEachDStream} diff --git a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala index 80d827706f..894b765fc6 100644 --- a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala +++ b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.streaming import spark.streaming.StreamingContext._ -- cgit v1.2.3 From 4ff494de20c36151dc29a60825d67e094d14acd4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 17:26:48 -0700 Subject: Some missing license headers --- bin/compute-classpath.cmd | 17 +++++++++++++++++ bin/compute-classpath.sh | 17 +++++++++++++++++ 2 files changed, 34 insertions(+) diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 44826f339c..835d1af794 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -1,5 +1,22 @@ @echo off +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run" rem script and the ExecutorRunner in standalone cluster mode. diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index bd48b43da3..4c539649a2 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -1,5 +1,22 @@ #!/bin/bash +# +# 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. +# + # This script computes Spark's classpath and prints it to stdout; it's used by both the "run" # script and the ExecutorRunner in standalone cluster mode. -- cgit v1.2.3 From a1d2c3434657f0d901a2f8cfddf62065b5ad4385 Mon Sep 17 00:00:00 2001 From: ctn Date: Wed, 17 Jul 2013 11:27:11 -0700 Subject: [BUGFIX] Fix for sbt/sbt script SPARK_HOME setting In some environments, this command export SPARK_HOME=$(cd "$(dirname $0)/.."; pwd) echoes two paths, one by the "cd ..", and one by the "pwd". Note the resulting erroneous -jar paths below: ctn@ubuntu:~/src/spark$ sbt/sbt + EXTRA_ARGS= + '[' '' '!=' '' ']' +++ dirname sbt/sbt ++ cd sbt/.. ++ pwd + export 'SPARK_HOME=/home/ctn/src/spark /home/ctn/src/spark' + SPARK_HOME='/home/ctn/src/spark /home/ctn/src/spark' + export SPARK_TESTING=1 + SPARK_TESTING=1 + java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=128m -jar /home/ctn/src/spark /home/ctn/src/spark/sbt/sbt-launch-0.11.3-2.jar Error: Invalid or corrupt jarfile /home/ctn/src/spark Committer: ctn On branch master Changes to be committed: - Send output of the "cd .." part to /dev/null modified: sbt/sbt --- sbt/sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbt/sbt b/sbt/sbt index 8c9a42824f..d64645e32d 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -22,7 +22,7 @@ if [ "$MESOS_HOME" != "" ]; then EXTRA_ARGS="-Djava.library.path=$MESOS_HOME/lib/java" fi -export SPARK_HOME=$(cd "$(dirname $0)/.."; pwd) +export SPARK_HOME=$(cd "$(dirname $0)/.." 2>&1 >/dev/null ; pwd) export SPARK_TESTING=1 # To put test classes on classpath java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=128m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" -- cgit v1.2.3 From 84fa20c2a135f54745ddde9abb4f5e60af8856d1 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 14:04:05 -0700 Subject: Allow initial weight vectors in LogisticRegression. Also move LogisticGradient to the LogisticRegression file and fix the unit tests log path. --- .../scala/spark/mllib/optimization/Gradient.scala | 18 ------- .../spark/mllib/optimization/GradientDescent.scala | 5 +- .../mllib/regression/LogisticRegression.scala | 61 +++++++++++++++++++++- mllib/src/test/resources/log4j.properties | 2 +- .../mllib/regression/LogisticRegressionSuite.scala | 52 ++++++++++++++---- 5 files changed, 106 insertions(+), 32 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala index 2fb0c8136f..d5338360c8 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala @@ -30,21 +30,3 @@ abstract class Gradient extends Serializable { def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): (DoubleMatrix, Double) } - -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) - } -} diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index e1b73bc25e..2ac0808357 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -48,16 +48,17 @@ object GradientDescent { updater: Updater, stepSize: Double, numIters: Int, + initialWeights: Array[Double], miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { val lossHistory = new ArrayBuffer[Double](numIters) - val nfeatures: Int = data.take(1)(0)._2.length val nexamples: Long = data.count() val miniBatchSize = nexamples * miniBatchFraction // Initialize weights as a column matrix - var weights = DoubleMatrix.ones(nfeatures) + var weights = new DoubleMatrix(1, initialWeights.length, + initialWeights:_*) var reg_val = 0.0 for (i <- 1 to numIters) { diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index bb294c2257..dab15aa386 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -46,6 +46,24 @@ class LogisticRegressionModel( } } +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) + } +} + class LogisticRegression private (var stepSize: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -80,13 +98,30 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D } def train(input: RDD[(Double, Array[Double])]): LogisticRegressionModel = { + val nfeatures: Int = input.take(1)(0)._2.length + val initialWeights = Array.fill(nfeatures)(1.0) + train(input, initialWeights) + } + + def train( + input: RDD[(Double, Array[Double])], + initialWeights: Array[Double]): LogisticRegressionModel = { + // Add a extra variable consisting of all 1.0's for the intercept. val data = input.map { case (y, features) => (y, Array(1.0, features:_*)) } + val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) + val (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, miniBatchFraction) + data, + new LogisticGradient(), + new SimpleUpdater(), + stepSize, + numIters, + initalWeightsWithIntercept, + miniBatchFraction) val weightsScaled = weights.getRange(1, weights.length) val intercept = weights.get(0) @@ -105,6 +140,30 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D */ object LogisticRegression { + /** + * 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. + * + * @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[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double, + initialWeights: Array[Double]) + : LogisticRegressionModel = + { + new LogisticRegression(stepSize, miniBatchFraction, numIterations).train(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 diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties index a112e0b506..4265ba6e5d 100644 --- a/mllib/src/test/resources/log4j.properties +++ b/mllib/src/test/resources/log4j.properties @@ -19,7 +19,7 @@ log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false -log4j.appender.file.file=ml/target/unit-tests.log +log4j.appender.file.file=mllib/target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n diff --git a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala index bc9bfd054f..2ff248d256 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala @@ -34,16 +34,14 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } - // Test if we can correctly learn A, B where Y = logistic(A + B*X) - test("logistic regression") { - val nPoints = 10000 + // Generate input of the form Y = logistic(offset + scale*X) + def generateLogisticInput( + offset: Double, + scale: Double, + nPoints: Int) : Seq[(Double, Array[Double])] = { val rnd = new Random(42) - val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - val A = 2.0 - val B = -1.5 - // 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 => @@ -51,14 +49,24 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { math.log(u) - math.log(1.0-u) } - // y <- A + B*x + rlogis(100) + // y <- A + B*x + rLogis() // y <- as.numeric(y > 0) val y = (0 until nPoints).map { i => - val yVal = A + B * x1(i) + rLogis(i) + val yVal = offset + scale * x1(i) + rLogis(i) if (yVal > 0) 1.0 else 0.0 } - val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i)))).toArray + val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i)))) + testData + } + + // 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 = generateLogisticInput(A, B, nPoints) val testRDD = sc.parallelize(testData, 2) testRDD.cache() @@ -71,4 +79,28 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { 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]") } + + test("logistic regression with initial weights") { + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = generateLogisticInput(A, B, nPoints) + + 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 LogisticRegression().setStepSize(10.0) + .setNumIterations(10) + + val model = lr.train(testRDD, initialWeights) + + val weight0 = model.weights.get(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]") + } } -- cgit v1.2.3 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 3bf989713654129ad35a80309d1b354ca5ddd06c Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 14:20:24 -0700 Subject: Rename loss -> stochasticLoss and add a note to explain why we have multiple train methods. --- .../main/scala/spark/mllib/optimization/GradientDescent.scala | 9 +++++---- .../main/scala/spark/mllib/regression/LogisticRegression.scala | 10 ++++++---- .../main/scala/spark/mllib/regression/RidgeRegression.scala | 2 ++ 3 files changed, 13 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index 2ac0808357..77f5a7ae24 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -40,7 +40,8 @@ object GradientDescent { * one iteration of SGD. Default value 1.0. * * @return weights - Column matrix containing weights for every feature. - * @return lossHistory - Array containing the loss computed for every iteration. + * @return stochasticLossHistory - Array containing the stochastic loss computed for + * every iteration. */ def runMiniBatchSGD( data: RDD[(Double, Array[Double])], @@ -51,7 +52,7 @@ object GradientDescent { initialWeights: Array[Double], miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { - val lossHistory = new ArrayBuffer[Double](numIters) + val stochasticLossHistory = new ArrayBuffer[Double](numIters) val nexamples: Long = data.count() val miniBatchSize = nexamples * miniBatchFraction @@ -69,12 +70,12 @@ object GradientDescent { (grad, loss) }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) - lossHistory.append(lossSum / miniBatchSize + reg_val) + stochasticLossHistory.append(lossSum / miniBatchSize + reg_val) val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i) weights = update._1 reg_val = update._2 } - (weights, lossHistory.toArray) + (weights, stochasticLossHistory.toArray) } } diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index dab15aa386..664baf33a3 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -30,7 +30,7 @@ import org.jblas.DoubleMatrix class LogisticRegressionModel( val weights: DoubleMatrix, val intercept: Double, - val losses: Array[Double]) extends RegressionModel { + val stochasticLosses: Array[Double]) extends RegressionModel { override def predict(testData: spark.RDD[Array[Double]]) = { testData.map { x => @@ -114,7 +114,7 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) - val (weights, losses) = GradientDescent.runMiniBatchSGD( + val (weights, stochasticLosses) = GradientDescent.runMiniBatchSGD( data, new LogisticGradient(), new SimpleUpdater(), @@ -126,17 +126,19 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D val weightsScaled = weights.getRange(1, weights.length) val intercept = weights.get(0) - val model = new LogisticRegressionModel(weightsScaled, intercept, losses) + val model = new LogisticRegressionModel(weightsScaled, intercept, stochasticLosses) logInfo("Final model weights " + model.weights) logInfo("Final model intercept " + model.intercept) - logInfo("Last 10 losses " + model.losses.takeRight(10).mkString(", ")) + logInfo("Last 10 stochastic losses " + model.stochasticLosses.takeRight(10).mkString(", ")) model } } /** * Top-level methods for calling Logistic Regression. + * NOTE(shivaram): We use multiple train methods instead of default arguments to support + * Java programs. */ object LogisticRegression { diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index 7c7f912b43..f724edd732 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -164,6 +164,8 @@ class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) /** * Top-level methods for calling Ridge Regression. + * NOTE(shivaram): We use multiple train methods instead of default arguments to support + * Java programs. */ object RidgeRegression { -- cgit v1.2.3 From 45f3c855181539306d5610c5aa265f24b431c142 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 16:03:29 -0700 Subject: Change weights to be Array[Double] in LR model. Also ensure weights are initialized to a column vector. --- .../spark/mllib/optimization/GradientDescent.scala | 5 ++--- .../spark/mllib/regression/LogisticRegression.scala | 17 +++++++++++------ .../mllib/regression/LogisticRegressionSuite.scala | 4 ++-- 3 files changed, 15 insertions(+), 11 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index 77f5a7ae24..2c5038757b 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -57,9 +57,8 @@ object GradientDescent { val nexamples: Long = data.count() val miniBatchSize = nexamples * miniBatchFraction - // Initialize weights as a column matrix - var weights = new DoubleMatrix(1, initialWeights.length, - initialWeights:_*) + // Initialize weights as a column vector + var weights = new DoubleMatrix(initialWeights.length, 1, initialWeights:_*) var reg_val = 0.0 for (i <- 1 to numIters) { diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index 664baf33a3..ab865af0c6 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -28,20 +28,23 @@ import org.jblas.DoubleMatrix * Based on Matlab code written by John Duchi. */ class LogisticRegressionModel( - val weights: DoubleMatrix, + val weights: Array[Double], val intercept: Double, val stochasticLosses: Array[Double]) extends RegressionModel { + // Create a column vector that can be used for predictions + private val weightsMatrix = new DoubleMatrix(weights.length, 1, weights:_*) + override def predict(testData: spark.RDD[Array[Double]]) = { testData.map { x => - val margin = new DoubleMatrix(1, x.length, x:_*).mmul(this.weights).get(0) + this.intercept + val margin = new DoubleMatrix(1, x.length, x:_*).mmul(weightsMatrix).get(0) + this.intercept 1.0/ (1.0 + math.exp(margin * -1)) } } override def predict(testData: Array[Double]): Double = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) - val margin = dataMat.mmul(this.weights).get(0) + this.intercept + val margin = dataMat.mmul(weightsMatrix).get(0) + this.intercept 1.0/ (1.0 + math.exp(margin * -1)) } } @@ -123,12 +126,14 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D initalWeightsWithIntercept, miniBatchFraction) - val weightsScaled = weights.getRange(1, weights.length) - val intercept = weights.get(0) + val weightsArray = weights.toArray() + + val intercept = weightsArray(0) + val weightsScaled = weightsArray.tail val model = new LogisticRegressionModel(weightsScaled, intercept, stochasticLosses) - logInfo("Final model weights " + model.weights) + logInfo("Final model weights " + model.weights.mkString(",")) logInfo("Final model intercept " + model.intercept) logInfo("Last 10 stochastic losses " + model.stochasticLosses.takeRight(10).mkString(", ")) model diff --git a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala index 2ff248d256..47191d9a5a 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala @@ -75,7 +75,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val model = lr.train(testRDD) - val weight0 = model.weights.get(0) + 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]") } @@ -99,7 +99,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val model = lr.train(testRDD, initialWeights) - val weight0 = model.weights.get(0) + 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]") } -- cgit v1.2.3 From 217667174e267adba5469cf26b3e4418e3d1cc90 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 16:08:34 -0700 Subject: Return Array[Double] from SGD instead of DoubleMatrix --- mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala | 4 ++-- .../src/main/scala/spark/mllib/regression/LogisticRegression.scala | 6 ++---- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index 2c5038757b..4c996c0903 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -50,7 +50,7 @@ object GradientDescent { stepSize: Double, numIters: Int, initialWeights: Array[Double], - miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { + miniBatchFraction: Double=1.0) : (Array[Double], Array[Double]) = { val stochasticLossHistory = new ArrayBuffer[Double](numIters) @@ -75,6 +75,6 @@ object GradientDescent { reg_val = update._2 } - (weights, stochasticLossHistory.toArray) + (weights.toArray, stochasticLossHistory.toArray) } } diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index ab865af0c6..711e205c39 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -126,10 +126,8 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D initalWeightsWithIntercept, miniBatchFraction) - val weightsArray = weights.toArray() - - val intercept = weightsArray(0) - val weightsScaled = weightsArray.tail + val intercept = weights(0) + val weightsScaled = weights.tail val model = new LogisticRegressionModel(weightsScaled, intercept, stochasticLosses) -- cgit v1.2.3 From 88a0823c58304fa1b290667d95a482105a8a5f52 Mon Sep 17 00:00:00 2001 From: Ubuntu Date: Thu, 18 Jul 2013 00:51:18 +0000 Subject: Consistently invoke bash with /usr/bin/env bash in scripts to make code more portable (JIRA Ticket SPARK-817) --- bin/compute-classpath.sh | 2 +- docs/spark-simple-tutorial.md | 2 +- ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh | 2 +- ec2/deploy.generic/root/spark-ec2/ec2-variables.sh | 2 +- repl-bin/src/deb/bin/run | 2 +- repl-bin/src/deb/bin/spark-executor | 2 +- repl-bin/src/deb/bin/spark-shell | 2 +- run | 2 +- sbt/sbt | 2 +- spark-shell | 5 ++++- 10 files changed, 13 insertions(+), 10 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 4c539649a2..e6fdc5306e 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/docs/spark-simple-tutorial.md b/docs/spark-simple-tutorial.md index 9875de62bd..fbdbc7d19d 100644 --- a/docs/spark-simple-tutorial.md +++ b/docs/spark-simple-tutorial.md @@ -13,7 +13,7 @@ title: Tutorial - Running a Simple Spark Application 3. Edit the ~/SparkTest/sbt/sbt file to look like this: - #!/bin/bash + #!/usr/bin/env bash java -Xmx800M -XX:MaxPermSize=150m -jar $(dirname $0)/sbt-launch-*.jar "$@" 4. To build a Spark application, you need Spark and its dependencies in a single Java archive (JAR) file. Create this JAR in Spark's main directory with sbt as: diff --git a/ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh b/ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh index 50ecf83404..ede6c78428 100644 --- a/ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh +++ b/ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # These variables are automatically filled in by the mesos-ec2 script. export MESOS_MASTERS="{{master_list}}" diff --git a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh index 166a884c88..685ed8be8c 100644 --- a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh +++ b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # These variables are automatically filled in by the mesos-ec2 script. export MESOS_MASTERS="{{master_list}}" diff --git a/repl-bin/src/deb/bin/run b/repl-bin/src/deb/bin/run index 0e5100e00b..8b5d8300f2 100755 --- a/repl-bin/src/deb/bin/run +++ b/repl-bin/src/deb/bin/run @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/repl-bin/src/deb/bin/spark-executor b/repl-bin/src/deb/bin/spark-executor index d3c1c23413..bcfae22677 100755 --- a/repl-bin/src/deb/bin/spark-executor +++ b/repl-bin/src/deb/bin/spark-executor @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/repl-bin/src/deb/bin/spark-shell b/repl-bin/src/deb/bin/spark-shell index 8b258543db..ec7e33e1e3 100755 --- a/repl-bin/src/deb/bin/spark-shell +++ b/repl-bin/src/deb/bin/spark-shell @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/run b/run index 72ee76d722..f49aa92796 100755 --- a/run +++ b/run @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/sbt/sbt b/sbt/sbt index 8c9a42824f..b6f91f5085 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -1,4 +1,4 @@ -#!/bin/bash +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more diff --git a/spark-shell b/spark-shell index 5371fc540a..31a4138124 100755 --- a/spark-shell +++ b/spark-shell @@ -1,4 +1,4 @@ -#!/bin/bash --posix +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one or more @@ -25,6 +25,9 @@ # -c Set the number of cores for REPL to use # +# Enter posix mode for bash +set -o posix + FWDIR="`dirname $0`" for o in "$@"; do -- cgit v1.2.3 From 3aad45265324537e51cab79db3f09d9310339f77 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 19 Jul 2013 02:29:46 +0800 Subject: fix a bug in build process that pulls in two versionf of ASM. --- project/SparkBuild.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index e9819b877d..59edda5097 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -183,10 +183,10 @@ object SparkBuild extends Build { if (HADOOP_YARN) { Seq( // Exclude rule required for all ? - "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty), - "org.apache.hadoop" % "hadoop-yarn-api" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty), - "org.apache.hadoop" % "hadoop-yarn-common" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty), - "org.apache.hadoop" % "hadoop-yarn-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty) + "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-api" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-common" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm) ) } else { Seq( -- 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 From 81bb5dc64007121df52cbafdf5f025c97f145953 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 19 Jul 2013 14:08:30 -0700 Subject: Creates Executors tab for application with RDD block and memory/disk used, solves SPARK-808 --- core/src/main/scala/spark/executor/Executor.scala | 1 + .../main/scala/spark/executor/TaskMetrics.scala | 5 ++ core/src/main/scala/spark/ui/Page.scala | 2 +- core/src/main/scala/spark/ui/SparkUI.scala | 5 +- core/src/main/scala/spark/ui/UIUtils.scala | 5 ++ .../src/main/scala/spark/ui/exec/ExecutorsUI.scala | 79 ++++++++++++++++++++++ .../main/scala/spark/ui/storage/IndexPage.scala | 18 +---- 7 files changed, 96 insertions(+), 19 deletions(-) create mode 100644 core/src/main/scala/spark/ui/exec/ExecutorsUI.scala diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 2e81151882..82e5f5a015 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -126,6 +126,7 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert val taskFinish = System.currentTimeMillis() task.metrics.foreach{ m => m.hostname = Utils.localHostName + m.executorId = executorId m.executorDeserializeTime = (taskStart - startTime).toInt m.executorRunTime = (taskFinish - taskStart).toInt } diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index 3151627839..e14dc57501 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -23,6 +23,11 @@ class TaskMetrics extends Serializable { */ var hostname: String = _ + /** + * Executor's ID the task runs on + */ + var executorId: String = _ + /** * Time taken on the executor to deserialize this task */ diff --git a/core/src/main/scala/spark/ui/Page.scala b/core/src/main/scala/spark/ui/Page.scala index a31e750d06..03034a4520 100644 --- a/core/src/main/scala/spark/ui/Page.scala +++ b/core/src/main/scala/spark/ui/Page.scala @@ -17,4 +17,4 @@ package spark.ui -private[spark] object Page extends Enumeration { val Storage, Jobs, Environment = Value } +private[spark] object Page extends Enumeration { val Storage, Jobs, Environment, Executors = Value } diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 9396f22063..1534705bc5 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -23,6 +23,7 @@ import org.eclipse.jetty.server.{Handler, Server} import spark.{Logging, SparkContext, Utils} import spark.ui.env.EnvironmentUI +import spark.ui.exec.ExecutorsUI import spark.ui.storage.BlockManagerUI import spark.ui.jobs.JobProgressUI import spark.ui.JettyUtils._ @@ -41,7 +42,9 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { val storage = new BlockManagerUI(sc) val jobs = new JobProgressUI(sc) val env = new EnvironmentUI(sc) - val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ handlers + val exec = new ExecutorsUI(sc) + val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ + exec.getHandlers ++ handlers /** Bind the HTTP server which backs this web interface */ def bind() { diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala index b1d11954dd..e33c80282a 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -40,6 +40,10 @@ private[spark] object UIUtils { case Environment =>
      • Environment
      • case _ =>
      • Environment
      • } + val executors = page match { + case Executors =>
      • Executors
      • + case _ =>
      • Executors
      • + } @@ -66,6 +70,7 @@ private[spark] object UIUtils { {storage} {jobs} {environment} + {executors}
      • Application: {sc.appName}
      • diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala new file mode 100644 index 0000000000..de0e571553 --- /dev/null +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -0,0 +1,79 @@ +package spark.ui.exec + + +import javax.servlet.http.HttpServletRequest + +import org.eclipse.jetty.server.Handler + +import scala.util.Properties + +import spark.scheduler.cluster.TaskInfo +import spark.executor.TaskMetrics +import spark.ui.JettyUtils._ +import spark.ui.UIUtils.headerSparkPage +import spark.ui.Page.Executors +import spark.storage.{StorageStatus, StorageUtils} +import spark.SparkContext +import spark.ui.UIUtils +import spark.Utils + +import scala.xml.{Node, XML} + +private[spark] class ExecutorsUI(val sc: SparkContext) { + + def getHandlers = Seq[(String, Handler)]( + ("/executors", (request: HttpServletRequest) => render(request)) + ) + + def render(request: HttpServletRequest): Seq[Node] = { + val storageStatusList = sc.getExecutorStorageStatus + + val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) + .reduceOption(_+_).getOrElse(0L) + + val execTables = + for (a <- 0 until storageStatusList.size) + yield getExecTable(a) + + val content = +
        +
        +
          +
        • Memory: + {Utils.memoryBytesToString(maxMem - remainingMem)} Used + ({Utils.memoryBytesToString(remainingMem)} Available)
        • +
        • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
        • +
        +
        +
        +
        +
        + {execTables} +
        +
        ; + + headerSparkPage(content, sc, "Executors", Executors) + } + + def getExecTable(a: Int): Seq[Node] = { + val memUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).maxMem) + val maxMem = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).memUsed()) + val diskUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).diskUsed()) + val rddBlocks = sc.getExecutorStorageStatus(a).blocks.size.toString + val execInfo = Seq( + ("RDD blocks", rddBlocks), + ("Memory used", "%s/%s".format(memUsed, maxMem)), + ("Disk used", diskUsed) + ) + def execRow(kv: (String, String)) = {kv._1}{kv._2} + val table = UIUtils.listingTable(Seq("Name", "Value"), execRow, execInfo) + val execId = sc.getExecutorStorageStatus(a).blockManagerId.executorId + val hostPort = sc.getExecutorStorageStatus(a).blockManagerId.hostPort + val header = +

        Executor {execId}

        +

        {hostPort}

        ; + header ++ table + } +} \ No newline at end of file diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index 4e0360d19a..f76192eba8 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -33,10 +33,6 @@ private[spark] class IndexPage(parent: BlockManagerUI) { def render(request: HttpServletRequest): Seq[Node] = { val storageStatusList = sc.getExecutorStorageStatus // Calculate macro-level statistics - val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) - .reduceOption(_+_).getOrElse(0L) val rddHeaders = Seq( "RDD Name", @@ -46,19 +42,7 @@ private[spark] class IndexPage(parent: BlockManagerUI) { "Size in Memory", "Size on Disk") val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) - val rddTable = listingTable(rddHeaders, rddRow, rdds) - - val content = -
        -
        -
          -
        • Memory: - {Utils.memoryBytesToString(maxMem - remainingMem)} Used - ({Utils.memoryBytesToString(remainingMem)} Available)
        • -
        • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
        • -
        -
        -
        ++ {rddTable}; + val content = listingTable(rddHeaders, rddRow, rdds) headerSparkPage(content, parent.sc, "Spark Storage ", Storage) } -- cgit v1.2.3 From 865dc63bac1cb72ecb84038c0a7cd708cc9e19d7 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 19 Jul 2013 15:57:01 -0700 Subject: Changed table format for executors --- .../src/main/scala/spark/ui/exec/ExecutorsUI.scala | 40 +++++++++++++--------- 1 file changed, 23 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index de0e571553..fa516a4a34 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -33,9 +33,19 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) .reduceOption(_+_).getOrElse(0L) - val execTables = - for (a <- 0 until storageStatusList.size) - yield getExecTable(a) + val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used") + def execRow(kv: Seq[String]) = + + {kv(0)} + {kv(1)} + {kv(2)} + {kv(3)} + {kv(4)} + + val execInfo = + for (b <- 0 until storageStatusList.size) + yield getExecInfo(b) + val execTable = UIUtils.listingTable(execHead, execRow, execInfo) val content =
        @@ -50,30 +60,26 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
        - {execTables} + {execTable}
        ; headerSparkPage(content, sc, "Executors", Executors) } - def getExecTable(a: Int): Seq[Node] = { + def getExecInfo(a: Int): Seq[String] = { + val execId = sc.getExecutorStorageStatus(a).blockManagerId.executorId + val hostPort = sc.getExecutorStorageStatus(a).blockManagerId.hostPort val memUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).maxMem) val maxMem = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).memUsed()) val diskUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).diskUsed()) val rddBlocks = sc.getExecutorStorageStatus(a).blocks.size.toString - val execInfo = Seq( - ("RDD blocks", rddBlocks), - ("Memory used", "%s/%s".format(memUsed, maxMem)), - ("Disk used", diskUsed) + Seq( + execId, + hostPort, + rddBlocks, + "%s/%s".format(memUsed, maxMem), + diskUsed ) - def execRow(kv: (String, String)) = {kv._1}{kv._2} - val table = UIUtils.listingTable(Seq("Name", "Value"), execRow, execInfo) - val execId = sc.getExecutorStorageStatus(a).blockManagerId.executorId - val hostPort = sc.getExecutorStorageStatus(a).blockManagerId.hostPort - val header = -

        Executor {execId}

        -

        {hostPort}

        ; - header ++ table } } \ No newline at end of file -- cgit v1.2.3 From f4d514810e6fd9f42868ebb9a89390c62c3b42e1 Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Tue, 2 Jul 2013 17:03:24 -0700 Subject: Building spark assembly for further consumption of the Spark project with a deployed cluster --- assembly/README | 13 ++++ assembly/pom.xml | 104 ++++++++++++++++++++++++++++++++ assembly/src/main/assembly/assembly.xml | 68 +++++++++++++++++++++ pom.xml | 21 ++++++- spark-shell | 1 + 5 files changed, 206 insertions(+), 1 deletion(-) create mode 100644 assembly/README create mode 100644 assembly/pom.xml create mode 100644 assembly/src/main/assembly/assembly.xml diff --git a/assembly/README b/assembly/README new file mode 100644 index 0000000000..6ee2a536d7 --- /dev/null +++ b/assembly/README @@ -0,0 +1,13 @@ +This is an assembly module for Spark project. + +It creates a single tar.gz file that includes all needed dependency of the project +except for org.apache.hadoop.* jars that are supposed to be available from the +deployed Hadoop cluster. + +This module is off by default to avoid spending extra time on top of repl-bin +module. To activate it specify the profile in the command line + -Passembly + +In case you want to avoid building time-expensive repl-bin module, that shaders +all the dependency into a big flat jar supplement maven command with + -DnoExpensive diff --git a/assembly/pom.xml b/assembly/pom.xml new file mode 100644 index 0000000000..1382539f24 --- /dev/null +++ b/assembly/pom.xml @@ -0,0 +1,104 @@ + + + 4.0.0 + + org.spark-project + spark-parent + 0.8.0-SNAPSHOT + ../pom.xml + + + org.spark-project + spark-assembly + Spark Project Assembly + http://spark-project.org/ + + + + + org.apache.maven.plugins + maven-assembly-plugin + 2.4 + + + dist + package + + single + + + + src/main/assembly/assembly.xml + + + + + + + + + + + hadoop1 + + hadoop1 + + + + hadoop2 + + hadoop2 + + + + hadoop2-yarn + + hadoop2-yarn + + + + + + org.spark-project + spark-core + ${classifier.name} + 0.8.0-SNAPSHOT + + + org.spark-project + spark-bagel + ${classifier.name} + 0.8.0-SNAPSHOT + + + org.spark-project + spark-examples + ${classifier.name} + 0.8.0-SNAPSHOT + + + org.spark-project + spark-examples + javadoc + 0.8.0-SNAPSHOT + + + org.spark-project + spark-examples + sources + 0.8.0-SNAPSHOT + + + org.spark-project + spark-repl + ${classifier.name} + 0.8.0-SNAPSHOT + + + org.spark-project + spark-streaming + ${classifier.name} + 0.8.0-SNAPSHOT + + + \ No newline at end of file diff --git a/assembly/src/main/assembly/assembly.xml b/assembly/src/main/assembly/assembly.xml new file mode 100644 index 0000000000..dd05f35f1f --- /dev/null +++ b/assembly/src/main/assembly/assembly.xml @@ -0,0 +1,68 @@ + + dist + + tar.gz + dir + + false + + + + + README + + + + + ${project.parent.basedir}/core/src/main/resources/spark/ui/static/ + + /ui-resources/spark/ui/static + + **/* + + + + + ${project.parent.basedir}/bin/ + + /bin + + **/* + + + + + ${project.parent.basedir} + + /bin + + run* + spark-shell* + spark-executor* + + + + + + + + org.spark-project:*:jar + + + org.spark-project:spark-dist:jar + + + + lib + true + false + runtime + false + + org.apache.hadoop:*:jar + org.spark-project:*:jar + + + + + diff --git a/pom.xml b/pom.xml index 4b48072c6e..6d9437b69f 100644 --- a/pom.xml +++ b/pom.xml @@ -60,7 +60,6 @@ examples streaming repl - repl-bin @@ -618,5 +617,25 @@ + + assembly + + false + + + assembly + + + + expensive-modules + + + !noExpensive + + + + repl-bin + + diff --git a/spark-shell b/spark-shell index 31a4138124..62fc18550d 100755 --- a/spark-shell +++ b/spark-shell @@ -79,6 +79,7 @@ if [[ ! $? ]]; then saved_stty="" fi +export SPARK_LAUNCH_WITH_SCALA=${SPARK_LAUNCH_WITH_SCALA:-1} $FWDIR/run $OPTIONS spark.repl.Main "$@" # record the exit status lest it be overwritten: -- cgit v1.2.3 From 0337d88321f3681009de548ce10ba7e0ca8f1a58 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Sun, 21 Jul 2013 18:07:19 -0700 Subject: Add a public method getCachedRdds to SparkContext --- core/src/main/scala/spark/SparkContext.scala | 8 +++++++- core/src/test/scala/spark/RDDSuite.scala | 6 ++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index c01e315e35..1b46665d2c 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -546,6 +546,12 @@ class SparkContext( StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this) } + /** + * Returns an immutable map of RDDs that have marked themselves as cached via cache() call. + * Note that this does not necessarily mean the caching or computation was successful. + */ + def getCachedRDDs: Map[Int, RDD[_]] = persistentRdds.asInstanceOf[Map[Int, RDD[_]]] + def getStageInfo: Map[Stage,StageInfo] = { dagScheduler.stageToInfos } @@ -580,7 +586,7 @@ class SparkContext( case null | "file" => if (SparkHadoopUtil.isYarnMode()) { logWarning("local jar specified as parameter to addJar under Yarn mode") - return + return } env.httpFileServer.addJar(new File(uri.getPath)) case _ => path diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index cbddf4e523..ff2dcd72d8 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -90,15 +90,19 @@ class RDDSuite extends FunSuite with SharedSparkContext { } test("basic caching") { + val origCachedRdds = sc.getCachedRDDs.size val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() assert(rdd.collect().toList === List(1, 2, 3, 4)) assert(rdd.collect().toList === List(1, 2, 3, 4)) assert(rdd.collect().toList === List(1, 2, 3, 4)) + // Should only result in one cached RDD + assert(sc.getCachedRDDs.size === origCachedRdds + 1) } test("caching with failures") { val onlySplit = new Partition { override def index: Int = 0 } var shouldFail = true + val origCachedRdds = sc.getCachedRDDs.size val rdd = new RDD[Int](sc, Nil) { override def getPartitions: Array[Partition] = Array(onlySplit) override val getDependencies = List[Dependency[_]]() @@ -110,12 +114,14 @@ class RDDSuite extends FunSuite with SharedSparkContext { } } }.cache() + assert(sc.getCachedRDDs.size === origCachedRdds + 1) val thrown = intercept[Exception]{ rdd.collect() } assert(thrown.getMessage.contains("injected failure")) shouldFail = false assert(rdd.collect().toList === List(1, 2, 3, 4)) + assert(sc.getCachedRDDs.size === origCachedRdds + 1) } test("empty RDD") { -- cgit v1.2.3 From 8901f379c98d19d3145f0cb2ae678d1b0060a311 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 22 Jul 2013 09:58:03 -0700 Subject: Fixed memory used/remaining/total bug --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index fa516a4a34..5fb75c60fd 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -30,10 +30,12 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + val memUsed = storageStatusList.map(_.memUsed()).reduce(_+_) val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) .reduceOption(_+_).getOrElse(0L) - val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used") + val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used/Memory total", + "Disk used") def execRow(kv: Seq[String]) = {kv(0)} @@ -52,8 +54,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
        • Memory: - {Utils.memoryBytesToString(maxMem - remainingMem)} Used - ({Utils.memoryBytesToString(remainingMem)} Available)
        • + {Utils.memoryBytesToString(memUsed)} Used + ({Utils.memoryBytesToString(maxMem)} Total)
        • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
        @@ -70,8 +72,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { def getExecInfo(a: Int): Seq[String] = { val execId = sc.getExecutorStorageStatus(a).blockManagerId.executorId val hostPort = sc.getExecutorStorageStatus(a).blockManagerId.hostPort - val memUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).maxMem) - val maxMem = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).memUsed()) + val memUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).memUsed()) + val maxMem = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).maxMem) val diskUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).diskUsed()) val rddBlocks = sc.getExecutorStorageStatus(a).blocks.size.toString Seq( -- cgit v1.2.3 From f649dabb4a3a57cb25a852808297fb718cbfedd4 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Jul 2013 13:14:33 -0700 Subject: Fix bug: DoubleRDDFunctions.sampleStdev() computed non-sample stdev(). Update JavaDoubleRDD to add new methods and docs. Fixes SPARK-825. --- core/src/main/scala/spark/DoubleRDDFunctions.scala | 8 +++++- .../main/scala/spark/api/java/JavaDoubleRDD.scala | 33 ++++++++++++++++------ core/src/test/scala/spark/JavaAPISuite.java | 2 ++ 3 files changed, 33 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/DoubleRDDFunctions.scala b/core/src/main/scala/spark/DoubleRDDFunctions.scala index 93ef097702..104168e61c 100644 --- a/core/src/main/scala/spark/DoubleRDDFunctions.scala +++ b/core/src/main/scala/spark/DoubleRDDFunctions.scala @@ -54,7 +54,13 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { * Compute the sample standard deviation of this RDD's elements (which corrects for bias in * estimating the standard deviation by dividing by N-1 instead of N). */ - def sampleStdev(): Double = stats().stdev + def sampleStdev(): Double = stats().sampleStdev + + /** + * Compute the sample variance of this RDD's elements (which corrects for bias in + * estimating the variance by dividing by N-1 instead of N). + */ + def sampleVariance(): Double = stats().sampleVariance /** (Experimental) Approximate operation to return the mean within a timeout. */ def meanApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala index 392556f261..8ce7df6213 100644 --- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala @@ -115,33 +115,48 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav // Double RDD functions - /** Return the sum of the elements in this RDD. */ + /** Add up the elements in this RDD. */ def sum(): Double = srdd.sum() - /** Return a [[spark.StatCounter]] describing the elements in this RDD. */ + /** + * Return a [[spark.util.StatCounter]] object that captures the mean, variance and count + * of the RDD's elements in one operation. + */ def stats(): StatCounter = srdd.stats() - /** Return the mean of the elements in this RDD. */ + /** Compute the mean of this RDD's elements. */ def mean(): Double = srdd.mean() - /** Return the variance of the elements in this RDD. */ + /** Compute the variance of this RDD's elements. */ def variance(): Double = srdd.variance() - /** Return the standard deviation of the elements in this RDD. */ + /** Compute the standard deviation of this RDD's elements. */ def stdev(): Double = srdd.stdev() + /** + * Compute the sample standard deviation of this RDD's elements (which corrects for bias in + * estimating the standard deviation by dividing by N-1 instead of N). + */ + def sampleStdev(): Double = srdd.sampleStdev() + + /** + * Compute the sample variance of this RDD's elements (which corrects for bias in + * estimating the standard variance by dividing by N-1 instead of N). + */ + def sampleVariance(): Double = srdd.sampleVariance() + /** Return the approximate mean of the elements in this RDD. */ def meanApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = srdd.meanApprox(timeout, confidence) - /** Return the approximate mean of the elements in this RDD. */ + /** (Experimental) Approximate operation to return the mean within a timeout. */ def meanApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.meanApprox(timeout) - /** Return the approximate sum of the elements in this RDD. */ + /** (Experimental) Approximate operation to return the sum within a timeout. */ def sumApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = srdd.sumApprox(timeout, confidence) - - /** Return the approximate sum of the elements in this RDD. */ + + /** (Experimental) Approximate operation to return the sum within a timeout. */ def sumApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.sumApprox(timeout) } diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index aaf03e683b..5e2bf2d231 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -326,7 +326,9 @@ public class JavaAPISuite implements Serializable { Assert.assertEquals(20/6.0, rdd.mean(), 0.01); Assert.assertEquals(20/6.0, rdd.mean(), 0.01); Assert.assertEquals(6.22222, rdd.variance(), 0.01); + Assert.assertEquals(7.46667, rdd.sampleVariance(), 0.01); Assert.assertEquals(2.49444, rdd.stdev(), 0.01); + Assert.assertEquals(2.73252, rdd.sampleStdev(), 0.01); Double first = rdd.first(); List take = rdd.take(5); -- cgit v1.2.3 From 85c4d7bf3bf0969f58ebcda6ca68719972ff0c46 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 22 Jul 2013 14:35:47 -0700 Subject: Shows number of complete/total/failed tasks (bug: failed tasks assigned to null executor) --- core/src/main/scala/spark/ui/SparkUI.scala | 1 + .../src/main/scala/spark/ui/exec/ExecutorsUI.scala | 63 ++++++++++++++++++---- 2 files changed, 55 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 1534705bc5..7599f82a94 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -67,6 +67,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { // This server must register all handlers, including JobProgressUI, before binding // JobProgressUI registers a listener with SparkContext, which requires sc to initialize jobs.start() + exec.start() } def stop() { diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 5fb75c60fd..a981c680d2 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -5,15 +5,18 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler +import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.Properties -import spark.scheduler.cluster.TaskInfo +import spark.{ExceptionFailure, Logging, SparkContext, Success, Utils} import spark.executor.TaskMetrics +import spark.scheduler.cluster.TaskInfo +import spark.scheduler._ +import spark.SparkContext +import spark.storage.{StorageStatus, StorageUtils} import spark.ui.JettyUtils._ -import spark.ui.UIUtils.headerSparkPage import spark.ui.Page.Executors -import spark.storage.{StorageStatus, StorageUtils} -import spark.SparkContext +import spark.ui.UIUtils.headerSparkPage import spark.ui.UIUtils import spark.Utils @@ -21,6 +24,14 @@ import scala.xml.{Node, XML} private[spark] class ExecutorsUI(val sc: SparkContext) { + private var _listener: Option[ExecutorsListener] = None + def listener = _listener.get + + def start() { + _listener = Some(new ExecutorsListener) + sc.addSparkListener(listener) + } + def getHandlers = Seq[(String, Handler)]( ("/executors", (request: HttpServletRequest) => render(request)) ) @@ -29,13 +40,12 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val storageStatusList = sc.getExecutorStorageStatus val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) val memUsed = storageStatusList.map(_.memUsed()).reduce(_+_) val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) .reduceOption(_+_).getOrElse(0L) - val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used/Memory total", - "Disk used") + val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used", + "Tasks: Complete/Total") def execRow(kv: Seq[String]) = {kv(0)} @@ -43,6 +53,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { {kv(2)} {kv(3)} {kv(4)} + {kv(5)} val execInfo = for (b <- 0 until storageStatusList.size) @@ -76,12 +87,46 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val maxMem = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).maxMem) val diskUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).diskUsed()) val rddBlocks = sc.getExecutorStorageStatus(a).blocks.size.toString + val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0) + val totalTasks = listener.executorToTaskInfos(a.toString).size + val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0) match { + case f if f > 0 => " (%s failed)".format(f) + case _ => "" + } + Seq( execId, hostPort, rddBlocks, - "%s/%s".format(memUsed, maxMem), - diskUsed + "%s / %s".format(memUsed, maxMem), + diskUsed, + "%s / %s".format(completedTasks, totalTasks) + failedTasks ) } + + private[spark] class ExecutorsListener extends SparkListener with Logging { + val executorToTasksComplete = HashMap[String, Int]() + val executorToTasksFailed = HashMap[String, Int]() + val executorToTaskInfos = + HashMap[String, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val eid = taskEnd.taskMetrics.executorId + val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = + taskEnd.reason match { + case e: ExceptionFailure => + executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 + logInfo("Executor %s has %s failed tasks.".format(eid, executorToTasksFailed(eid))) + (Some(e), e.metrics) + case _ => + executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 + logInfo("Executor %s has %s completed tasks.".format(eid, executorToTasksComplete(eid))) + (None, Some(taskEnd.taskMetrics)) + } + val taskList = executorToTaskInfos.getOrElse( + eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) + taskList += ((taskEnd.taskInfo, metrics, failureInfo)) + executorToTaskInfos(eid) = taskList + } + } } \ No newline at end of file -- cgit v1.2.3 From 2eea974795dfa2bb79e66496454f36cb499065b0 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 22 Jul 2013 15:15:54 -0700 Subject: Executors UI now calls executor ID from TaskInfo instead of TaskMetrics --- core/src/main/scala/spark/executor/Executor.scala | 1 - core/src/main/scala/spark/executor/TaskMetrics.scala | 5 ----- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 4 +--- 3 files changed, 1 insertion(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 82e5f5a015..2e81151882 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -126,7 +126,6 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert val taskFinish = System.currentTimeMillis() task.metrics.foreach{ m => m.hostname = Utils.localHostName - m.executorId = executorId m.executorDeserializeTime = (taskStart - startTime).toInt m.executorRunTime = (taskFinish - taskStart).toInt } diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index e14dc57501..3151627839 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -23,11 +23,6 @@ class TaskMetrics extends Serializable { */ var hostname: String = _ - /** - * Executor's ID the task runs on - */ - var executorId: String = _ - /** * Time taken on the executor to deserialize this task */ diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index a981c680d2..40bee325b2 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -111,16 +111,14 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { HashMap[String, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val eid = taskEnd.taskMetrics.executorId + val eid = taskEnd.taskInfo.executorId val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 - logInfo("Executor %s has %s failed tasks.".format(eid, executorToTasksFailed(eid))) (Some(e), e.metrics) case _ => executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 - logInfo("Executor %s has %s completed tasks.".format(eid, executorToTasksComplete(eid))) (None, Some(taskEnd.taskMetrics)) } val taskList = executorToTaskInfos.getOrElse( -- cgit v1.2.3 From 8e38e7723230c1ca9564ddb79087fd1c4326b351 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 22 Jul 2013 16:05:32 -0700 Subject: Fix a test that was using an outdated config setting --- core/src/test/scala/spark/ShuffleSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 3a56c26bef..752e4b85e6 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -33,9 +33,9 @@ import spark.rdd.ShuffledRDD import spark.SparkContext._ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { - test("groupByKey with compression") { + test("groupByKey without compression") { try { - System.setProperty("spark.shuffle.compress", "true") + System.setProperty("spark.shuffle.compress", "false") sc = new SparkContext("local", "test") val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4) val groups = pairs.groupByKey(4).collect() @@ -45,7 +45,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { val valuesFor2 = groups.find(_._1 == 2).get._2 assert(valuesFor2.toList.sorted === List(1)) } finally { - System.setProperty("spark.blockManager.compress", "false") + System.setProperty("spark.shuffle.compress", "true") } } -- cgit v1.2.3 From c83680434bfd6c241ed2e66f2e8704d3b93f752f Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 17 Jul 2013 22:59:00 -0700 Subject: Add JavaAPICompletenessChecker. This is used to find methods in the Scala API that need to be ported to the Java API. To use it: ./run spark.tools.JavaAPICompletenessChecker Conflicts: project/SparkBuild.scala run run2.cmd --- bin/compute-classpath.cmd | 2 + bin/compute-classpath.sh | 2 + pom.xml | 1 + project/SparkBuild.scala | 8 +- tools/pom.xml | 63 ++++ .../spark/tools/JavaAPICompletenessChecker.scala | 359 +++++++++++++++++++++ 6 files changed, 434 insertions(+), 1 deletion(-) create mode 100644 tools/pom.xml create mode 100644 tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 835d1af794..eb836b0ffd 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -33,6 +33,7 @@ set REPL_DIR=%FWDIR%repl set EXAMPLES_DIR=%FWDIR%examples set BAGEL_DIR=%FWDIR%bagel set MLLIB_DIR=%FWDIR%mllib +set TOOLS_DIR=%FWDIR%tools set STREAMING_DIR=%FWDIR%streaming set PYSPARK_DIR=%FWDIR%python @@ -48,6 +49,7 @@ set CLASSPATH=%CLASSPATH%;%FWDIR%repl\lib\* set CLASSPATH=%CLASSPATH%;%FWDIR%python\lib\* set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes set CLASSPATH=%CLASSPATH%;%MLLIB_DIR%\target\scala-%SCALA_VERSION%\classes +set CLASSPATH=%CLASSPATH%;%TOOLS_DIR%\target\scala-%SCALA_VERSION%\classes rem Add hadoop conf dir - else FileSystem.*, etc fail rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index e6fdc5306e..eb270a5428 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -36,6 +36,7 @@ REPL_BIN_DIR="$FWDIR/repl-bin" EXAMPLES_DIR="$FWDIR/examples" BAGEL_DIR="$FWDIR/bagel" MLLIB_DIR="$FWDIR/mllib" +TOOLS_DIR="$FWDIR/tools" STREAMING_DIR="$FWDIR/streaming" PYSPARK_DIR="$FWDIR/python" @@ -70,6 +71,7 @@ function dev_classpath { fi CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$MLLIB_DIR/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$TOOLS_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do CLASSPATH="$CLASSPATH:$jar" done diff --git a/pom.xml b/pom.xml index 4b48072c6e..eb7bd7e9df 100644 --- a/pom.xml +++ b/pom.xml @@ -58,6 +58,7 @@ core bagel examples + tools streaming repl repl-bin diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 2f2cbf646a..f3f67b57c8 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -40,7 +40,7 @@ object SparkBuild extends Build { //val HADOOP_MAJOR_VERSION = "2" //val HADOOP_YARN = true - lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming, mllib) + lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming, mllib, tools) lazy val core = Project("core", file("core"), settings = coreSettings) @@ -48,6 +48,8 @@ object SparkBuild extends Build { lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) dependsOn (streaming) + lazy val tools = Project("tools", file("tools"), settings = examplesSettings) dependsOn (core) dependsOn (streaming) + lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn (core) lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn (core) @@ -233,6 +235,10 @@ object SparkBuild extends Build { ) ) + def toolsSettings = sharedSettings ++ Seq( + name := "spark-tools" + ) + def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") def mllibSettings = sharedSettings ++ Seq( diff --git a/tools/pom.xml b/tools/pom.xml new file mode 100644 index 0000000000..3dfba5808c --- /dev/null +++ b/tools/pom.xml @@ -0,0 +1,63 @@ + + + + 4.0.0 + + org.spark-project + spark-parent + 0.7.4-SNAPSHOT + ../pom.xml + + + org.spark-project + spark-tools_2.9.3 + jar + Spark Project Tools + http://spark-project.org/ + + + + org.spark-project + spark-core_2.9.3 + ${project.version} + + + org.spark-project + spark-streaming_2.9.3 + ${project.version} + + + + target/scala-${scala.version}/classes + target/scala-${scala.version}/test-classes + + + org.apache.maven.plugins + maven-source-plugin + + + org.codehaus.mojo + build-helper-maven-plugin + + + org.scalatest + scalatest-maven-plugin + + + + diff --git a/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala new file mode 100644 index 0000000000..89ef7236f4 --- /dev/null +++ b/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala @@ -0,0 +1,359 @@ +/* + * 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.tools + +import spark._ +import java.lang.reflect.Method +import scala.collection.mutable.ArrayBuffer +import spark.api.java._ +import spark.streaming.{PairDStreamFunctions, DStream, StreamingContext} +import spark.streaming.api.java.{JavaPairDStream, JavaDStream, JavaStreamingContext} +import scala.Tuple2 + + +private[spark] abstract class SparkType(val name: String) + +private[spark] case class BaseType(override val name: String) extends SparkType(name) { + override def toString: String = { + name + } +} + +private[spark] +case class ParameterizedType(override val name: String, + parameters: Seq[SparkType], + typebounds: String = "") extends SparkType(name) { + override def toString: String = { + if (typebounds != "") { + typebounds + " " + name + "<" + parameters.mkString(", ") + ">" + } else { + name + "<" + parameters.mkString(", ") + ">" + } + } +} + +private[spark] +case class SparkMethod(name: String, returnType: SparkType, parameters: Seq[SparkType]) { + override def toString: String = { + returnType + " " + name + "(" + parameters.mkString(", ") + ")" + } +} + +/** + * A tool for identifying methods that need to be ported from Scala to the Java API. + * + * It uses reflection to find methods in the Scala API and rewrites those methods' signatures + * into appropriate Java equivalents. If those equivalent methods have not been implemented in + * the Java API, they are printed. + */ +object JavaAPICompletenessChecker { + + private def parseType(typeStr: String): SparkType = { + if (!typeStr.contains("<")) { + // Base types might begin with "class" or "interface", so we have to strip that off: + BaseType(typeStr.trim.split(" ").last) + } else if (typeStr.endsWith("[]")) { + ParameterizedType("Array", Seq(parseType(typeStr.stripSuffix("[]")))) + } else { + val parts = typeStr.split("<", 2) + val name = parts(0).trim + assert (parts(1).last == '>') + val parameters = parts(1).dropRight(1) + ParameterizedType(name, parseTypeList(parameters)) + } + } + + private def parseTypeList(typeStr: String): Seq[SparkType] = { + val types: ArrayBuffer[SparkType] = new ArrayBuffer[SparkType] + var stack = 0 + var token: StringBuffer = new StringBuffer() + for (c <- typeStr.trim) { + if (c == ',' && stack == 0) { + types += parseType(token.toString) + token = new StringBuffer() + } else if (c == ' ' && stack != 0) { + // continue + } else { + if (c == '<') { + stack += 1 + } else if (c == '>') { + stack -= 1 + } + token.append(c) + } + } + assert (stack == 0) + if (token.toString != "") { + types += parseType(token.toString) + } + types.toSeq + } + + private def parseReturnType(typeStr: String): SparkType = { + if (typeStr(0) == '<') { + val parts = typeStr.drop(0).split(">", 2) + val parsed = parseType(parts(1)).asInstanceOf[ParameterizedType] + ParameterizedType(parsed.name, parsed.parameters, parts(0)) + } else { + parseType(typeStr) + } + } + + private def toSparkMethod(method: Method): SparkMethod = { + val returnType = parseReturnType(method.getGenericReturnType.toString) + val name = method.getName + val parameters = method.getGenericParameterTypes.map(t => parseType(t.toString)) + SparkMethod(name, returnType, parameters) + } + + private def toJavaType(scalaType: SparkType): SparkType = { + val renameSubstitutions = Map( + "scala.collection.Map" -> "java.util.Map", + // TODO: the JavaStreamingContext API accepts Array arguments + // instead of Lists, so this isn't a trivial translation / sub: + "scala.collection.Seq" -> "java.util.List", + "scala.Function2" -> "spark.api.java.function.Function2", + "scala.collection.Iterator" -> "java.util.Iterator", + "scala.collection.mutable.Queue" -> "java.util.Queue", + "double" -> "java.lang.Double" + ) + // Keep applying the substitutions until we've reached a fixedpoint. + def applySubs(scalaType: SparkType): SparkType = { + scalaType match { + case ParameterizedType(name, parameters, typebounds) => + name match { + case "spark.RDD" => + if (parameters(0).name == classOf[Tuple2[_, _]].getName) { + val tupleParams = + parameters(0).asInstanceOf[ParameterizedType].parameters.map(toJavaType) + ParameterizedType(classOf[JavaPairRDD[_, _]].getName, tupleParams) + } else { + ParameterizedType(classOf[JavaRDD[_]].getName, parameters.map(toJavaType)) + } + case "spark.streaming.DStream" => + if (parameters(0).name == classOf[Tuple2[_, _]].getName) { + val tupleParams = + parameters(0).asInstanceOf[ParameterizedType].parameters.map(toJavaType) + ParameterizedType("spark.streaming.api.java.JavaPairDStream", tupleParams) + } else { + ParameterizedType("spark.streaming.api.java.JavaDStream", + parameters.map(toJavaType)) + } + // TODO: Spark Streaming uses Guava's Optional in place of Option, leading to some + // false-positives here: + case "scala.Option" => + toJavaType(parameters(0)) + case "scala.Function1" => + val firstParamName = parameters.last.name + if (firstParamName.startsWith("scala.collection.Traversable") || + firstParamName.startsWith("scala.collection.Iterator")) { + ParameterizedType("spark.api.java.function.FlatMapFunction", + Seq(parameters(0), + parameters.last.asInstanceOf[ParameterizedType].parameters(0)).map(toJavaType)) + } else if (firstParamName == "scala.runtime.BoxedUnit") { + ParameterizedType("spark.api.java.function.VoidFunction", + parameters.dropRight(1).map(toJavaType)) + } else { + ParameterizedType("spark.api.java.function.Function", parameters.map(toJavaType)) + } + case _ => + ParameterizedType(renameSubstitutions.getOrElse(name, name), + parameters.map(toJavaType)) + } + case BaseType(name) => + if (renameSubstitutions.contains(name)) { + BaseType(renameSubstitutions(name)) + } else { + scalaType + } + } + } + var oldType = scalaType + var newType = applySubs(scalaType) + while (oldType != newType) { + oldType = newType + newType = applySubs(scalaType) + } + newType + } + + private def toJavaMethod(method: SparkMethod): SparkMethod = { + val params = method.parameters + .filterNot(_.name == "scala.reflect.ClassManifest").map(toJavaType) + SparkMethod(method.name, toJavaType(method.returnType), params) + } + + private def isExcludedByName(method: Method): Boolean = { + val name = method.getDeclaringClass.getName + "." + method.getName + // Scala methods that are declared as private[mypackage] become public in the resulting + // Java bytecode. As a result, we need to manually exclude those methods here. + // This list also includes a few methods that are only used by the web UI or other + // internal Spark components. + val excludedNames = Seq( + "spark.RDD.origin", + "spark.RDD.elementClassManifest", + "spark.RDD.checkpointData", + "spark.RDD.partitioner", + "spark.RDD.partitions", + "spark.RDD.firstParent", + "spark.RDD.doCheckpoint", + "spark.RDD.markCheckpointed", + "spark.RDD.clearDependencies", + "spark.RDD.getDependencies", + "spark.RDD.getPartitions", + "spark.RDD.dependencies", + "spark.RDD.getPreferredLocations", + "spark.RDD.collectPartitions", + "spark.RDD.computeOrReadCheckpoint", + "spark.PairRDDFunctions.getKeyClass", + "spark.PairRDDFunctions.getValueClass", + "spark.SparkContext.stringToText", + "spark.SparkContext.makeRDD", + "spark.SparkContext.runJob", + "spark.SparkContext.runApproximateJob", + "spark.SparkContext.clean", + "spark.SparkContext.metadataCleaner", + "spark.SparkContext.ui", + "spark.SparkContext.newShuffleId", + "spark.SparkContext.newRddId", + "spark.SparkContext.cleanup", + "spark.SparkContext.receiverJobThread", + "spark.SparkContext.getRDDStorageInfo", + "spark.SparkContext.addedFiles", + "spark.SparkContext.addedJars", + "spark.SparkContext.persistentRdds", + "spark.SparkContext.executorEnvs", + "spark.SparkContext.checkpointDir", + "spark.SparkContext.getSparkHome", + "spark.SparkContext.executorMemoryRequested", + "spark.SparkContext.getExecutorStorageStatus", + "spark.streaming.DStream.generatedRDDs", + "spark.streaming.DStream.zeroTime", + "spark.streaming.DStream.rememberDuration", + "spark.streaming.DStream.storageLevel", + "spark.streaming.DStream.mustCheckpoint", + "spark.streaming.DStream.checkpointDuration", + "spark.streaming.DStream.checkpointData", + "spark.streaming.DStream.graph", + "spark.streaming.DStream.isInitialized", + "spark.streaming.DStream.parentRememberDuration", + "spark.streaming.DStream.initialize", + "spark.streaming.DStream.validate", + "spark.streaming.DStream.setContext", + "spark.streaming.DStream.setGraph", + "spark.streaming.DStream.remember", + "spark.streaming.DStream.getOrCompute", + "spark.streaming.DStream.generateJob", + "spark.streaming.DStream.clearOldMetadata", + "spark.streaming.DStream.addMetadata", + "spark.streaming.DStream.updateCheckpointData", + "spark.streaming.DStream.restoreCheckpointData", + "spark.streaming.DStream.isTimeValid", + "spark.streaming.StreamingContext.nextNetworkInputStreamId", + "spark.streaming.StreamingContext.networkInputTracker", + "spark.streaming.StreamingContext.checkpointDir", + "spark.streaming.StreamingContext.checkpointDuration", + "spark.streaming.StreamingContext.receiverJobThread", + "spark.streaming.StreamingContext.scheduler", + "spark.streaming.StreamingContext.initialCheckpoint", + "spark.streaming.StreamingContext.getNewNetworkStreamId", + "spark.streaming.StreamingContext.validate", + "spark.streaming.StreamingContext.createNewSparkContext", + "spark.streaming.StreamingContext.rddToFileName", + "spark.streaming.StreamingContext.getSparkCheckpointDir", + "spark.streaming.StreamingContext.env", + "spark.streaming.StreamingContext.graph", + "spark.streaming.StreamingContext.isCheckpointPresent" + ) + val excludedPatterns = Seq( + """^spark\.SparkContext\..*To.*Functions""", + """^spark\.SparkContext\..*WritableConverter""", + """^spark\.SparkContext\..*To.*Writable""" + ).map(_.r) + lazy val excludedByPattern = + !excludedPatterns.map(_.findFirstIn(name)).filter(_.isDefined).isEmpty + name.contains("$") || excludedNames.contains(name) || excludedByPattern + } + + private def isExcludedByAnnotation(method: Method): Boolean = { + method.getAnnotation(classOf[ExcludeFromJavaAPI]) != null + } + + private def isExcludedByInterface(method: Method): Boolean = { + val excludedInterfaces = + Set("spark.Logging", "org.apache.hadoop.mapreduce.HadoopMapReduceUtil") + def toComparisionKey(method: Method) = + (method.getReturnType, method.getName, method.getGenericReturnType) + val interfaces = method.getDeclaringClass.getInterfaces.filter { i => + excludedInterfaces.contains(i.getName) + } + val excludedMethods = interfaces.flatMap(_.getMethods.map(toComparisionKey)) + excludedMethods.contains(toComparisionKey(method)) + } + + private def printMissingMethods(scalaClass: Class[_], javaClass: Class[_]) { + val methods = scalaClass.getMethods + .filterNot(_.isAccessible) + .filterNot(isExcludedByAnnotation) + .filterNot(isExcludedByName) + .filterNot(isExcludedByInterface) + val javaEquivalents = methods.map(m => toJavaMethod(toSparkMethod(m))).toSet + + val javaMethods = javaClass.getMethods.map(toSparkMethod).toSet + + val missingMethods = javaEquivalents -- javaMethods + + for (method <- missingMethods) { + println(method) + } + } + + def main(args: Array[String]) { + println("Missing RDD methods") + printMissingMethods(classOf[RDD[_]], classOf[JavaRDD[_]]) + println() + + println("Missing PairRDD methods") + printMissingMethods(classOf[PairRDDFunctions[_, _]], classOf[JavaPairRDD[_, _]]) + println() + + println("Missing DoubleRDD methods") + printMissingMethods(classOf[DoubleRDDFunctions], classOf[JavaDoubleRDD]) + println() + + println("Missing OrderedRDD methods") + printMissingMethods(classOf[OrderedRDDFunctions[_, _]], classOf[JavaPairRDD[_, _]]) + println() + + println("Missing SparkContext methods") + printMissingMethods(classOf[SparkContext], classOf[JavaSparkContext]) + println() + + println("Missing StreamingContext methods") + printMissingMethods(classOf[StreamingContext], classOf[JavaStreamingContext]) + println() + + println("Missing DStream methods") + printMissingMethods(classOf[DStream[_]], classOf[JavaDStream[_]]) + println() + + println("Missing PairDStream methods") + printMissingMethods(classOf[PairDStreamFunctions[_, _]], classOf[JavaPairDStream[_, _]]) + println() + } +} -- cgit v1.2.3 From e17e1b388eaddea39a7272bae201fd3d0060c821 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 20 Jul 2013 22:04:30 -0700 Subject: Remove annotation code that broke build. --- tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala | 5 ----- 1 file changed, 5 deletions(-) diff --git a/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala index 89ef7236f4..3a55f50812 100644 --- a/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala +++ b/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala @@ -290,10 +290,6 @@ object JavaAPICompletenessChecker { name.contains("$") || excludedNames.contains(name) || excludedByPattern } - private def isExcludedByAnnotation(method: Method): Boolean = { - method.getAnnotation(classOf[ExcludeFromJavaAPI]) != null - } - private def isExcludedByInterface(method: Method): Boolean = { val excludedInterfaces = Set("spark.Logging", "org.apache.hadoop.mapreduce.HadoopMapReduceUtil") @@ -309,7 +305,6 @@ object JavaAPICompletenessChecker { private def printMissingMethods(scalaClass: Class[_], javaClass: Class[_]) { val methods = scalaClass.getMethods .filterNot(_.isAccessible) - .filterNot(isExcludedByAnnotation) .filterNot(isExcludedByName) .filterNot(isExcludedByInterface) val javaEquivalents = methods.map(m => toJavaMethod(toSparkMethod(m))).toSet -- cgit v1.2.3 From 872c97ad829ba20e866c4e45054e7d2d05b02042 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 22 Jul 2013 16:54:37 -0700 Subject: Split task columns, memory columns sort by numeric value --- .../src/main/scala/spark/ui/exec/ExecutorsUI.scala | 36 +++++++++++++--------- 1 file changed, 21 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 40bee325b2..20ea54d6a6 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -45,15 +45,21 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { .reduceOption(_+_).getOrElse(0L) val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used", - "Tasks: Complete/Total") + "Failed tasks", "Complete tasks", "Total tasks") def execRow(kv: Seq[String]) = {kv(0)} {kv(1)} {kv(2)} - {kv(3)} - {kv(4)} - {kv(5)} + + {Utils.memoryBytesToString(kv(3).toLong)} / {Utils.memoryBytesToString(kv(4).toLong)} + + + {Utils.memoryBytesToString(kv(5).toLong)} + + {kv(6)} + {kv(7)} + {kv(8)} val execInfo = for (b <- 0 until storageStatusList.size) @@ -83,24 +89,24 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { def getExecInfo(a: Int): Seq[String] = { val execId = sc.getExecutorStorageStatus(a).blockManagerId.executorId val hostPort = sc.getExecutorStorageStatus(a).blockManagerId.hostPort - val memUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).memUsed()) - val maxMem = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).maxMem) - val diskUsed = Utils.memoryBytesToString(sc.getExecutorStorageStatus(a).diskUsed()) val rddBlocks = sc.getExecutorStorageStatus(a).blocks.size.toString - val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0) - val totalTasks = listener.executorToTaskInfos(a.toString).size - val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0) match { - case f if f > 0 => " (%s failed)".format(f) - case _ => "" - } + val memUsed = sc.getExecutorStorageStatus(a).memUsed().toString + val maxMem = sc.getExecutorStorageStatus(a).maxMem.toString + val diskUsed = sc.getExecutorStorageStatus(a).diskUsed().toString + val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0).toString + val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0).toString + val totalTasks = listener.executorToTaskInfos(a.toString).size.toString Seq( execId, hostPort, rddBlocks, - "%s / %s".format(memUsed, maxMem), + memUsed, + maxMem, diskUsed, - "%s / %s".format(completedTasks, totalTasks) + failedTasks + failedTasks, + completedTasks, + totalTasks ) } -- cgit v1.2.3 From 2c2bfbe294c0082520c80a01562a2dbeeba63b7a Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 23 Jul 2013 01:36:44 -0700 Subject: Add toMap method to TimeStampedHashMap and use it --- core/src/main/scala/spark/SparkContext.scala | 2 +- core/src/main/scala/spark/util/TimeStampedHashMap.scala | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 1b46665d2c..0fb7dfa810 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -550,7 +550,7 @@ class SparkContext( * Returns an immutable map of RDDs that have marked themselves as cached via cache() call. * Note that this does not necessarily mean the caching or computation was successful. */ - def getCachedRDDs: Map[Int, RDD[_]] = persistentRdds.asInstanceOf[Map[Int, RDD[_]]] + def getCachedRDDs: Map[Int, RDD[_]] = persistentRdds.toMap def getStageInfo: Map[Stage,StageInfo] = { dagScheduler.stageToInfos diff --git a/core/src/main/scala/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/spark/util/TimeStampedHashMap.scala index cc7909194a..07772a0afb 100644 --- a/core/src/main/scala/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/spark/util/TimeStampedHashMap.scala @@ -20,6 +20,7 @@ package spark.util import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConversions import scala.collection.mutable.Map +import scala.collection.immutable import spark.scheduler.MapStatus /** @@ -99,6 +100,8 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() with spark.Logging { } } + def toMap: immutable.Map[A, B] = iterator.toMap + /** * Removes old key-value pairs that have timestamp earlier than `threshTime` */ -- cgit v1.2.3 From 4830e225624091fa836012651420cf2b5b97dcca Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 23 Jul 2013 09:50:13 -0700 Subject: Rename method per rxin feedback --- core/src/main/scala/spark/SparkContext.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 0fb7dfa810..24ba605646 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -547,10 +547,10 @@ class SparkContext( } /** - * Returns an immutable map of RDDs that have marked themselves as cached via cache() call. + * Returns an immutable map of RDDs that have marked themselves as persistent via cache() call. * Note that this does not necessarily mean the caching or computation was successful. */ - def getCachedRDDs: Map[Int, RDD[_]] = persistentRdds.toMap + def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap def getStageInfo: Map[Stage,StageInfo] = { dagScheduler.stageToInfos -- cgit v1.2.3 From efd6418c1b99c1ecc2b0a4c72e6430eea4d86260 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 23 Jul 2013 10:40:41 -0700 Subject: Move getPersistentRDDs testing to a new Suite --- core/src/test/scala/spark/RDDSuite.scala | 6 --- .../test/scala/spark/SparkContextInfoSuite.scala | 60 ++++++++++++++++++++++ 2 files changed, 60 insertions(+), 6 deletions(-) create mode 100644 core/src/test/scala/spark/SparkContextInfoSuite.scala diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index ff2dcd72d8..cbddf4e523 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -90,19 +90,15 @@ class RDDSuite extends FunSuite with SharedSparkContext { } test("basic caching") { - val origCachedRdds = sc.getCachedRDDs.size val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() assert(rdd.collect().toList === List(1, 2, 3, 4)) assert(rdd.collect().toList === List(1, 2, 3, 4)) assert(rdd.collect().toList === List(1, 2, 3, 4)) - // Should only result in one cached RDD - assert(sc.getCachedRDDs.size === origCachedRdds + 1) } test("caching with failures") { val onlySplit = new Partition { override def index: Int = 0 } var shouldFail = true - val origCachedRdds = sc.getCachedRDDs.size val rdd = new RDD[Int](sc, Nil) { override def getPartitions: Array[Partition] = Array(onlySplit) override val getDependencies = List[Dependency[_]]() @@ -114,14 +110,12 @@ class RDDSuite extends FunSuite with SharedSparkContext { } } }.cache() - assert(sc.getCachedRDDs.size === origCachedRdds + 1) val thrown = intercept[Exception]{ rdd.collect() } assert(thrown.getMessage.contains("injected failure")) shouldFail = false assert(rdd.collect().toList === List(1, 2, 3, 4)) - assert(sc.getCachedRDDs.size === origCachedRdds + 1) } test("empty RDD") { diff --git a/core/src/test/scala/spark/SparkContextInfoSuite.scala b/core/src/test/scala/spark/SparkContextInfoSuite.scala new file mode 100644 index 0000000000..6d50bf5e1b --- /dev/null +++ b/core/src/test/scala/spark/SparkContextInfoSuite.scala @@ -0,0 +1,60 @@ +/* + * 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 + +import org.scalatest.FunSuite +import spark.SparkContext._ + +class SparkContextInfoSuite extends FunSuite with LocalSparkContext { + test("getPersistentRDDs only returns RDDs that are marked as cached") { + sc = new SparkContext("local", "test") + assert(sc.getPersistentRDDs.isEmpty === true) + + val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2) + assert(sc.getPersistentRDDs.isEmpty === true) + + rdd.cache() + assert(sc.getPersistentRDDs.size === 1) + assert(sc.getPersistentRDDs.values.head === rdd) + } + + test("getPersistentRDDs returns an immutable map") { + sc = new SparkContext("local", "test") + val rdd1 = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() + + val myRdds = sc.getPersistentRDDs + assert(myRdds.size === 1) + assert(myRdds.values.head === rdd1) + + val rdd2 = sc.makeRDD(Array(5, 6, 7, 8), 1).cache() + + // getPersistentRDDs should have 2 RDDs, but myRdds should not change + assert(sc.getPersistentRDDs.size === 2) + assert(myRdds.size === 1) + } + + test("getRDDStorageInfo only reports on RDDs that actually persist data") { + sc = new SparkContext("local", "test") + val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() + + assert(sc.getRDDStorageInfo.size === 0) + + rdd.collect() + assert(sc.getRDDStorageInfo.size === 1) + } +} \ No newline at end of file -- cgit v1.2.3 From 87a9dd898ff51fd110799edae087d59f6b714211 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 12:13:27 -0700 Subject: Made RegressionModel serializable and added unit tests to make sure predict methods would work. --- .../spark/mllib/optimization/GradientDescent.scala | 6 ++-- .../scala/spark/mllib/optimization/Updater.scala | 6 ++-- .../mllib/regression/LogisticRegression.scala | 6 +++- .../scala/spark/mllib/regression/Regression.scala | 2 +- .../spark/mllib/regression/RidgeRegression.scala | 5 +++- .../mllib/regression/LogisticRegressionSuite.scala | 33 +++++++++++++++++----- 6 files changed, 42 insertions(+), 16 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index 4c996c0903..185a2a24f6 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -39,9 +39,9 @@ object GradientDescent { * @param miniBatchFraction - fraction of the input data set that should be used for * one iteration of SGD. Default value 1.0. * - * @return weights - Column matrix containing weights for every feature. - * @return stochasticLossHistory - Array containing the stochastic loss computed for - * every iteration. + * @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])], diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index b864fd4634..18cb5f3a95 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -23,13 +23,13 @@ abstract class Updater extends Serializable { /** * Compute an updated value for weights given the gradient, stepSize and iteration number. * - * @param weightsOld - Column matrix of size nx1 where n is the number of features. + * @param weightsOlds - 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 * - * @return weightsNew - Column matrix containing updated weights - * @return reg_val - regularization value + * @return A tuple of 2 elements. The first element is a column matrix containing updated weights, + * and the second element is the regularization value. */ def compute(weightsOlds: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): (DoubleMatrix, Double) diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala index 711e205c39..4b22546017 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala @@ -36,8 +36,12 @@ class LogisticRegressionModel( private val weightsMatrix = new DoubleMatrix(weights.length, 1, weights:_*) override def predict(testData: spark.RDD[Array[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 margin = new DoubleMatrix(1, x.length, x:_*).mmul(weightsMatrix).get(0) + this.intercept + val margin = new DoubleMatrix(1, x.length, x:_*).mmul(localWeights).get(0) + localIntercept 1.0/ (1.0 + math.exp(margin * -1)) } } diff --git a/mllib/src/main/scala/spark/mllib/regression/Regression.scala b/mllib/src/main/scala/spark/mllib/regression/Regression.scala index 645204ddf3..b845ba1a89 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Regression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/Regression.scala @@ -19,7 +19,7 @@ package spark.mllib.regression import spark.RDD -trait RegressionModel { +trait RegressionModel extends Serializable { /** * Predict values for the given data set using the model trained. * diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index f724edd732..6ba141e8fb 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -37,8 +37,11 @@ class RidgeRegressionModel( extends RegressionModel { override def predict(testData: RDD[Array[Double]]): RDD[Double] = { + // A small optimization to avoid serializing the entire model. + val localIntercept = this.intercept + val localWeights = this.weights testData.map { x => - (new DoubleMatrix(1, x.length, x:_*).mmul(this.weights)).get(0) + this.intercept + (new DoubleMatrix(1, x.length, x:_*).mmul(localWeights)).get(0) + localIntercept } } diff --git a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala index 47191d9a5a..6a8098b59d 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala @@ -23,7 +23,6 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext -import spark.SparkContext._ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { @@ -51,15 +50,24 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { // y <- A + B*x + rLogis() // y <- as.numeric(y > 0) - val y = (0 until nPoints).map { i => + val y: Seq[Double] = (0 until nPoints).map { i => val yVal = offset + scale * x1(i) + rLogis(i) if (yVal > 0) 1.0 else 0.0 } - val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i)))) + val testData = (0 until nPoints).map(i => (y(i), Array(x1(i)))) testData } + def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { + val offPredictions = 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) > 0.5 + }.size + // At least 80% of the predictions should be on. + assert(offPredictions < input.length / 5) + } + // Test if we can correctly learn A, B where Y = logistic(A + B*X) test("logistic regression") { val nPoints = 10000 @@ -70,14 +78,20 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val lr = new LogisticRegression().setStepSize(10.0) - .setNumIterations(20) + val lr = new LogisticRegression().setStepSize(10.0).setNumIterations(20) val model = lr.train(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]") + + // Test prediction on RDD. + validatePrediction(model.predict(testRDD.map(_._2)).collect(), testData) + + // Test prediction on Array. + validatePrediction(testData.map(row => model.predict(row._2)), testData) } test("logistic regression with initial weights") { @@ -94,13 +108,18 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { testRDD.cache() // Use half as many iterations as the previous test. - val lr = new LogisticRegression().setStepSize(10.0) - .setNumIterations(10) + val lr = new LogisticRegression().setStepSize(10.0).setNumIterations(10) val model = lr.train(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]") + + // Test prediction on RDD. + validatePrediction(model.predict(testRDD.map(_._2)).collect(), testData) + + // Test prediction on Array. + validatePrediction(testData.map(row => model.predict(row._2)), testData) } } -- cgit v1.2.3 From 2210e8ccf8d77f65442a344c4eae39e000fba927 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 12:52:15 -0700 Subject: Use a different validation dataset for Logistic Regression prediction testing. --- .../mllib/regression/LogisticRegressionSuite.scala | 29 +++++++++++++--------- 1 file changed, 17 insertions(+), 12 deletions(-) diff --git a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala index 6a8098b59d..0a99b78cf8 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala @@ -35,10 +35,11 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { // Generate input of the form Y = logistic(offset + scale*X) def generateLogisticInput( - offset: Double, - scale: Double, - nPoints: Int) : Seq[(Double, Array[Double])] = { - val rnd = new Random(42) + offset: Double, + scale: Double, + nPoints: Int, + seed: Int): Seq[(Double, Array[Double])] = { + 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) @@ -60,12 +61,12 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { } def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { - val offPredictions = predictions.zip(input).filter { case (prediction, (expected, _)) => + 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) > 0.5 }.size // At least 80% of the predictions should be on. - assert(offPredictions < input.length / 5) + assert(numOffPredictions < input.length / 5) } // Test if we can correctly learn A, B where Y = logistic(A + B*X) @@ -74,7 +75,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val A = 2.0 val B = -1.5 - val testData = generateLogisticInput(A, B, nPoints) + val testData = generateLogisticInput(A, B, nPoints, 42) val testRDD = sc.parallelize(testData, 2) testRDD.cache() @@ -87,11 +88,13 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { 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 = generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) // Test prediction on RDD. - validatePrediction(model.predict(testRDD.map(_._2)).collect(), testData) + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) // Test prediction on Array. - validatePrediction(testData.map(row => model.predict(row._2)), testData) + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) } test("logistic regression with initial weights") { @@ -99,7 +102,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val A = 2.0 val B = -1.5 - val testData = generateLogisticInput(A, B, nPoints) + val testData = generateLogisticInput(A, B, nPoints, 42) val initialB = -1.0 val initialWeights = Array(initialB) @@ -116,10 +119,12 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { 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 = generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) // Test prediction on RDD. - validatePrediction(model.predict(testRDD.map(_._2)).collect(), testData) + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) // Test prediction on Array. - validatePrediction(testData.map(row => model.predict(row._2)), testData) + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) } } -- cgit v1.2.3 From 0200801a55b580c7504687e3476b7a71c7699001 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 23 Jul 2013 13:35:43 -0700 Subject: Tracks task start events and shows number of active tasks on Executor UI --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 8 ++++++++ .../main/scala/spark/scheduler/DAGSchedulerEvent.scala | 2 ++ core/src/main/scala/spark/scheduler/JobLogger.scala | 16 +++++++++++++++- core/src/main/scala/spark/scheduler/SparkListener.scala | 9 ++++++++- .../scala/spark/scheduler/TaskSchedulerListener.scala | 3 +++ .../spark/scheduler/cluster/ClusterTaskSetManager.scala | 6 ++++++ .../spark/scheduler/local/LocalTaskSetManager.scala | 5 +++++ core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 12 +++++++++++- 8 files changed, 58 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 29e879aa42..b02bf8f4bf 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -52,6 +52,11 @@ class DAGScheduler( } taskSched.setListener(this) + //Called by TaskScheduler to report task's starting. + override def taskStarted(task: Task[_], taskInfo: TaskInfo) { + eventQueue.put(BeginEvent(task, taskInfo)) + } + // Called by TaskScheduler to report task completions or failures. override def taskEnded( task: Task[_], @@ -343,6 +348,9 @@ class DAGScheduler( case ExecutorLost(execId) => handleExecutorLost(execId) + case begin: BeginEvent => + sparkListeners.foreach(_.onTaskStart(SparkListenerTaskStart(begin.task, begin.taskInfo))) + case completion: CompletionEvent => sparkListeners.foreach(_.onTaskEnd(SparkListenerTaskEnd(completion.task, completion.reason, completion.taskInfo, completion.taskMetrics))) diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index 506c87f65b..3b4ee6287a 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -43,6 +43,8 @@ private[spark] case class JobSubmitted( properties: Properties = null) extends DAGSchedulerEvent +private[spark] case class BeginEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent + private[spark] case class CompletionEvent( task: Task[_], reason: TaskEndReason, diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index 85b5ddd4a8..f7565b8c57 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -68,6 +68,8 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { processStageCompletedEvent(stageInfo) case SparkListenerJobEnd(job, result) => processJobEndEvent(job, result) + case SparkListenerTaskStart(task, taskInfo) => + processTaskStartEvent(task, taskInfo) case SparkListenerTaskEnd(task, reason, taskInfo, taskMetrics) => processTaskEndEvent(task, reason, taskInfo, taskMetrics) case _ => @@ -252,7 +254,19 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { stageInfo.stage.id + " STATUS=COMPLETED") } - + + override def onTaskStart(taskStart: SparkListenerTaskStart) { + eventQueue.put(taskStart) + } + + protected def processTaskStartEvent(task: Task[_], taskInfo: TaskInfo) { + var taskStatus = "" + task match { + case resultTask: ResultTask[_, _] => taskStatus = "TASK_TYPE=RESULT_TASK" + case shuffleMapTask: ShuffleMapTask => taskStatus = "TASK_TYPE=SHUFFLE_MAP_TASK" + } + } + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { eventQueue.put(taskEnd) } diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 4fb1c5d42d..4eb7e4e6a5 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -29,6 +29,8 @@ case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int) extends Spar case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents +case class SparkListenerTaskStart(task: Task[_], taskInfo: TaskInfo) extends SparkListenerEvents + case class SparkListenerTaskEnd(task: Task[_], reason: TaskEndReason, taskInfo: TaskInfo, taskMetrics: TaskMetrics) extends SparkListenerEvents @@ -48,7 +50,12 @@ trait SparkListener { * Called when a stage is submitted */ def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { } - + + /** + * Called when a task starts + */ + def onTaskStart(taskEnd: SparkListenerTaskStart) { } + /** * Called when a task ends */ diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala index 245e7ccb52..2cdeb1c8c0 100644 --- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala +++ b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala @@ -27,6 +27,9 @@ import spark.executor.TaskMetrics * Interface for getting events back from the TaskScheduler. */ private[spark] trait TaskSchedulerListener { + // A task has started. + def taskStarted(task: Task[_], taskInfo: TaskInfo) + // A task has finished or failed. def taskEnded(task: Task[_], reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any], taskInfo: TaskInfo, taskMetrics: TaskMetrics): Unit diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 3d06520675..14e87af653 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -496,6 +496,8 @@ private[spark] class ClusterTaskSetManager( logInfo("Serialized task %s:%d as %d bytes in %d ms".format( taskSet.id, index, serializedTask.limit, timeTaken)) val taskName = "task %s:%d".format(taskSet.id, index) + if (taskAttempts(index).size == 1) + taskStarted(task,info) return Some(new TaskDescription(taskId, execId, taskName, serializedTask)) } case _ => @@ -518,6 +520,10 @@ private[spark] class ClusterTaskSetManager( } } + def taskStarted(task: Task[_], info: TaskInfo) { + sched.listener.taskStarted(task, info) + } + def taskFinished(tid: Long, state: TaskState, serializedData: ByteBuffer) { val info = taskInfos(tid) if (info.failed) { diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index e662ad6709..b500451990 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -117,6 +117,7 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas val taskName = "task %s:%d".format(taskSet.id, index) copiesRunning(index) += 1 increaseRunningTasks(1) + taskStarted(task, info) return Some(new TaskDescription(taskId, null, taskName, bytes)) case None => {} } @@ -146,6 +147,10 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas } } + def taskStarted(task: Task[_], info: TaskInfo) { + sched.listener.taskStarted(task, info) + } + def taskEnded(tid: Long, state: TaskState, serializedData: ByteBuffer) { val info = taskInfos(tid) val index = info.index diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 20ea54d6a6..9ac33326c0 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -45,7 +45,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { .reduceOption(_+_).getOrElse(0L) val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used", - "Failed tasks", "Complete tasks", "Total tasks") + "Active tasks", "Failed tasks", "Complete tasks", "Total tasks") def execRow(kv: Seq[String]) = {kv(0)} @@ -60,6 +60,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { {kv(6)} {kv(7)} {kv(8)} + {kv(9)} val execInfo = for (b <- 0 until storageStatusList.size) @@ -93,6 +94,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val memUsed = sc.getExecutorStorageStatus(a).memUsed().toString val maxMem = sc.getExecutorStorageStatus(a).maxMem.toString val diskUsed = sc.getExecutorStorageStatus(a).diskUsed().toString + val activeTasks = listener.executorToTasksActive.getOrElse(a.toString, 0).toString val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0).toString val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0).toString val totalTasks = listener.executorToTaskInfos(a.toString).size.toString @@ -104,6 +106,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { memUsed, maxMem, diskUsed, + activeTasks, failedTasks, completedTasks, totalTasks @@ -111,13 +114,20 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } private[spark] class ExecutorsListener extends SparkListener with Logging { + val executorToTasksActive = HashMap[String, Int]() val executorToTasksComplete = HashMap[String, Int]() val executorToTasksFailed = HashMap[String, Int]() val executorToTaskInfos = HashMap[String, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() + override def onTaskStart(taskStart: SparkListenerTaskStart) { + val eid = taskStart.taskInfo.executorId + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 + } + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) - 1 val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => -- cgit v1.2.3 From 9f2dbb2a7ca9c1cf878cf96df7122b05d95e799b Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 23 Jul 2013 15:10:09 -0700 Subject: Adds/removes active tasks only once --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 9ac33326c0..bb2b003486 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -94,7 +94,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val memUsed = sc.getExecutorStorageStatus(a).memUsed().toString val maxMem = sc.getExecutorStorageStatus(a).maxMem.toString val diskUsed = sc.getExecutorStorageStatus(a).diskUsed().toString - val activeTasks = listener.executorToTasksActive.getOrElse(a.toString, 0).toString + val activeTasks = listener.executorToTasksActive.getOrElse(a.toString, Seq[Long]()).size.toString val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0).toString val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0).toString val totalTasks = listener.executorToTaskInfos(a.toString).size.toString @@ -114,7 +114,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } private[spark] class ExecutorsListener extends SparkListener with Logging { - val executorToTasksActive = HashMap[String, Int]() + val executorToTasksActive = HashMap[String, Seq[Long]]() val executorToTasksComplete = HashMap[String, Int]() val executorToTasksFailed = HashMap[String, Int]() val executorToTaskInfos = @@ -122,12 +122,12 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = taskStart.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, Seq[Long]()) :+ taskStart.taskInfo.taskId } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) - 1 + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, Seq[Long]()).filterNot(_ == taskEnd.taskInfo.taskId) val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => -- cgit v1.2.3 From 101b8cc78a1fb49332265d7a7e4bf4ec6a22dd42 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 15:28:20 -0700 Subject: SPARK-829: scheduler shouldn't hang if a task contains unserializable objects in its closure. --- .../main/scala/spark/scheduler/DAGScheduler.scala | 29 +++++++++++++++------- core/src/test/scala/spark/FailureSuite.scala | 29 ++++++++++++++++------ 2 files changed, 42 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 29e879aa42..5fcd807aff 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -17,19 +17,17 @@ package spark.scheduler -import cluster.TaskInfo -import java.util.concurrent.atomic.AtomicInteger -import java.util.concurrent.LinkedBlockingQueue -import java.util.concurrent.TimeUnit +import java.io.NotSerializableException import java.util.Properties +import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} +import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import spark._ import spark.executor.TaskMetrics -import spark.partial.ApproximateActionListener -import spark.partial.ApproximateEvaluator -import spark.partial.PartialResult +import spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} +import spark.scheduler.cluster.TaskInfo import spark.storage.{BlockManager, BlockManagerMaster} import spark.util.{MetadataCleaner, TimeStampedHashMap} @@ -258,7 +256,8 @@ class DAGScheduler( assert(partitions.size > 0) val waiter = new JobWaiter(partitions.size, resultHandler) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] - val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties) + val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter, + properties) return (toSubmit, waiter) } @@ -283,7 +282,7 @@ class DAGScheduler( "Total number of partitions: " + maxPartitions) } - val (toSubmit, waiter) = prepareJob( + val (toSubmit: JobSubmitted, waiter: JobWaiter[_]) = prepareJob( finalRdd, func, partitions, callSite, allowLocal, resultHandler, properties) eventQueue.put(toSubmit) waiter.awaitResult() match { @@ -466,6 +465,18 @@ class DAGScheduler( /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { logDebug("submitStage(" + stage + ")") + + // Preemptively serialize the stage RDD to make sure the tasks for this stage will be + // serializable. We are catching this exception here because it would be fairly hard to + // catch the non-serializable exception down the road, where we have several different + // implementations for local scheduler and cluster schedulers. + try { + SparkEnv.get.closureSerializer.newInstance().serialize(stage.rdd) + } catch { + case e: NotSerializableException => abortStage(stage, e.toString) + return + } + if (!waiting(stage) && !running(stage) && !failed(stage)) { val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/spark/FailureSuite.scala index 6c847b8fef..c3c52f9118 100644 --- a/core/src/test/scala/spark/FailureSuite.scala +++ b/core/src/test/scala/spark/FailureSuite.scala @@ -18,9 +18,6 @@ package spark import org.scalatest.FunSuite -import org.scalatest.prop.Checkers - -import scala.collection.mutable.ArrayBuffer import SparkContext._ @@ -40,7 +37,7 @@ object FailureSuiteState { } class FailureSuite extends FunSuite with LocalSparkContext { - + // Run a 3-task map job in which task 1 deterministically fails once, and check // whether the job completes successfully and we ran 4 tasks in total. test("failure in a single-stage job") { @@ -66,7 +63,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { test("failure in a two-stage job") { sc = new SparkContext("local[1,1]", "test") val results = sc.makeRDD(1 to 3).map(x => (x, x)).groupByKey(3).map { - case (k, v) => + case (k, v) => FailureSuiteState.synchronized { FailureSuiteState.tasksRun += 1 if (k == 1 && FailureSuiteState.tasksFailed == 0) { @@ -87,15 +84,33 @@ class FailureSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local[1,1]", "test") val results = sc.makeRDD(1 to 3).map(x => new NonSerializable) - val thrown = intercept[spark.SparkException] { + val thrown = intercept[SparkException] { results.collect() } - assert(thrown.getClass === classOf[spark.SparkException]) + assert(thrown.getClass === classOf[SparkException]) assert(thrown.getMessage.contains("NotSerializableException")) FailureSuiteState.clear() } + test("failure because task closure is not serializable") { + sc = new SparkContext("local[1,1]", "test") + val a = new NonSerializable + val thrown = intercept[SparkException] { + sc.parallelize(1 to 10, 2).map(x => a).count() + } + assert(thrown.getClass === classOf[SparkException]) + assert(thrown.getMessage.contains("NotSerializableException")) + + val thrown1 = intercept[SparkException] { + sc.parallelize(1 to 10, 2).map(x => (x, a)).partitionBy(new HashPartitioner(3)).count() + } + assert(thrown1.getClass === classOf[SparkException]) + assert(thrown1.getMessage.contains("NotSerializableException")) + + FailureSuiteState.clear() + } + // TODO: Need to add tests with shuffle fetch failures. } -- cgit v1.2.3 From 5ed38b4d1d7154235d5b72c35d3a8e63bac3a2de Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 15:28:59 -0700 Subject: Scheduler code style cleanup. --- .../scheduler/cluster/ClusterTaskSetManager.scala | 154 ++++++++++++--------- .../spark/scheduler/cluster/TaskSetManager.scala | 22 ++- .../spark/scheduler/local/LocalScheduler.scala | 22 +-- .../scheduler/local/LocalTaskSetManager.scala | 82 ++++++----- .../spark/scheduler/ClusterSchedulerSuite.scala | 2 +- .../scala/spark/scheduler/DAGSchedulerSuite.scala | 13 +- 6 files changed, 170 insertions(+), 125 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 3d06520675..d934293b70 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -17,7 +17,8 @@ package spark.scheduler.cluster -import java.util.{HashMap => JHashMap, NoSuchElementException, Arrays} +import java.nio.ByteBuffer +import java.util.{Arrays, NoSuchElementException} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap @@ -25,12 +26,14 @@ import scala.collection.mutable.HashSet import scala.math.max import scala.math.min -import spark._ -import spark.scheduler._ +import spark.{FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskState, Utils} +import spark.{ExceptionFailure, SparkException, TaskResultTooBigFailure} import spark.TaskState.TaskState -import java.nio.ByteBuffer +import spark.scheduler.{ShuffleMapTask, Task, TaskResult, TaskSet} -private[spark] object TaskLocality extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY") with Logging { + +private[spark] object TaskLocality + extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY") with Logging { // process local is expected to be used ONLY within tasksetmanager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value @@ -43,8 +46,10 @@ private[spark] object TaskLocality extends Enumeration("PROCESS_LOCAL", "NODE_LO assert (constraint != TaskLocality.PROCESS_LOCAL) constraint match { - case TaskLocality.NODE_LOCAL => condition == TaskLocality.NODE_LOCAL - case TaskLocality.RACK_LOCAL => condition == TaskLocality.NODE_LOCAL || condition == TaskLocality.RACK_LOCAL + case TaskLocality.NODE_LOCAL => + condition == TaskLocality.NODE_LOCAL + case TaskLocality.RACK_LOCAL => + condition == TaskLocality.NODE_LOCAL || condition == TaskLocality.RACK_LOCAL // For anything else, allow case _ => true } @@ -56,11 +61,10 @@ private[spark] object TaskLocality extends Enumeration("PROCESS_LOCAL", "NODE_LO val retval = TaskLocality.withName(str) // Must not specify PROCESS_LOCAL ! assert (retval != TaskLocality.PROCESS_LOCAL) - retval } catch { case nEx: NoSuchElementException => { - logWarning("Invalid task locality specified '" + str + "', defaulting to NODE_LOCAL"); + logWarning("Invalid task locality specified '" + str + "', defaulting to NODE_LOCAL") // default to preserve earlier behavior NODE_LOCAL } @@ -71,11 +75,8 @@ private[spark] object TaskLocality extends Enumeration("PROCESS_LOCAL", "NODE_LO /** * Schedules the tasks within a single TaskSet in the ClusterScheduler. */ -private[spark] class ClusterTaskSetManager( - sched: ClusterScheduler, - val taskSet: TaskSet) - extends TaskSetManager - with Logging { +private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: TaskSet) + extends TaskSetManager with Logging { // Maximum time to wait to run a task in a preferred location (in ms) val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong @@ -106,13 +107,14 @@ private[spark] class ClusterTaskSetManager( var runningTasks = 0 var priority = taskSet.priority var stageId = taskSet.stageId - var name = "TaskSet_"+taskSet.stageId.toString - var parent:Schedulable = null + var name = "TaskSet_" + taskSet.stageId.toString + var parent: Schedulable = null // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis - // List of pending tasks for each node (process local to container). These collections are actually + // List of pending tasks for each node (process local to container). + // These collections are actually // treated as stacks, in which new tasks are added to the end of the // ArrayBuffer and removed from the end. This makes it faster to detect // tasks that repeatedly fail because whenever a task failed, it is put @@ -172,9 +174,11 @@ private[spark] class ClusterTaskSetManager( // Note that it follows the hierarchy. // if we search for NODE_LOCAL, the output will include PROCESS_LOCAL and // if we search for RACK_LOCAL, it will include PROCESS_LOCAL & NODE_LOCAL - private def findPreferredLocations(_taskPreferredLocations: Seq[String], scheduler: ClusterScheduler, - taskLocality: TaskLocality.TaskLocality): HashSet[String] = { - + private def findPreferredLocations( + _taskPreferredLocations: Seq[String], + scheduler: ClusterScheduler, + taskLocality: TaskLocality.TaskLocality): HashSet[String] = + { if (TaskLocality.PROCESS_LOCAL == taskLocality) { // straight forward comparison ! Special case it. val retval = new HashSet[String]() @@ -189,13 +193,14 @@ private[spark] class ClusterTaskSetManager( return retval } - val taskPreferredLocations = + val taskPreferredLocations = { if (TaskLocality.NODE_LOCAL == taskLocality) { _taskPreferredLocations } else { assert (TaskLocality.RACK_LOCAL == taskLocality) // Expand set to include all 'seen' rack local hosts. - // This works since container allocation/management happens within master - so any rack locality information is updated in msater. + // This works since container allocation/management happens within master - + // so any rack locality information is updated in msater. // Best case effort, and maybe sort of kludge for now ... rework it later ? val hosts = new HashSet[String] _taskPreferredLocations.foreach(h => { @@ -213,6 +218,7 @@ private[spark] class ClusterTaskSetManager( hosts } + } val retval = new HashSet[String] scheduler.synchronized { @@ -229,11 +235,13 @@ private[spark] class ClusterTaskSetManager( // Add a task to all the pending-task lists that it should be on. private def addPendingTask(index: Int) { - // We can infer hostLocalLocations from rackLocalLocations by joining it against tasks(index).preferredLocations (with appropriate - // hostPort <-> host conversion). But not doing it for simplicity sake. If this becomes a performance issue, modify it. - val processLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.PROCESS_LOCAL) - val hostLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) - val rackLocalLocations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) + // We can infer hostLocalLocations from rackLocalLocations by joining it against + // tasks(index).preferredLocations (with appropriate hostPort <-> host conversion). + // But not doing it for simplicity sake. If this becomes a performance issue, modify it. + val locs = tasks(index).preferredLocations + val processLocalLocations = findPreferredLocations(locs, sched, TaskLocality.PROCESS_LOCAL) + val hostLocalLocations = findPreferredLocations(locs, sched, TaskLocality.NODE_LOCAL) + val rackLocalLocations = findPreferredLocations(locs, sched, TaskLocality.RACK_LOCAL) if (rackLocalLocations.size == 0) { // Current impl ensures this. @@ -298,18 +306,24 @@ private[spark] class ClusterTaskSetManager( } // Number of pending tasks for a given host Port (which would be process local) - def numPendingTasksForHostPort(hostPort: String): Int = { - getPendingTasksForHostPort(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) + override def numPendingTasksForHostPort(hostPort: String): Int = { + getPendingTasksForHostPort(hostPort).count { index => + copiesRunning(index) == 0 && !finished(index) + } } // Number of pending tasks for a given host (which would be data local) - def numPendingTasksForHost(hostPort: String): Int = { - getPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) + override def numPendingTasksForHost(hostPort: String): Int = { + getPendingTasksForHost(hostPort).count { index => + copiesRunning(index) == 0 && !finished(index) + } } // Number of pending rack local tasks for a given host - def numRackLocalPendingTasksForHost(hostPort: String): Int = { - getRackLocalPendingTasksForHost(hostPort).count( index => copiesRunning(index) == 0 && !finished(index) ) + override def numRackLocalPendingTasksForHost(hostPort: String): Int = { + getRackLocalPendingTasksForHost(hostPort).count { index => + copiesRunning(index) == 0 && !finished(index) + } } @@ -337,12 +351,12 @@ private[spark] class ClusterTaskSetManager( speculatableTasks.retain(index => !finished(index)) // Remove finished tasks from set if (speculatableTasks.size > 0) { - val localTask = speculatableTasks.find { - index => - val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) - val attemptLocs = taskAttempts(index).map(_.hostPort) - (locations.size == 0 || locations.contains(hostPort)) && !attemptLocs.contains(hostPort) - } + val localTask = speculatableTasks.find { index => + val locations = findPreferredLocations(tasks(index).preferredLocations, sched, + TaskLocality.NODE_LOCAL) + val attemptLocs = taskAttempts(index).map(_.hostPort) + (locations.size == 0 || locations.contains(hostPort)) && !attemptLocs.contains(hostPort) + } if (localTask != None) { speculatableTasks -= localTask.get @@ -351,11 +365,11 @@ private[spark] class ClusterTaskSetManager( // check for rack locality if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { - val rackTask = speculatableTasks.find { - index => - val locations = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) - val attemptLocs = taskAttempts(index).map(_.hostPort) - locations.contains(hostPort) && !attemptLocs.contains(hostPort) + val rackTask = speculatableTasks.find { index => + val locations = findPreferredLocations(tasks(index).preferredLocations, sched, + TaskLocality.RACK_LOCAL) + val attemptLocs = taskAttempts(index).map(_.hostPort) + locations.contains(hostPort) && !attemptLocs.contains(hostPort) } if (rackTask != None) { @@ -367,7 +381,9 @@ private[spark] class ClusterTaskSetManager( // Any task ... if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { // Check for attemptLocs also ? - val nonLocalTask = speculatableTasks.find(i => !taskAttempts(i).map(_.hostPort).contains(hostPort)) + val nonLocalTask = speculatableTasks.find { i => + !taskAttempts(i).map(_.hostPort).contains(hostPort) + } if (nonLocalTask != None) { speculatableTasks -= nonLocalTask.get return nonLocalTask @@ -397,7 +413,8 @@ private[spark] class ClusterTaskSetManager( } } - // Look for no pref tasks AFTER rack local tasks - this has side effect that we will get to failed tasks later rather than sooner. + // Look for no pref tasks AFTER rack local tasks - this has side effect that we will get to + // failed tasks later rather than sooner. // TODO: That code path needs to be revisited (adding to no prefs list when host:port goes down). val noPrefTask = findTaskFromList(pendingTasksWithNoPrefs) if (noPrefTask != None) { @@ -433,7 +450,8 @@ private[spark] class ClusterTaskSetManager( locs.find(h => Utils.parseHostPort(h)._1 == host).isDefined } - // Does a host count as a rack local preferred location for a task? (assumes host is NOT preferred location). + // Does a host count as a rack local preferred location for a task? + // (assumes host is NOT preferred location). // This is true if either the task has preferred locations and this host is one, or it has // no preferred locations (in which we still count the launch as preferred). private def isRackLocalLocation(task: Task[_], hostPort: String): Boolean = { @@ -454,14 +472,22 @@ private[spark] class ClusterTaskSetManager( } // Respond to an offer of a single slave from the scheduler by finding a task - def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = { - + override def slaveOffer( + execId: String, + hostPort: String, + availableCpus: Double, + overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = + { if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { // If explicitly specified, use that val locality = if (overrideLocality != null) overrideLocality else { // expand only if we have waited for more than LOCALITY_WAIT for a host local task ... val time = System.currentTimeMillis - if (time - lastPreferredLaunchTime < LOCALITY_WAIT) TaskLocality.NODE_LOCAL else TaskLocality.ANY + if (time - lastPreferredLaunchTime < LOCALITY_WAIT) { + TaskLocality.NODE_LOCAL + } else { + TaskLocality.ANY + } } findTask(hostPort, locality) match { @@ -504,7 +530,7 @@ private[spark] class ClusterTaskSetManager( return None } - def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { + override def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { state match { case TaskState.FINISHED => taskFinished(tid, state, serializedData) @@ -536,7 +562,8 @@ private[spark] class ClusterTaskSetManager( try { val result = ser.deserialize[TaskResult[_]](serializedData) result.metrics.resultSize = serializedData.limit() - sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) + sched.listener.taskEnded( + tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) } catch { case cnf: ClassNotFoundException => val loader = Thread.currentThread().getContextClassLoader @@ -582,8 +609,8 @@ private[spark] class ClusterTaskSetManager( return case taskResultTooBig: TaskResultTooBigFailure => - logInfo("Loss was due to task %s result exceeding Akka frame size; " + - "aborting job".format(tid)) + logInfo("Loss was due to task %s result exceeding Akka frame size; aborting job".format( + tid)) abort("Task %s result exceeded Akka frame size".format(tid)) return @@ -634,7 +661,7 @@ private[spark] class ClusterTaskSetManager( } } - def error(message: String) { + override def error(message: String) { // Save the error message abort("Error: " + message) } @@ -662,7 +689,8 @@ private[spark] class ClusterTaskSetManager( } } - //TODO: for now we just find Pool not TaskSetManager, we can extend this function in future if needed + // TODO: for now we just find Pool not TaskSetManager, + // we can extend this function in future if needed override def getSchedulableByName(name: String): Schedulable = { return null } @@ -687,13 +715,15 @@ private[spark] class ClusterTaskSetManager( // If some task has preferred locations only on hostname, and there are no more executors there, // put it in the no-prefs list to avoid the wait from delay scheduling - // host local tasks - should we push this to rack local or no pref list ? For now, preserving behavior and moving to - // no prefs list. Note, this was done due to impliations related to 'waiting' for data local tasks, etc. - // Note: NOT checking process local list - since host local list is super set of that. We need to ad to no prefs only if - // there is no host local node for the task (not if there is no process local node for the task) + // host local tasks - should we push this to rack local or no pref list ? For now, preserving + // behavior and moving to no prefs list. Note, this was done due to impliations related to + // 'waiting' for data local tasks, etc. + // Note: NOT checking process local list - since host local list is super set of that. We need + // to ad to no prefs only if there is no host local node for the task (not if there is no + // process local node for the task) for (index <- getPendingTasksForHost(Utils.parseHostPort(hostPort)._1)) { - // val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.RACK_LOCAL) - val newLocs = findPreferredLocations(tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) + val newLocs = findPreferredLocations( + tasks(index).preferredLocations, sched, TaskLocality.NODE_LOCAL) if (newLocs.isEmpty) { pendingTasksWithNoPrefs += index } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 07c3ddcc7e..7978a5df74 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -17,18 +17,28 @@ package spark.scheduler.cluster -import scala.collection.mutable.ArrayBuffer -import spark.scheduler._ -import spark.TaskState.TaskState import java.nio.ByteBuffer +import spark.TaskState.TaskState +import spark.scheduler.TaskSet + private[spark] trait TaskSetManager extends Schedulable { + def taskSet: TaskSet - def slaveOffer(execId: String, hostPort: String, availableCpus: Double, - overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] + + def slaveOffer( + execId: String, + hostPort: String, + availableCpus: Double, + overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] + def numPendingTasksForHostPort(hostPort: String): Int - def numRackLocalPendingTasksForHost(hostPort :String): Int + + def numRackLocalPendingTasksForHost(hostPort: String): Int + def numPendingTasksForHost(hostPort: String): Int + def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) + def error(message: String) } diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 1f73cb99a7..edd83d4cb4 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -37,10 +37,15 @@ import akka.actor._ * testing fault recovery. */ -private[spark] case class LocalReviveOffers() -private[spark] case class LocalStatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) +private[spark] +case class LocalReviveOffers() + +private[spark] +case class LocalStatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) + +private[spark] +class LocalActor(localScheduler: LocalScheduler, var freeCores: Int) extends Actor with Logging { -private[spark] class LocalActor(localScheduler: LocalScheduler, var freeCores: Int) extends Actor with Logging { def receive = { case LocalReviveOffers => launchTask(localScheduler.resourceOffer(freeCores)) @@ -55,7 +60,7 @@ private[spark] class LocalActor(localScheduler: LocalScheduler, var freeCores: I freeCores -= 1 localScheduler.threadPool.submit(new Runnable { def run() { - localScheduler.runTask(task.taskId,task.serializedTask) + localScheduler.runTask(task.taskId, task.serializedTask) } }) } @@ -110,7 +115,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: override def submitTasks(taskSet: TaskSet) { synchronized { - var manager = new LocalTaskSetManager(this, taskSet) + val manager = new LocalTaskSetManager(this, taskSet) schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) activeTaskSets(taskSet.id) = manager taskSetTaskIds(taskSet.id) = new HashSet[Long]() @@ -124,14 +129,15 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: val tasks = new ArrayBuffer[TaskDescription](freeCores) val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() for (manager <- sortedTaskSetQueue) { - logDebug("parentName:%s,name:%s,runningTasks:%s".format(manager.parent.name, manager.name, manager.runningTasks)) + logDebug("parentName:%s,name:%s,runningTasks:%s".format( + manager.parent.name, manager.name, manager.runningTasks)) } var launchTask = false for (manager <- sortedTaskSetQueue) { do { launchTask = false - manager.slaveOffer(null,null,freeCpuCores) match { + manager.slaveOffer(null, null, freeCpuCores) match { case Some(task) => tasks += task taskIdToTaskSetId(task.taskId) = manager.taskSet.id @@ -139,7 +145,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: freeCpuCores -= 1 launchTask = true case None => {} - } + } } while(launchTask) } return tasks diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index e662ad6709..bbce9eda64 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -17,27 +17,26 @@ package spark.scheduler.local -import java.io.File -import java.util.concurrent.atomic.AtomicInteger import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet -import spark._ +import spark.{ExceptionFailure, Logging, SparkEnv, Success, TaskState} import spark.TaskState.TaskState -import spark.scheduler._ -import spark.scheduler.cluster._ +import spark.scheduler.{Task, TaskResult, TaskSet} +import spark.scheduler.cluster.{Schedulable, TaskDescription, TaskInfo, TaskLocality, TaskSetManager} + + +private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) + extends TaskSetManager with Logging { -private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) extends TaskSetManager with Logging { var parent: Schedulable = null var weight: Int = 1 var minShare: Int = 0 var runningTasks: Int = 0 var priority: Int = taskSet.priority var stageId: Int = taskSet.stageId - var name: String = "TaskSet_"+taskSet.stageId.toString - + var name: String = "TaskSet_" + taskSet.stageId.toString var failCount = new Array[Int](taskSet.tasks.size) val taskInfos = new HashMap[Long, TaskInfo] @@ -49,49 +48,45 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas val numFailures = new Array[Int](numTasks) val MAX_TASK_FAILURES = sched.maxFailures - def increaseRunningTasks(taskNum: Int): Unit = { - runningTasks += taskNum - if (parent != null) { - parent.increaseRunningTasks(taskNum) - } + override def increaseRunningTasks(taskNum: Int): Unit = { + runningTasks += taskNum + if (parent != null) { + parent.increaseRunningTasks(taskNum) + } } - def decreaseRunningTasks(taskNum: Int): Unit = { + override def decreaseRunningTasks(taskNum: Int): Unit = { runningTasks -= taskNum if (parent != null) { parent.decreaseRunningTasks(taskNum) } } - def addSchedulable(schedulable: Schedulable): Unit = { + override def addSchedulable(schedulable: Schedulable): Unit = { //nothing } - def removeSchedulable(schedulable: Schedulable): Unit = { + override def removeSchedulable(schedulable: Schedulable): Unit = { //nothing } - def getSchedulableByName(name: String): Schedulable = { + override def getSchedulableByName(name: String): Schedulable = { return null } - def executorLost(executorId: String, host: String): Unit = { + override def executorLost(executorId: String, host: String): Unit = { //nothing } - def checkSpeculatableTasks(): Boolean = { - return true - } + override def checkSpeculatableTasks() = true - def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { + override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] sortedTaskSetQueue += this return sortedTaskSetQueue } - def hasPendingTasks(): Boolean = { - return true - } + override def hasPendingTasks() = true def findTask(): Option[Int] = { for (i <- 0 to numTasks-1) { @@ -102,17 +97,25 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas return None } - def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = { + override def slaveOffer( + execId: String, + hostPort: String, + availableCpus: Double, + overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = + { SparkEnv.set(sched.env) - logDebug("availableCpus:%d,numFinished:%d,numTasks:%d".format(availableCpus.toInt, numFinished, numTasks)) + logDebug("availableCpus:%d,numFinished:%d,numTasks:%d".format( + availableCpus.toInt, numFinished, numTasks)) if (availableCpus > 0 && numFinished < numTasks) { findTask() match { case Some(index) => val taskId = sched.attemptId.getAndIncrement() val task = taskSet.tasks(index) - val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) + val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1", + TaskLocality.NODE_LOCAL) taskInfos(taskId) = info - val bytes = Task.serializeWithDependencies(task, sched.sc.addedFiles, sched.sc.addedJars, ser) + val bytes = Task.serializeWithDependencies( + task, sched.sc.addedFiles, sched.sc.addedJars, ser) logInfo("Size of task " + taskId + " is " + bytes.limit + " bytes") val taskName = "task %s:%d".format(taskSet.id, index) copiesRunning(index) += 1 @@ -124,19 +127,19 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas return None } - def numPendingTasksForHostPort(hostPort: String): Int = { + override def numPendingTasksForHostPort(hostPort: String): Int = { return 0 } - def numRackLocalPendingTasksForHost(hostPort :String): Int = { + override def numRackLocalPendingTasksForHost(hostPort :String): Int = { return 0 } - def numPendingTasksForHost(hostPort: String): Int = { + override def numPendingTasksForHost(hostPort: String): Int = { return 0 } - def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { + override def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { state match { case TaskState.FINISHED => taskEnded(tid, state, serializedData) @@ -168,15 +171,18 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas val task = taskSet.tasks(index) info.markFailed() decreaseRunningTasks(1) - val reason: ExceptionFailure = ser.deserialize[ExceptionFailure](serializedData, getClass.getClassLoader) + val reason: ExceptionFailure = ser.deserialize[ExceptionFailure]( + serializedData, getClass.getClassLoader) sched.listener.taskEnded(task, reason, null, null, info, reason.metrics.getOrElse(null)) if (!finished(index)) { copiesRunning(index) -= 1 numFailures(index) += 1 val locs = reason.stackTrace.map(loc => "\tat %s".format(loc.toString)) - logInfo("Loss was due to %s\n%s\n%s".format(reason.className, reason.description, locs.mkString("\n"))) + logInfo("Loss was due to %s\n%s\n%s".format( + reason.className, reason.description, locs.mkString("\n"))) if (numFailures(index) > MAX_TASK_FAILURES) { - val errorMessage = "Task %s:%d failed more than %d times; aborting job %s".format(taskSet.id, index, 4, reason.description) + val errorMessage = "Task %s:%d failed more than %d times; aborting job %s".format( + taskSet.id, index, 4, reason.description) decreaseRunningTasks(runningTasks) sched.listener.taskSetFailed(taskSet, errorMessage) // need to delete failed Taskset from schedule queue @@ -185,6 +191,6 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas } } - def error(message: String) { + override def error(message: String) { } } diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala index 8f81d0b6ee..05afcd6567 100644 --- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala @@ -33,7 +33,7 @@ class DummyTaskSetManager( initNumTasks: Int, clusterScheduler: ClusterScheduler, taskSet: TaskSet) - extends ClusterTaskSetManager(clusterScheduler,taskSet) { + extends ClusterTaskSetManager(clusterScheduler, taskSet) { parent = null weight = 1 diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index f802b66cf1..a8b88d7936 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -23,21 +23,14 @@ import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter import spark.LocalSparkContext - -import spark.storage.BlockManager -import spark.storage.BlockManagerId -import spark.storage.BlockManagerMaster -import spark.{Dependency, ShuffleDependency, OneToOneDependency} -import spark.FetchFailedException import spark.MapOutputTracker import spark.RDD import spark.SparkContext -import spark.SparkException import spark.Partition import spark.TaskContext -import spark.TaskEndReason - -import spark.{FetchFailed, Success} +import spark.{Dependency, ShuffleDependency, OneToOneDependency} +import spark.{FetchFailed, Success, TaskEndReason} +import spark.storage.{BlockManagerId, BlockManagerMaster} /** * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler -- cgit v1.2.3 From f2422d4f29abb80b9bc76c4596d1cc31d9e6d590 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 15:28:20 -0700 Subject: SPARK-829: scheduler shouldn't hang if a task contains unserializable objects in its closure. --- .../main/scala/spark/scheduler/DAGScheduler.scala | 29 +++++++++++++++------- core/src/test/scala/spark/FailureSuite.scala | 29 ++++++++++++++++------ 2 files changed, 42 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 29e879aa42..5fcd807aff 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -17,19 +17,17 @@ package spark.scheduler -import cluster.TaskInfo -import java.util.concurrent.atomic.AtomicInteger -import java.util.concurrent.LinkedBlockingQueue -import java.util.concurrent.TimeUnit +import java.io.NotSerializableException import java.util.Properties +import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} +import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import spark._ import spark.executor.TaskMetrics -import spark.partial.ApproximateActionListener -import spark.partial.ApproximateEvaluator -import spark.partial.PartialResult +import spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} +import spark.scheduler.cluster.TaskInfo import spark.storage.{BlockManager, BlockManagerMaster} import spark.util.{MetadataCleaner, TimeStampedHashMap} @@ -258,7 +256,8 @@ class DAGScheduler( assert(partitions.size > 0) val waiter = new JobWaiter(partitions.size, resultHandler) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] - val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties) + val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter, + properties) return (toSubmit, waiter) } @@ -283,7 +282,7 @@ class DAGScheduler( "Total number of partitions: " + maxPartitions) } - val (toSubmit, waiter) = prepareJob( + val (toSubmit: JobSubmitted, waiter: JobWaiter[_]) = prepareJob( finalRdd, func, partitions, callSite, allowLocal, resultHandler, properties) eventQueue.put(toSubmit) waiter.awaitResult() match { @@ -466,6 +465,18 @@ class DAGScheduler( /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { logDebug("submitStage(" + stage + ")") + + // Preemptively serialize the stage RDD to make sure the tasks for this stage will be + // serializable. We are catching this exception here because it would be fairly hard to + // catch the non-serializable exception down the road, where we have several different + // implementations for local scheduler and cluster schedulers. + try { + SparkEnv.get.closureSerializer.newInstance().serialize(stage.rdd) + } catch { + case e: NotSerializableException => abortStage(stage, e.toString) + return + } + if (!waiting(stage) && !running(stage) && !failed(stage)) { val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/spark/FailureSuite.scala index 6c847b8fef..c3c52f9118 100644 --- a/core/src/test/scala/spark/FailureSuite.scala +++ b/core/src/test/scala/spark/FailureSuite.scala @@ -18,9 +18,6 @@ package spark import org.scalatest.FunSuite -import org.scalatest.prop.Checkers - -import scala.collection.mutable.ArrayBuffer import SparkContext._ @@ -40,7 +37,7 @@ object FailureSuiteState { } class FailureSuite extends FunSuite with LocalSparkContext { - + // Run a 3-task map job in which task 1 deterministically fails once, and check // whether the job completes successfully and we ran 4 tasks in total. test("failure in a single-stage job") { @@ -66,7 +63,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { test("failure in a two-stage job") { sc = new SparkContext("local[1,1]", "test") val results = sc.makeRDD(1 to 3).map(x => (x, x)).groupByKey(3).map { - case (k, v) => + case (k, v) => FailureSuiteState.synchronized { FailureSuiteState.tasksRun += 1 if (k == 1 && FailureSuiteState.tasksFailed == 0) { @@ -87,15 +84,33 @@ class FailureSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local[1,1]", "test") val results = sc.makeRDD(1 to 3).map(x => new NonSerializable) - val thrown = intercept[spark.SparkException] { + val thrown = intercept[SparkException] { results.collect() } - assert(thrown.getClass === classOf[spark.SparkException]) + assert(thrown.getClass === classOf[SparkException]) assert(thrown.getMessage.contains("NotSerializableException")) FailureSuiteState.clear() } + test("failure because task closure is not serializable") { + sc = new SparkContext("local[1,1]", "test") + val a = new NonSerializable + val thrown = intercept[SparkException] { + sc.parallelize(1 to 10, 2).map(x => a).count() + } + assert(thrown.getClass === classOf[SparkException]) + assert(thrown.getMessage.contains("NotSerializableException")) + + val thrown1 = intercept[SparkException] { + sc.parallelize(1 to 10, 2).map(x => (x, a)).partitionBy(new HashPartitioner(3)).count() + } + assert(thrown1.getClass === classOf[SparkException]) + assert(thrown1.getMessage.contains("NotSerializableException")) + + FailureSuiteState.clear() + } + // TODO: Need to add tests with shuffle fetch failures. } -- cgit v1.2.3 From 383684daaa62bebc177b4c74573ce509f154f74b Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 23 Jul 2013 15:33:27 -0700 Subject: Replaces Seq with HashSet, removes redundant import --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index bb2b003486..606e1eb2fc 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -5,7 +5,7 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler -import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.util.Properties import spark.{ExceptionFailure, Logging, SparkContext, Success, Utils} @@ -18,7 +18,6 @@ import spark.ui.JettyUtils._ import spark.ui.Page.Executors import spark.ui.UIUtils.headerSparkPage import spark.ui.UIUtils -import spark.Utils import scala.xml.{Node, XML} @@ -114,7 +113,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } private[spark] class ExecutorsListener extends SparkListener with Logging { - val executorToTasksActive = HashMap[String, Seq[Long]]() + val executorToTasksActive = HashMap[String, HashSet[Long]]() val executorToTasksComplete = HashMap[String, Int]() val executorToTasksFailed = HashMap[String, Int]() val executorToTaskInfos = @@ -122,12 +121,14 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = taskStart.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, Seq[Long]()) :+ taskStart.taskInfo.taskId + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, HashSet[Long]()) + + taskStart.taskInfo.taskId } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, Seq[Long]()).filterNot(_ == taskEnd.taskInfo.taskId) + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, HashSet[Long]()) - + taskEnd.taskInfo.taskId val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => -- cgit v1.2.3 From abc78cd3318fb7bc69d10fd5422d20b299a8d7d8 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 23 Jul 2013 15:47:16 -0700 Subject: Modifies instead of copies HashSets, fixes comment style --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 2 +- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 10 ++++++---- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index b02bf8f4bf..7bf50de660 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -52,7 +52,7 @@ class DAGScheduler( } taskSched.setListener(this) - //Called by TaskScheduler to report task's starting. + // Called by TaskScheduler to report task's starting. override def taskStarted(task: Task[_], taskInfo: TaskInfo) { eventQueue.put(BeginEvent(task, taskInfo)) } diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 606e1eb2fc..69fb306074 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -121,14 +121,16 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = taskStart.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, HashSet[Long]()) + - taskStart.taskInfo.taskId + if (!executorToTasksActive.contains(eid)) + executorToTasksActive(eid) = HashSet[Long]() + executorToTasksActive(eid) += taskStart.taskInfo.taskId } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, HashSet[Long]()) - - taskEnd.taskInfo.taskId + if (!executorToTasksActive.contains(eid)) + executorToTasksActive(eid) = HashSet[Long]() + executorToTasksActive(eid) -= taskStart.taskInfo.taskId val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => -- cgit v1.2.3 From 6a31b7191d5a8203563a2a4e600210c67439abf5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 23 Jul 2013 16:20:24 -0700 Subject: Small bug fix --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 69fb306074..bad5c442ab 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -130,7 +130,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val eid = taskEnd.taskInfo.executorId if (!executorToTasksActive.contains(eid)) executorToTasksActive(eid) = HashSet[Long]() - executorToTasksActive(eid) -= taskStart.taskInfo.taskId + executorToTasksActive(eid) -= taskEnd.taskInfo.taskId val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => @@ -146,4 +146,4 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { executorToTaskInfos(eid) = taskList } } -} \ No newline at end of file +} -- cgit v1.2.3 From 85ab8114bc1367a0f4f32d4b8635c41aa547bc72 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 20:25:58 -0700 Subject: Moved non-serializable closure catching exception from submitStage to submitMissingTasks --- .../main/scala/spark/scheduler/DAGScheduler.scala | 25 +++++++++++----------- core/src/test/scala/spark/FailureSuite.scala | 10 +++++++++ 2 files changed, 23 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 5fcd807aff..fde998494f 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -465,18 +465,6 @@ class DAGScheduler( /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { logDebug("submitStage(" + stage + ")") - - // Preemptively serialize the stage RDD to make sure the tasks for this stage will be - // serializable. We are catching this exception here because it would be fairly hard to - // catch the non-serializable exception down the road, where we have several different - // implementations for local scheduler and cluster schedulers. - try { - SparkEnv.get.closureSerializer.newInstance().serialize(stage.rdd) - } catch { - case e: NotSerializableException => abortStage(stage, e.toString) - return - } - if (!waiting(stage) && !running(stage) && !failed(stage)) { val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) @@ -515,6 +503,19 @@ class DAGScheduler( } } if (tasks.size > 0) { + // Preemptively serialize a task to make sure it can be serialized. We are catching this + // exception here because it would be fairly hard to catch the non-serializable exception + // down the road, where we have several different implementations for local scheduler and + // cluster schedulers. + try { + SparkEnv.get.closureSerializer.newInstance().serialize(tasks.head) + } catch { + case e: NotSerializableException => + abortStage(stage, e.toString) + running -= stage + return + } + sparkListeners.foreach(_.onStageSubmitted(SparkListenerStageSubmitted(stage, tasks.size))) logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/spark/FailureSuite.scala index c3c52f9118..5b133cdd6e 100644 --- a/core/src/test/scala/spark/FailureSuite.scala +++ b/core/src/test/scala/spark/FailureSuite.scala @@ -96,18 +96,28 @@ class FailureSuite extends FunSuite with LocalSparkContext { test("failure because task closure is not serializable") { sc = new SparkContext("local[1,1]", "test") val a = new NonSerializable + + // Non-serializable closure in the final result stage val thrown = intercept[SparkException] { sc.parallelize(1 to 10, 2).map(x => a).count() } assert(thrown.getClass === classOf[SparkException]) assert(thrown.getMessage.contains("NotSerializableException")) + // Non-serializable closure in an earlier stage val thrown1 = intercept[SparkException] { sc.parallelize(1 to 10, 2).map(x => (x, a)).partitionBy(new HashPartitioner(3)).count() } assert(thrown1.getClass === classOf[SparkException]) assert(thrown1.getMessage.contains("NotSerializableException")) + // Non-serializable closure in foreach function + val thrown2 = intercept[SparkException] { + sc.parallelize(1 to 10, 2).foreach(x => println(a)) + } + assert(thrown2.getClass === classOf[SparkException]) + assert(thrown2.getMessage.contains("NotSerializableException")) + FailureSuiteState.clear() } -- cgit v1.2.3 From d33b8a2a0ffd6d085cbd8de22863a1f35c106270 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 20:28:39 -0700 Subject: Added comments on task closure serialization. --- core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala | 2 ++ core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala | 2 ++ 2 files changed, 4 insertions(+) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index d934293b70..f64818876b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -515,6 +515,8 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: } // Serialize and return the task val startTime = System.currentTimeMillis + // We rely on the DAGScheduler to catch non-serializable closures and RDDs, so in here + // we assume the task can be serialized without exceptions. val serializedTask = Task.serializeWithDependencies( task, sched.sc.addedFiles, sched.sc.addedJars, ser) val timeTaken = System.currentTimeMillis - startTime diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index bbce9eda64..a9b49cad0e 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -114,6 +114,8 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL) taskInfos(taskId) = info + // We rely on the DAGScheduler to catch non-serializable closures and RDDs, so in here + // we assume the task can be serialized without exceptions. val bytes = Task.serializeWithDependencies( task, sched.sc.addedFiles, sched.sc.addedJars, ser) logInfo("Size of task " + taskId + " is " + bytes.limit + " bytes") -- cgit v1.2.3 From 3dae1df66f34d3716c697f85cfe4aedeee428688 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 20:25:58 -0700 Subject: Moved non-serializable closure catching exception from submitStage to submitMissingTasks --- .../main/scala/spark/scheduler/DAGScheduler.scala | 25 +++++++++++----------- core/src/test/scala/spark/FailureSuite.scala | 10 +++++++++ 2 files changed, 23 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 5fcd807aff..fde998494f 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -465,18 +465,6 @@ class DAGScheduler( /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { logDebug("submitStage(" + stage + ")") - - // Preemptively serialize the stage RDD to make sure the tasks for this stage will be - // serializable. We are catching this exception here because it would be fairly hard to - // catch the non-serializable exception down the road, where we have several different - // implementations for local scheduler and cluster schedulers. - try { - SparkEnv.get.closureSerializer.newInstance().serialize(stage.rdd) - } catch { - case e: NotSerializableException => abortStage(stage, e.toString) - return - } - if (!waiting(stage) && !running(stage) && !failed(stage)) { val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) @@ -515,6 +503,19 @@ class DAGScheduler( } } if (tasks.size > 0) { + // Preemptively serialize a task to make sure it can be serialized. We are catching this + // exception here because it would be fairly hard to catch the non-serializable exception + // down the road, where we have several different implementations for local scheduler and + // cluster schedulers. + try { + SparkEnv.get.closureSerializer.newInstance().serialize(tasks.head) + } catch { + case e: NotSerializableException => + abortStage(stage, e.toString) + running -= stage + return + } + sparkListeners.foreach(_.onStageSubmitted(SparkListenerStageSubmitted(stage, tasks.size))) logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/spark/FailureSuite.scala index c3c52f9118..5b133cdd6e 100644 --- a/core/src/test/scala/spark/FailureSuite.scala +++ b/core/src/test/scala/spark/FailureSuite.scala @@ -96,18 +96,28 @@ class FailureSuite extends FunSuite with LocalSparkContext { test("failure because task closure is not serializable") { sc = new SparkContext("local[1,1]", "test") val a = new NonSerializable + + // Non-serializable closure in the final result stage val thrown = intercept[SparkException] { sc.parallelize(1 to 10, 2).map(x => a).count() } assert(thrown.getClass === classOf[SparkException]) assert(thrown.getMessage.contains("NotSerializableException")) + // Non-serializable closure in an earlier stage val thrown1 = intercept[SparkException] { sc.parallelize(1 to 10, 2).map(x => (x, a)).partitionBy(new HashPartitioner(3)).count() } assert(thrown1.getClass === classOf[SparkException]) assert(thrown1.getMessage.contains("NotSerializableException")) + // Non-serializable closure in foreach function + val thrown2 = intercept[SparkException] { + sc.parallelize(1 to 10, 2).foreach(x => println(a)) + } + assert(thrown2.getClass === classOf[SparkException]) + assert(thrown2.getMessage.contains("NotSerializableException")) + FailureSuiteState.clear() } -- cgit v1.2.3 From 503acd3a379a3686d343fdf072fc231b8fba78f9 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 09:47:27 +0800 Subject: Build metrics system framwork --- conf/metrics.properties | 13 +++++ .../spark/metrics/AbstractInstrumentation.scala | 68 ++++++++++++++++++++++ .../main/scala/spark/metrics/MetricsConfig.scala | 55 +++++++++++++++++ .../scala/spark/metrics/sink/ConsoleSink.scala | 43 ++++++++++++++ .../main/scala/spark/metrics/sink/CsvSink.scala | 53 +++++++++++++++++ .../main/scala/spark/metrics/sink/JmxSink.scala | 17 ++++++ core/src/main/scala/spark/metrics/sink/Sink.scala | 6 ++ 7 files changed, 255 insertions(+) create mode 100644 conf/metrics.properties create mode 100644 core/src/main/scala/spark/metrics/AbstractInstrumentation.scala create mode 100644 core/src/main/scala/spark/metrics/MetricsConfig.scala create mode 100644 core/src/main/scala/spark/metrics/sink/ConsoleSink.scala create mode 100644 core/src/main/scala/spark/metrics/sink/CsvSink.scala create mode 100644 core/src/main/scala/spark/metrics/sink/JmxSink.scala create mode 100644 core/src/main/scala/spark/metrics/sink/Sink.scala diff --git a/conf/metrics.properties b/conf/metrics.properties new file mode 100644 index 0000000000..78749cf381 --- /dev/null +++ b/conf/metrics.properties @@ -0,0 +1,13 @@ +# syntax: [prefix].[sink].[instance].[options] + +*.sink.console.period=10 + +*.sink.console.unit=second + +master.sink.console.period=10 + +master.sink.console.unit=second + +worker.sink.console.period=20 + +worker.sink.console.unit=second diff --git a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala b/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala new file mode 100644 index 0000000000..0fed608488 --- /dev/null +++ b/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala @@ -0,0 +1,68 @@ +package spark.metrics + +import scala.collection.mutable + +import com.codahale.metrics.{JmxReporter, MetricRegistry} + +import java.util.Properties +import java.util.concurrent.TimeUnit + +import spark.Logging +import spark.metrics.sink._ + +trait AbstractInstrumentation extends Logging { + initLogging() + + def registryHandler: MetricRegistry + def instance: String + + val confFile = System.getProperty("spark.metrics.conf.file", MetricsConfig.DEFAULT_CONFIG_FILE) + val metricsConfig = new MetricsConfig(confFile) + + val sinks = new mutable.ArrayBuffer[Sink] + + def registerSinks() { + val instConfig = metricsConfig.getInstance(instance) + val sinkConfigs = MetricsConfig.subProperties(instConfig, AbstractInstrumentation.SINK_REGEX) + + // Register JMX sink as a default sink + sinks += new JmxSink(registryHandler) + + // Register other sinks according to conf + sinkConfigs.foreach { kv => + val classPath = if (AbstractInstrumentation.DEFAULT_SINKS.contains(kv._1)) { + AbstractInstrumentation.DEFAULT_SINKS(kv._1) + } else { + kv._2.getProperty("class") + } + try { + val sink = Class.forName(classPath).getConstructor(classOf[Properties], classOf[MetricRegistry]) + .newInstance(kv._2, registryHandler) + sinks += sink.asInstanceOf[Sink] + } catch { + case e: Exception => logError("class " + classPath + "cannot be instantialize", e) + } + } + + sinks.foreach(_.registerSink) + } + + def unregisterSinks() { + sinks.foreach(_.unregisterSink) + } +} + +object AbstractInstrumentation { + val DEFAULT_SINKS = Map( + "console" -> "spark.metrics.sink.ConsoleSink", + "csv" -> "spark.metrics.sink.CsvSink") + + val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r + + val timeUnits = Map( + "millisecond" -> TimeUnit.MILLISECONDS, + "second" -> TimeUnit.SECONDS, + "minute" -> TimeUnit.MINUTES, + "hour" -> TimeUnit.HOURS, + "day" -> TimeUnit.DAYS) +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala new file mode 100644 index 0000000000..0fec1988ea --- /dev/null +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -0,0 +1,55 @@ +package spark.metrics + +import java.util.Properties +import java.io.FileInputStream + +import scala.collection.mutable +import scala.util.matching.Regex + +class MetricsConfig(val configFile: String) { + val properties = new Properties() + var fis: FileInputStream = _ + + try { + fis = new FileInputStream(configFile) + properties.load(fis) + } finally { + fis.close() + } + + val propertyCategories = MetricsConfig.subProperties(properties, MetricsConfig.INSTANCE_REGEX) + if (propertyCategories.contains(MetricsConfig.DEFAULT_PREFIX)) { + import scala.collection.JavaConversions._ + val defaultProperty = propertyCategories(MetricsConfig.DEFAULT_PREFIX) + for ((inst, prop) <- propertyCategories; p <- defaultProperty + if inst != MetricsConfig.DEFAULT_PREFIX; if prop.getProperty(p._1) == null) { + prop.setProperty(p._1, p._2) + } + } + + def getInstance(inst: String) = { + propertyCategories.get(inst) match { + case Some(s) => s + case None => propertyCategories(MetricsConfig.DEFAULT_PREFIX) + } + } +} + +object MetricsConfig { + val DEFAULT_CONFIG_FILE = "/home/jerryshao/project/sotc_cloud-spark/conf/metrics.properties" + val DEFAULT_PREFIX = "*" + val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r + + def subProperties(prop: Properties, regex: Regex) = { + val subProperties = new mutable.HashMap[String, Properties] + + import scala.collection.JavaConversions._ + prop.foreach { kv => + val regex(a, b) = kv._1 + subProperties.getOrElseUpdate(a, new Properties).setProperty(b, kv._2) + println(">>>>>subProperties added " + a + " " + b + " " + kv._2) + } + + subProperties + } +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala new file mode 100644 index 0000000000..5426af8c4c --- /dev/null +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -0,0 +1,43 @@ +package spark.metrics.sink + +import java.util.Properties +import java.util.concurrent.TimeUnit + +import com.codahale.metrics.{ConsoleReporter, MetricRegistry} + +import spark.metrics.AbstractInstrumentation + +class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { + val pollPeriod = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_PERIOD)) match { + case Some(s) => s.toInt + case None => ConsoleSink.CONSOLE_DEFAULT_PERIOD.toInt + } + + val pollUnit = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_UNIT)) match { + case Some(s) => AbstractInstrumentation.timeUnits(s) + case None => AbstractInstrumentation.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) + } + + var reporter: ConsoleReporter = _ + + override def registerSink() { + reporter = ConsoleReporter.forRegistry(registry) + .convertDurationsTo(TimeUnit.MILLISECONDS) + .convertRatesTo(TimeUnit.SECONDS) + .build() + + reporter.start(pollPeriod, pollUnit) + } + + override def unregisterSink() { + reporter.stop() + } +} + +object ConsoleSink { + val CONSOLE_DEFAULT_PERIOD = "10" + val CONSOLE_DEFAULT_UNIT = "second" + + val CONSOLE_KEY_PERIOD = "period" + val CONSOLE_KEY_UNIT = "unit" +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala new file mode 100644 index 0000000000..3a80c36901 --- /dev/null +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -0,0 +1,53 @@ +package spark.metrics.sink + +import java.io.File +import java.util.{Locale, Properties} +import java.util.concurrent.TimeUnit + +import com.codahale.metrics.{CsvReporter, MetricRegistry} + +import spark.metrics.AbstractInstrumentation + +class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { + val pollPeriod = Option(property.getProperty(CsvSink.CSV_KEY_PERIOD)) match { + case Some(s) => s.toInt + case None => CsvSink.CSV_DEFAULT_PERIOD.toInt + } + + val pollUnit = Option(property.getProperty(CsvSink.CSV_KEY_UNIT)) match { + case Some(s) => AbstractInstrumentation.timeUnits(s) + case None => AbstractInstrumentation.timeUnits(CsvSink.CSV_DEFAULT_UNIT) + } + + val pollDir = Option(property.getProperty(CsvSink.CSV_KEY_DIR)) match { + case Some(s) => s + case None => CsvSink.CSV_DEFAULT_DIR + } + + var reporter: CsvReporter = _ + + override def registerSink() { + reporter = CsvReporter.forRegistry(registry) + .formatFor(Locale.US) + .convertDurationsTo(TimeUnit.MILLISECONDS) + .convertRatesTo(TimeUnit.SECONDS) + .build(new File(pollDir)) + + reporter.start(pollPeriod, pollUnit) + } + + override def unregisterSink() { + reporter.stop() + } +} + +object CsvSink { + val CSV_KEY_PERIOD = "period" + val CSV_KEY_UNIT = "unit" + val CSV_KEY_DIR = "directory" + + val CSV_DEFAULT_PERIOD = "1" + val CSV_DEFAULT_UNIT = "minute" + val CSV_DEFAULT_DIR = "/tmp/" +} + diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala new file mode 100644 index 0000000000..56e5677700 --- /dev/null +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -0,0 +1,17 @@ +package spark.metrics.sink + +import com.codahale.metrics.{JmxReporter, MetricRegistry} + +class JmxSink(registry: MetricRegistry) extends Sink { + var reporter: JmxReporter = _ + + override def registerSink() { + reporter = JmxReporter.forRegistry(registry).build() + reporter.start() + } + + override def unregisterSink() { + reporter.stop() + } + +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/sink/Sink.scala b/core/src/main/scala/spark/metrics/sink/Sink.scala new file mode 100644 index 0000000000..65ebcb4eac --- /dev/null +++ b/core/src/main/scala/spark/metrics/sink/Sink.scala @@ -0,0 +1,6 @@ +package spark.metrics.sink + +trait Sink { + def registerSink: Unit + def unregisterSink: Unit +} \ No newline at end of file -- cgit v1.2.3 From 9dec8c73e6f0c3b6b55a11ff92cc9bff18dadd24 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 09:48:41 +0800 Subject: Add Master and Worker instrumentation support --- .../main/scala/spark/deploy/master/Master.scala | 9 +++- .../deploy/master/MasterInstrumentation.scala | 44 +++++++++++++++++ .../main/scala/spark/deploy/worker/Worker.scala | 11 +++-- .../deploy/worker/WorkerInstrumentation.scala | 55 ++++++++++++++++++++++ 4 files changed, 115 insertions(+), 4 deletions(-) create mode 100644 core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala create mode 100644 core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index eddcafd84d..3a7c4e5a52 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -33,7 +33,8 @@ import spark.util.AkkaUtils import ui.MasterWebUI -private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { +private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor +with Logging with MasterInstrumentation { val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 @@ -73,6 +74,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) + + initialize(this) } override def postStop() { @@ -316,6 +319,10 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act removeWorker(worker) } } + + override def postStop() { + uninitialize() + } } private[spark] object Master { diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala new file mode 100644 index 0000000000..13088189a4 --- /dev/null +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -0,0 +1,44 @@ +package spark.deploy.master + +import com.codahale.metrics.{Gauge, JmxReporter, MetricRegistry} + +import spark.metrics.AbstractInstrumentation + +private[spark] trait MasterInstrumentation extends AbstractInstrumentation { + var masterInst: Option[Master] = None + val metricRegistry = new MetricRegistry() + + override def registryHandler = metricRegistry + + override def instance = "master" + + def initialize(master: Master) { + masterInst = Some(master) + + // Register and start all the sinks + registerSinks + } + + def uninitialize() { + unregisterSinks + } + + // Gauge for worker numbers in cluster + metricRegistry.register(MetricRegistry.name(classOf[Master], "workers", "number"), + new Gauge[Int] { + override def getValue: Int = masterInst.map(_.workers.size).getOrElse(0) + }) + + // Gauge for application numbers in cluster + metricRegistry.register(MetricRegistry.name(classOf[Master], "apps", "number"), + new Gauge[Int] { + override def getValue: Int = masterInst.map(_.apps.size).getOrElse(0) + }) + + // Gauge for waiting application numbers in cluster + metricRegistry.register(MetricRegistry.name(classOf[Master], "waiting_apps", "number"), + new Gauge[Int] { + override def getValue: Int = masterInst.map(_.waitingApps.size).getOrElse(0) + }) + +} \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 0bd88ea253..b64bdb8d28 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -41,7 +41,7 @@ private[spark] class Worker( memory: Int, masterUrl: String, workDirPath: String = null) - extends Actor with Logging { + extends Actor with Logging with WorkerInstrumentation { Utils.checkHost(host, "Expected hostname") assert (port > 0) @@ -97,6 +97,9 @@ private[spark] class Worker( webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) webUi.start() connectToMaster() + startWebUi() + + initialize(this) } def connectToMaster() { @@ -155,10 +158,10 @@ private[spark] class Worker( case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) => masterDisconnected() - + case RequestWorkerState => { sender ! WorkerState(host, port, workerId, executors.values.toList, - finishedExecutors.values.toList, masterUrl, cores, memory, + finishedExecutors.values.toList, masterUrl, cores, memory, coresUsed, memoryUsed, masterWebUiUrl) } } @@ -178,6 +181,8 @@ private[spark] class Worker( override def postStop() { executors.values.foreach(_.kill()) webUi.stop() + + uninitialize() } } diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala new file mode 100644 index 0000000000..04c43ce33b --- /dev/null +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -0,0 +1,55 @@ +package spark.deploy.worker + +import com.codahale.metrics.{JmxReporter, Gauge, MetricRegistry} + +import spark.metrics.AbstractInstrumentation + +private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { + var workerInst: Option[Worker] = None + val metricRegistry = new MetricRegistry() + + override def registryHandler = metricRegistry + + override def instance = "worker" + + def initialize(worker: Worker) { + workerInst = Some(worker) + + // Register and start all the sinks + registerSinks() + } + + def uninitialize() { + unregisterSinks() + } + + // Gauge for executors number + metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.executors.size).getOrElse(0) + }) + + // Gauge for cores used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.coresUsed).getOrElse(0) + }) + + // Gauge for memory used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "Mbytes"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) + }) + + // Gauge for cores free of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.coresFree).getOrElse(0) + }) + + // Gauge for memory used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), + new Gauge[Int] { + override def getValue: Int = workerInst.map(_.memoryFree).getOrElse(0) + }) +} \ No newline at end of file -- cgit v1.2.3 From c3daad3f65630eb4ed536d06c0d467cde57a8142 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 12:00:19 +0800 Subject: Update metric source support for instrumentation --- conf/metrics.properties | 4 ++- .../deploy/master/MasterInstrumentation.scala | 7 ++++-- .../deploy/worker/WorkerInstrumentation.scala | 5 +++- .../spark/metrics/AbstractInstrumentation.scala | 29 +++++++++++++++++++--- .../main/scala/spark/metrics/MetricsConfig.scala | 12 +++++---- .../main/scala/spark/metrics/sink/CsvSink.scala | 4 +-- core/src/main/scala/spark/metrics/sink/Sink.scala | 1 + .../scala/spark/metrics/source/JvmSource.scala | 17 +++++++++++++ .../main/scala/spark/metrics/source/Source.scala | 5 ++++ 9 files changed, 70 insertions(+), 14 deletions(-) create mode 100644 core/src/main/scala/spark/metrics/source/JvmSource.scala create mode 100644 core/src/main/scala/spark/metrics/source/Source.scala diff --git a/conf/metrics.properties b/conf/metrics.properties index 78749cf381..0bbb6b5229 100644 --- a/conf/metrics.properties +++ b/conf/metrics.properties @@ -1,9 +1,11 @@ -# syntax: [prefix].[sink].[instance].[options] +# syntax: [prefix].[sink|source].[instance].[options] *.sink.console.period=10 *.sink.console.unit=second +master.source.jvm.class=spark.metrics.source.JvmSource + master.sink.console.period=10 master.sink.console.unit=second diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index 13088189a4..c295e725d7 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -15,12 +15,15 @@ private[spark] trait MasterInstrumentation extends AbstractInstrumentation { def initialize(master: Master) { masterInst = Some(master) + // Register all the sources + registerSources() + // Register and start all the sinks - registerSinks + registerSinks() } def uninitialize() { - unregisterSinks + unregisterSinks() } // Gauge for worker numbers in cluster diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 04c43ce33b..2f725300b5 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -15,6 +15,9 @@ private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { def initialize(worker: Worker) { workerInst = Some(worker) + // Register all the sources + registerSources() + // Register and start all the sinks registerSinks() } @@ -36,7 +39,7 @@ private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { }) // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "Mbytes"), + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), new Gauge[Int] { override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) }) diff --git a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala b/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala index 0fed608488..9cae1e0220 100644 --- a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala +++ b/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala @@ -9,17 +9,39 @@ import java.util.concurrent.TimeUnit import spark.Logging import spark.metrics.sink._ +import spark.metrics.source._ -trait AbstractInstrumentation extends Logging { +private [spark] trait AbstractInstrumentation extends Logging { initLogging() + // Get MetricRegistry handler def registryHandler: MetricRegistry + // Get the instance name def instance: String val confFile = System.getProperty("spark.metrics.conf.file", MetricsConfig.DEFAULT_CONFIG_FILE) val metricsConfig = new MetricsConfig(confFile) val sinks = new mutable.ArrayBuffer[Sink] + val sources = new mutable.ArrayBuffer[Source] + + def registerSources() { + val instConfig = metricsConfig.getInstance(instance) + val sourceConfigs = MetricsConfig.subProperties(instConfig, AbstractInstrumentation.SOURCE_REGEX) + + // Register all the sources + sourceConfigs.foreach { kv => + val classPath = kv._2.getProperty("class") + try { + val source = Class.forName(classPath).getConstructor(classOf[MetricRegistry]) + .newInstance(registryHandler) + sources += source.asInstanceOf[Source] + } catch { + case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) + } + } + sources.foreach(_.registerSource) + } def registerSinks() { val instConfig = metricsConfig.getInstance(instance) @@ -33,6 +55,7 @@ trait AbstractInstrumentation extends Logging { val classPath = if (AbstractInstrumentation.DEFAULT_SINKS.contains(kv._1)) { AbstractInstrumentation.DEFAULT_SINKS(kv._1) } else { + // For non-default sink, a property class should be set and create using reflection kv._2.getProperty("class") } try { @@ -40,10 +63,9 @@ trait AbstractInstrumentation extends Logging { .newInstance(kv._2, registryHandler) sinks += sink.asInstanceOf[Sink] } catch { - case e: Exception => logError("class " + classPath + "cannot be instantialize", e) + case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e) } } - sinks.foreach(_.registerSink) } @@ -58,6 +80,7 @@ object AbstractInstrumentation { "csv" -> "spark.metrics.sink.CsvSink") val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r + val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r val timeUnits = Map( "millisecond" -> TimeUnit.MILLISECONDS, diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index 0fec1988ea..be4f670918 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -6,7 +6,7 @@ import java.io.FileInputStream import scala.collection.mutable import scala.util.matching.Regex -class MetricsConfig(val configFile: String) { +private [spark] class MetricsConfig(val configFile: String) { val properties = new Properties() var fis: FileInputStream = _ @@ -36,7 +36,7 @@ class MetricsConfig(val configFile: String) { } object MetricsConfig { - val DEFAULT_CONFIG_FILE = "/home/jerryshao/project/sotc_cloud-spark/conf/metrics.properties" + val DEFAULT_CONFIG_FILE = "conf/metrics.properties" val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r @@ -45,9 +45,11 @@ object MetricsConfig { import scala.collection.JavaConversions._ prop.foreach { kv => - val regex(a, b) = kv._1 - subProperties.getOrElseUpdate(a, new Properties).setProperty(b, kv._2) - println(">>>>>subProperties added " + a + " " + b + " " + kv._2) + if (regex.findPrefixOf(kv._1) != None) { + val regex(prefix, suffix) = kv._1 + subProperties.getOrElseUpdate(prefix, new Properties).setProperty(suffix, kv._2) + println(">>>>>subProperties added " + prefix + " " + suffix + " " + kv._2) + } } subProperties diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index 3a80c36901..1d663f6cff 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -46,8 +46,8 @@ object CsvSink { val CSV_KEY_UNIT = "unit" val CSV_KEY_DIR = "directory" - val CSV_DEFAULT_PERIOD = "1" - val CSV_DEFAULT_UNIT = "minute" + val CSV_DEFAULT_PERIOD = "10" + val CSV_DEFAULT_UNIT = "second" val CSV_DEFAULT_DIR = "/tmp/" } diff --git a/core/src/main/scala/spark/metrics/sink/Sink.scala b/core/src/main/scala/spark/metrics/sink/Sink.scala index 65ebcb4eac..26052b7231 100644 --- a/core/src/main/scala/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/spark/metrics/sink/Sink.scala @@ -2,5 +2,6 @@ package spark.metrics.sink trait Sink { def registerSink: Unit + def unregisterSink: Unit } \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala new file mode 100644 index 0000000000..8f6bf48843 --- /dev/null +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -0,0 +1,17 @@ +package spark.metrics.source + +import com.codahale.metrics.MetricRegistry +import com.codahale.metrics.jvm.{MemoryUsageGaugeSet, GarbageCollectorMetricSet} + +class JvmSource(registry: MetricRegistry) extends Source { + // Initialize memory usage gauge for jvm + val memUsageMetricSet = new MemoryUsageGaugeSet + + // Initialize garbage collection usage gauge for jvm + val gcMetricSet = new GarbageCollectorMetricSet + + override def registerSource() { + registry.registerAll(memUsageMetricSet) + registry.registerAll(gcMetricSet) + } +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/source/Source.scala b/core/src/main/scala/spark/metrics/source/Source.scala new file mode 100644 index 0000000000..35cfe0c8ff --- /dev/null +++ b/core/src/main/scala/spark/metrics/source/Source.scala @@ -0,0 +1,5 @@ +package spark.metrics.source + +trait Source { + def registerSource: Unit +} \ No newline at end of file -- cgit v1.2.3 From 03f9871116801abbdd7b4c7892c8d6affb1c4d9e Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 18:29:54 +0800 Subject: MetricsSystem refactor --- .../main/scala/spark/deploy/master/Master.scala | 17 +-- .../deploy/master/MasterInstrumentation.scala | 64 ++++------ .../main/scala/spark/deploy/worker/Worker.scala | 12 +- .../deploy/worker/WorkerInstrumentation.scala | 131 +++++++++++++-------- .../spark/metrics/AbstractInstrumentation.scala | 91 -------------- .../main/scala/spark/metrics/MetricsSystem.scala | 99 ++++++++++++++++ .../scala/spark/metrics/sink/ConsoleSink.scala | 10 +- .../main/scala/spark/metrics/sink/CsvSink.scala | 10 +- .../main/scala/spark/metrics/sink/JmxSink.scala | 4 +- core/src/main/scala/spark/metrics/sink/Sink.scala | 4 +- .../scala/spark/metrics/source/JvmSource.scala | 30 +++-- .../main/scala/spark/metrics/source/Source.scala | 6 +- 12 files changed, 261 insertions(+), 217 deletions(-) delete mode 100644 core/src/main/scala/spark/metrics/AbstractInstrumentation.scala create mode 100644 core/src/main/scala/spark/metrics/MetricsSystem.scala diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 3a7c4e5a52..e44f5e3168 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -29,12 +29,12 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import spark.deploy._ import spark.{Logging, SparkException, Utils} +import spark.metrics.MetricsSystem import spark.util.AkkaUtils import ui.MasterWebUI -private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor -with Logging with MasterInstrumentation { +private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 @@ -57,6 +57,8 @@ with Logging with MasterInstrumentation { val webUi = new MasterWebUI(self, webUiPort) Utils.checkHost(host, "Expected hostname") + + val masterInstrumentation = new MasterInstrumentation(this) val masterPublicAddress = { val envVar = System.getenv("SPARK_PUBLIC_DNS") @@ -75,7 +77,7 @@ with Logging with MasterInstrumentation { webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) - initialize(this) + Master.metricsSystem.registerSource(masterInstrumentation) } override def postStop() { @@ -319,21 +321,22 @@ with Logging with MasterInstrumentation { removeWorker(worker) } } - - override def postStop() { - uninitialize() - } } private[spark] object Master { private val systemName = "sparkMaster" private val actorName = "Master" private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r + + private val metricsSystem = MetricsSystem.createMetricsSystem("master") def main(argStrings: Array[String]) { val args = new MasterArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort) + + metricsSystem.start() actorSystem.awaitTermination() + metricsSystem.stop() } /** Returns an `akka://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */ diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index c295e725d7..5ea9a90319 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -1,47 +1,35 @@ package spark.deploy.master -import com.codahale.metrics.{Gauge, JmxReporter, MetricRegistry} +import java.util.{Map, HashMap => JHashMap} -import spark.metrics.AbstractInstrumentation +import com.codahale.metrics.{Gauge, Metric} -private[spark] trait MasterInstrumentation extends AbstractInstrumentation { - var masterInst: Option[Master] = None - val metricRegistry = new MetricRegistry() - - override def registryHandler = metricRegistry - - override def instance = "master" +import spark.metrics.source.Source + +private[spark] class MasterInstrumentation(val master: Master) extends Source { + val className = classOf[Master].getName() + val instrumentationName = "master" + + override def sourceName = instrumentationName - def initialize(master: Master) { - masterInst = Some(master) + override def getMetrics(): Map[String, Metric] = { + val gauges = new JHashMap[String, Metric] - // Register all the sources - registerSources() + // Gauge for worker numbers in cluster + gauges.put(className + ".workers.number", new Gauge[Int] { + override def getValue: Int = master.workers.size + }) - // Register and start all the sinks - registerSinks() - } - - def uninitialize() { - unregisterSinks() + // Gauge for application numbers in cluster + gauges.put(className + ".apps.number", new Gauge[Int] { + override def getValue: Int = master.apps.size + }) + + // Gauge for waiting application numbers in cluster + gauges.put(className + ".waiting_apps.number", new Gauge[Int] { + override def getValue: Int = master.waitingApps.size + }) + + gauges } - - // Gauge for worker numbers in cluster - metricRegistry.register(MetricRegistry.name(classOf[Master], "workers", "number"), - new Gauge[Int] { - override def getValue: Int = masterInst.map(_.workers.size).getOrElse(0) - }) - - // Gauge for application numbers in cluster - metricRegistry.register(MetricRegistry.name(classOf[Master], "apps", "number"), - new Gauge[Int] { - override def getValue: Int = masterInst.map(_.apps.size).getOrElse(0) - }) - - // Gauge for waiting application numbers in cluster - metricRegistry.register(MetricRegistry.name(classOf[Master], "waiting_apps", "number"), - new Gauge[Int] { - override def getValue: Int = masterInst.map(_.waitingApps.size).getOrElse(0) - }) - } \ No newline at end of file diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index b64bdb8d28..eaa1c1806f 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -23,6 +23,7 @@ import akka.util.duration._ import spark.{Logging, Utils} import spark.util.AkkaUtils import spark.deploy._ +import spark.metrics.MetricsSystem import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import java.text.SimpleDateFormat import java.util.Date @@ -41,7 +42,7 @@ private[spark] class Worker( memory: Int, masterUrl: String, workDirPath: String = null) - extends Actor with Logging with WorkerInstrumentation { + extends Actor with Logging { Utils.checkHost(host, "Expected hostname") assert (port > 0) @@ -67,6 +68,8 @@ private[spark] class Worker( var coresUsed = 0 var memoryUsed = 0 + val workerInstrumentation = new WorkerInstrumentation(this) + def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed @@ -99,7 +102,8 @@ private[spark] class Worker( connectToMaster() startWebUi() - initialize(this) + Worker.metricsSystem.registerSource(workerInstrumentation) + Worker.metricsSystem.start() } def connectToMaster() { @@ -182,11 +186,13 @@ private[spark] class Worker( executors.values.foreach(_.kill()) webUi.stop() - uninitialize() + Worker.metricsSystem.stop() } } private[spark] object Worker { + private val metricsSystem = MetricsSystem.createMetricsSystem("worker") + def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 2f725300b5..37fd154859 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -1,58 +1,89 @@ package spark.deploy.worker -import com.codahale.metrics.{JmxReporter, Gauge, MetricRegistry} +import com.codahale.metrics.{Gauge, Metric} -import spark.metrics.AbstractInstrumentation +import java.util.{Map, HashMap => JHashMap} -private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { - var workerInst: Option[Worker] = None - val metricRegistry = new MetricRegistry() - - override def registryHandler = metricRegistry - - override def instance = "worker" +import spark.metrics.source.Source + +private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { + val className = classOf[Worker].getName() - def initialize(worker: Worker) { - workerInst = Some(worker) + override def sourceName = "worker" - // Register all the sources - registerSources() + override def getMetrics: Map[String, Metric] = { + val gauges = new JHashMap[String, Metric] - // Register and start all the sinks - registerSinks() - } - - def uninitialize() { - unregisterSinks() + // Gauge for executors number + gauges.put(className + ".executor.number", new Gauge[Int]{ + override def getValue: Int = worker.executors.size + }) + + gauges.put(className + ".core_used.number", new Gauge[Int]{ + override def getValue: Int = worker.coresUsed + }) + + gauges.put(className + ".mem_used.MBytes", new Gauge[Int]{ + override def getValue: Int = worker.memoryUsed + }) + + gauges.put(className + ".core_free.number", new Gauge[Int]{ + override def getValue: Int = worker.coresFree + }) + + gauges.put(className + ".mem_free.MBytes", new Gauge[Int]{ + override def getValue: Int = worker.memoryFree + }) + + gauges } - - // Gauge for executors number - metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.executors.size).getOrElse(0) - }) - - // Gauge for cores used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.coresUsed).getOrElse(0) - }) - - // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) - }) - - // Gauge for cores free of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.coresFree).getOrElse(0) - }) - - // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), - new Gauge[Int] { - override def getValue: Int = workerInst.map(_.memoryFree).getOrElse(0) - }) -} \ No newline at end of file +} +//private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { +// var workerInst: Option[Worker] = None +// val metricRegistry = new MetricRegistry() +// +// override def registryHandler = metricRegistry +// +// override def instance = "worker" +// +// def initialize(worker: Worker) { +// workerInst = Some(worker) +// +// registerSources() +// registerSinks() +// } +// +// def uninitialize() { +// unregisterSinks() +// } +// +// // Gauge for executors number +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.executors.size).getOrElse(0) +// }) +// +// // Gauge for cores used of this worker +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.coresUsed).getOrElse(0) +// }) +// +// // Gauge for memory used of this worker +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) +// }) +// +// // Gauge for cores free of this worker +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.coresFree).getOrElse(0) +// }) +// +// // Gauge for memory used of this worker +// metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), +// new Gauge[Int] { +// override def getValue: Int = workerInst.map(_.memoryFree).getOrElse(0) +// }) +//} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala b/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala deleted file mode 100644 index 9cae1e0220..0000000000 --- a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala +++ /dev/null @@ -1,91 +0,0 @@ -package spark.metrics - -import scala.collection.mutable - -import com.codahale.metrics.{JmxReporter, MetricRegistry} - -import java.util.Properties -import java.util.concurrent.TimeUnit - -import spark.Logging -import spark.metrics.sink._ -import spark.metrics.source._ - -private [spark] trait AbstractInstrumentation extends Logging { - initLogging() - - // Get MetricRegistry handler - def registryHandler: MetricRegistry - // Get the instance name - def instance: String - - val confFile = System.getProperty("spark.metrics.conf.file", MetricsConfig.DEFAULT_CONFIG_FILE) - val metricsConfig = new MetricsConfig(confFile) - - val sinks = new mutable.ArrayBuffer[Sink] - val sources = new mutable.ArrayBuffer[Source] - - def registerSources() { - val instConfig = metricsConfig.getInstance(instance) - val sourceConfigs = MetricsConfig.subProperties(instConfig, AbstractInstrumentation.SOURCE_REGEX) - - // Register all the sources - sourceConfigs.foreach { kv => - val classPath = kv._2.getProperty("class") - try { - val source = Class.forName(classPath).getConstructor(classOf[MetricRegistry]) - .newInstance(registryHandler) - sources += source.asInstanceOf[Source] - } catch { - case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) - } - } - sources.foreach(_.registerSource) - } - - def registerSinks() { - val instConfig = metricsConfig.getInstance(instance) - val sinkConfigs = MetricsConfig.subProperties(instConfig, AbstractInstrumentation.SINK_REGEX) - - // Register JMX sink as a default sink - sinks += new JmxSink(registryHandler) - - // Register other sinks according to conf - sinkConfigs.foreach { kv => - val classPath = if (AbstractInstrumentation.DEFAULT_SINKS.contains(kv._1)) { - AbstractInstrumentation.DEFAULT_SINKS(kv._1) - } else { - // For non-default sink, a property class should be set and create using reflection - kv._2.getProperty("class") - } - try { - val sink = Class.forName(classPath).getConstructor(classOf[Properties], classOf[MetricRegistry]) - .newInstance(kv._2, registryHandler) - sinks += sink.asInstanceOf[Sink] - } catch { - case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e) - } - } - sinks.foreach(_.registerSink) - } - - def unregisterSinks() { - sinks.foreach(_.unregisterSink) - } -} - -object AbstractInstrumentation { - val DEFAULT_SINKS = Map( - "console" -> "spark.metrics.sink.ConsoleSink", - "csv" -> "spark.metrics.sink.CsvSink") - - val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r - val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r - - val timeUnits = Map( - "millisecond" -> TimeUnit.MILLISECONDS, - "second" -> TimeUnit.SECONDS, - "minute" -> TimeUnit.MINUTES, - "hour" -> TimeUnit.HOURS, - "day" -> TimeUnit.DAYS) -} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala new file mode 100644 index 0000000000..ea1bc490b5 --- /dev/null +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -0,0 +1,99 @@ +package spark.metrics + +import scala.collection.mutable + +import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} + +import java.util.Properties +import java.util.concurrent.TimeUnit + +import spark.Logging +import spark.metrics.sink._ +import spark.metrics.source._ + +private[spark] class MetricsSystem private (val instance: String) extends Logging { + initLogging() + + val confFile = System.getProperty("spark.metrics.conf.file", MetricsConfig.DEFAULT_CONFIG_FILE) + val metricsConfig = new MetricsConfig(confFile) + + val sinks = new mutable.ArrayBuffer[Sink] + val sources = new mutable.ArrayBuffer[Source] + + def start() { + registerSources() + registerSinks() + } + + def stop() { + sinks.foreach(_.stop) + } + + def registerSource(source: Source) { + sources += source + MetricsSystem.registry.registerAll(source.asInstanceOf[MetricSet]) + } + + def registerSources() { + val instConfig = metricsConfig.getInstance(instance) + val sourceConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) + + // Register all the sources related to instance + sourceConfigs.foreach { kv => + val classPath = kv._2.getProperty("class") + try { + val source = Class.forName(classPath).newInstance() + sources += source.asInstanceOf[Source] + MetricsSystem.registry.registerAll(source.asInstanceOf[MetricSet]) + } catch { + case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) + } + } + } + + def registerSinks() { + val instConfig = metricsConfig.getInstance(instance) + val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) + + // Register JMX sink as a default sink + sinks += new JmxSink(MetricsSystem.registry) + + // Register other sinks according to conf + sinkConfigs.foreach { kv => + val classPath = if (MetricsSystem.DEFAULT_SINKS.contains(kv._1)) { + MetricsSystem.DEFAULT_SINKS(kv._1) + } else { + // For non-default sink, a property class should be set and create using reflection + kv._2.getProperty("class") + } + try { + val sink = Class.forName(classPath).getConstructor(classOf[Properties], classOf[MetricRegistry]) + .newInstance(kv._2, MetricsSystem.registry) + sinks += sink.asInstanceOf[Sink] + } catch { + case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e) + } + } + sinks.foreach(_.start) + } +} + +private[spark] object MetricsSystem { + val registry = new MetricRegistry() + + val DEFAULT_SINKS = Map( + "console" -> "spark.metrics.sink.ConsoleSink", + "csv" -> "spark.metrics.sink.CsvSink") + + val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r + val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r + + val timeUnits = Map( + "millisecond" -> TimeUnit.MILLISECONDS, + "second" -> TimeUnit.SECONDS, + "minute" -> TimeUnit.MINUTES, + "hour" -> TimeUnit.HOURS, + "day" -> TimeUnit.DAYS) + + def createMetricsSystem(instance: String) = new MetricsSystem(instance) +} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index 5426af8c4c..b49a211fb3 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -5,7 +5,7 @@ import java.util.concurrent.TimeUnit import com.codahale.metrics.{ConsoleReporter, MetricRegistry} -import spark.metrics.AbstractInstrumentation +import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { val pollPeriod = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_PERIOD)) match { @@ -14,13 +14,13 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend } val pollUnit = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_UNIT)) match { - case Some(s) => AbstractInstrumentation.timeUnits(s) - case None => AbstractInstrumentation.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) + case Some(s) => MetricsSystem.timeUnits(s) + case None => MetricsSystem.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) } var reporter: ConsoleReporter = _ - override def registerSink() { + override def start() { reporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) @@ -29,7 +29,7 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend reporter.start(pollPeriod, pollUnit) } - override def unregisterSink() { + override def stop() { reporter.stop() } } diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index 1d663f6cff..3f572c8e05 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -6,7 +6,7 @@ import java.util.concurrent.TimeUnit import com.codahale.metrics.{CsvReporter, MetricRegistry} -import spark.metrics.AbstractInstrumentation +import spark.metrics.MetricsSystem class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { val pollPeriod = Option(property.getProperty(CsvSink.CSV_KEY_PERIOD)) match { @@ -15,8 +15,8 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si } val pollUnit = Option(property.getProperty(CsvSink.CSV_KEY_UNIT)) match { - case Some(s) => AbstractInstrumentation.timeUnits(s) - case None => AbstractInstrumentation.timeUnits(CsvSink.CSV_DEFAULT_UNIT) + case Some(s) => MetricsSystem.timeUnits(s) + case None => MetricsSystem.timeUnits(CsvSink.CSV_DEFAULT_UNIT) } val pollDir = Option(property.getProperty(CsvSink.CSV_KEY_DIR)) match { @@ -26,7 +26,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si var reporter: CsvReporter = _ - override def registerSink() { + override def start() { reporter = CsvReporter.forRegistry(registry) .formatFor(Locale.US) .convertDurationsTo(TimeUnit.MILLISECONDS) @@ -36,7 +36,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si reporter.start(pollPeriod, pollUnit) } - override def unregisterSink() { + override def stop() { reporter.stop() } } diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala index 56e5677700..e223dc26e9 100644 --- a/core/src/main/scala/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -5,12 +5,12 @@ import com.codahale.metrics.{JmxReporter, MetricRegistry} class JmxSink(registry: MetricRegistry) extends Sink { var reporter: JmxReporter = _ - override def registerSink() { + override def start() { reporter = JmxReporter.forRegistry(registry).build() reporter.start() } - override def unregisterSink() { + override def stop() { reporter.stop() } diff --git a/core/src/main/scala/spark/metrics/sink/Sink.scala b/core/src/main/scala/spark/metrics/sink/Sink.scala index 26052b7231..9fef894fde 100644 --- a/core/src/main/scala/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/spark/metrics/sink/Sink.scala @@ -1,7 +1,7 @@ package spark.metrics.sink trait Sink { - def registerSink: Unit + def start: Unit - def unregisterSink: Unit + def stop: Unit } \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala index 8f6bf48843..7a7c1b6ffb 100644 --- a/core/src/main/scala/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -1,17 +1,23 @@ package spark.metrics.source -import com.codahale.metrics.MetricRegistry -import com.codahale.metrics.jvm.{MemoryUsageGaugeSet, GarbageCollectorMetricSet} +import java.util.{Map, HashMap => JHashMap} -class JvmSource(registry: MetricRegistry) extends Source { - // Initialize memory usage gauge for jvm - val memUsageMetricSet = new MemoryUsageGaugeSet - - // Initialize garbage collection usage gauge for jvm - val gcMetricSet = new GarbageCollectorMetricSet - - override def registerSource() { - registry.registerAll(memUsageMetricSet) - registry.registerAll(gcMetricSet) +import com.codahale.metrics.Metric +import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} + +class JvmSource extends Source { + override def sourceName = "jvm" + + override def getMetrics(): Map[String, Metric] = { + val gauges = new JHashMap[String, Metric] + + import scala.collection.JavaConversions._ + val gcMetricSet = new GarbageCollectorMetricSet + gcMetricSet.getMetrics.foreach(kv => gauges.put(kv._1, kv._2)) + + val memGaugeSet = new MemoryUsageGaugeSet + memGaugeSet.getMetrics.foreach(kv => gauges.put(kv._1, kv._2)) + + gauges } } \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/source/Source.scala b/core/src/main/scala/spark/metrics/source/Source.scala index 35cfe0c8ff..edd59de46a 100644 --- a/core/src/main/scala/spark/metrics/source/Source.scala +++ b/core/src/main/scala/spark/metrics/source/Source.scala @@ -1,5 +1,7 @@ package spark.metrics.source -trait Source { - def registerSource: Unit +import com.codahale.metrics.MetricSet + +trait Source extends MetricSet { + def sourceName: String } \ No newline at end of file -- cgit v1.2.3 From 4d6dd67fa1f8f031f1ef46a442cec733fa3b1a7a Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Fri, 28 Jun 2013 02:26:55 +0800 Subject: refactor metrics system 1.change source abstract class to support MetricRegistry 2.change master/work/jvm source class --- .../deploy/master/MasterInstrumentation.scala | 44 +++++----- .../deploy/worker/WorkerInstrumentation.scala | 98 ++++++---------------- .../main/scala/spark/metrics/MetricsSystem.scala | 23 ++--- .../scala/spark/metrics/sink/ConsoleSink.scala | 11 ++- .../main/scala/spark/metrics/sink/CsvSink.scala | 9 +- .../main/scala/spark/metrics/sink/JmxSink.scala | 5 +- .../scala/spark/metrics/source/JvmSource.scala | 22 ++--- .../main/scala/spark/metrics/source/Source.scala | 6 +- 8 files changed, 84 insertions(+), 134 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index 5ea9a90319..46c90b94d2 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -4,32 +4,32 @@ import java.util.{Map, HashMap => JHashMap} import com.codahale.metrics.{Gauge, Metric} +import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} + import spark.metrics.source.Source +import spark.Logging private[spark] class MasterInstrumentation(val master: Master) extends Source { val className = classOf[Master].getName() val instrumentationName = "master" - - override def sourceName = instrumentationName - - override def getMetrics(): Map[String, Metric] = { - val gauges = new JHashMap[String, Metric] - - // Gauge for worker numbers in cluster - gauges.put(className + ".workers.number", new Gauge[Int] { + val metricRegistry = new MetricRegistry() + val sourceName = instrumentationName + + metricRegistry.register(MetricRegistry.name("workers","number"), + new Gauge[Int] { override def getValue: Int = master.workers.size - }) - - // Gauge for application numbers in cluster - gauges.put(className + ".apps.number", new Gauge[Int] { - override def getValue: Int = master.apps.size - }) - - // Gauge for waiting application numbers in cluster - gauges.put(className + ".waiting_apps.number", new Gauge[Int] { + }) + + // Gauge for application numbers in cluster + metricRegistry.register(MetricRegistry.name("apps", "number"), + new Gauge[Int] { + override def getValue: Int = master.apps.size + }) + + // Gauge for waiting application numbers in cluster + metricRegistry.register(MetricRegistry.name("waiting_apps", "number"), + new Gauge[Int] { override def getValue: Int = master.waitingApps.size - }) - - gauges - } -} \ No newline at end of file + }) + +} diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 37fd154859..5ce29cf04c 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -4,86 +4,42 @@ import com.codahale.metrics.{Gauge, Metric} import java.util.{Map, HashMap => JHashMap} +import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import spark.metrics.source.Source private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { val className = classOf[Worker].getName() - override def sourceName = "worker" - - override def getMetrics: Map[String, Metric] = { - val gauges = new JHashMap[String, Metric] - - // Gauge for executors number - gauges.put(className + ".executor.number", new Gauge[Int]{ + val sourceName = "worker" + + val metricRegistry = new MetricRegistry() + + metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), + new Gauge[Int] { override def getValue: Int = worker.executors.size - }) - - gauges.put(className + ".core_used.number", new Gauge[Int]{ + }) + + // Gauge for cores used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), + new Gauge[Int] { override def getValue: Int = worker.coresUsed - }) - - gauges.put(className + ".mem_used.MBytes", new Gauge[Int]{ + }) + + // Gauge for memory used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), + new Gauge[Int] { override def getValue: Int = worker.memoryUsed - }) - - gauges.put(className + ".core_free.number", new Gauge[Int]{ + }) + + // Gauge for cores free of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), + new Gauge[Int] { override def getValue: Int = worker.coresFree - }) - - gauges.put(className + ".mem_free.MBytes", new Gauge[Int]{ + }) + + // Gauge for memory used of this worker + metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), + new Gauge[Int] { override def getValue: Int = worker.memoryFree }) - - gauges - } } -//private[spark] trait WorkerInstrumentation extends AbstractInstrumentation { -// var workerInst: Option[Worker] = None -// val metricRegistry = new MetricRegistry() -// -// override def registryHandler = metricRegistry -// -// override def instance = "worker" -// -// def initialize(worker: Worker) { -// workerInst = Some(worker) -// -// registerSources() -// registerSinks() -// } -// -// def uninitialize() { -// unregisterSinks() -// } -// -// // Gauge for executors number -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.executors.size).getOrElse(0) -// }) -// -// // Gauge for cores used of this worker -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.coresUsed).getOrElse(0) -// }) -// -// // Gauge for memory used of this worker -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.memoryUsed).getOrElse(0) -// }) -// -// // Gauge for cores free of this worker -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.coresFree).getOrElse(0) -// }) -// -// // Gauge for memory used of this worker -// metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), -// new Gauge[Int] { -// override def getValue: Int = workerInst.map(_.memoryFree).getOrElse(0) -// }) -//} \ No newline at end of file diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index ea1bc490b5..a23ccd2692 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -5,6 +5,7 @@ import scala.collection.mutable import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import java.util.Properties +//import java.util._ import java.util.concurrent.TimeUnit import spark.Logging @@ -19,10 +20,13 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val sinks = new mutable.ArrayBuffer[Sink] val sources = new mutable.ArrayBuffer[Source] + var registry = new MetricRegistry() + + registerSources() + registerSinks() def start() { - registerSources() - registerSinks() + sinks.foreach(_.start) } def stop() { @@ -31,20 +35,20 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin def registerSource(source: Source) { sources += source - MetricsSystem.registry.registerAll(source.asInstanceOf[MetricSet]) + registry.register(source.sourceName,source.metricRegistry) } def registerSources() { val instConfig = metricsConfig.getInstance(instance) val sourceConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) - + // Register all the sources related to instance sourceConfigs.foreach { kv => val classPath = kv._2.getProperty("class") try { val source = Class.forName(classPath).newInstance() sources += source.asInstanceOf[Source] - MetricsSystem.registry.registerAll(source.asInstanceOf[MetricSet]) + registerSource(source.asInstanceOf[Source]) } catch { case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) } @@ -56,7 +60,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) // Register JMX sink as a default sink - sinks += new JmxSink(MetricsSystem.registry) + sinks += new JmxSink(registry) // Register other sinks according to conf sinkConfigs.foreach { kv => @@ -68,19 +72,16 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin } try { val sink = Class.forName(classPath).getConstructor(classOf[Properties], classOf[MetricRegistry]) - .newInstance(kv._2, MetricsSystem.registry) + .newInstance(kv._2, registry) sinks += sink.asInstanceOf[Sink] } catch { case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e) } } - sinks.foreach(_.start) } } private[spark] object MetricsSystem { - val registry = new MetricRegistry() - val DEFAULT_SINKS = Map( "console" -> "spark.metrics.sink.ConsoleSink", "csv" -> "spark.metrics.sink.CsvSink") @@ -96,4 +97,4 @@ private[spark] object MetricsSystem { "day" -> TimeUnit.DAYS) def createMetricsSystem(instance: String) = new MetricsSystem(instance) -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index b49a211fb3..9cd17556fa 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -18,14 +18,13 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend case None => MetricsSystem.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) } - var reporter: ConsoleReporter = _ - - override def start() { - reporter = ConsoleReporter.forRegistry(registry) + var reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) .build() - + + + override def start() { reporter.start(pollPeriod, pollUnit) } @@ -40,4 +39,4 @@ object ConsoleSink { val CONSOLE_KEY_PERIOD = "period" val CONSOLE_KEY_UNIT = "unit" -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index 3f572c8e05..62e51be0dc 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -24,15 +24,14 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si case None => CsvSink.CSV_DEFAULT_DIR } - var reporter: CsvReporter = _ - - override def start() { - reporter = CsvReporter.forRegistry(registry) + var reporter: CsvReporter = CsvReporter.forRegistry(registry) .formatFor(Locale.US) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) .build(new File(pollDir)) - + + + override def start() { reporter.start(pollPeriod, pollUnit) } diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala index e223dc26e9..98b55f7b7f 100644 --- a/core/src/main/scala/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -3,10 +3,9 @@ package spark.metrics.sink import com.codahale.metrics.{JmxReporter, MetricRegistry} class JmxSink(registry: MetricRegistry) extends Sink { - var reporter: JmxReporter = _ + var reporter: JmxReporter = JmxReporter.forRegistry(registry).build() override def start() { - reporter = JmxReporter.forRegistry(registry).build() reporter.start() } @@ -14,4 +13,4 @@ class JmxSink(registry: MetricRegistry) extends Sink { reporter.stop() } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala index 7a7c1b6ffb..13270dae3c 100644 --- a/core/src/main/scala/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -2,22 +2,16 @@ package spark.metrics.source import java.util.{Map, HashMap => JHashMap} -import com.codahale.metrics.Metric +import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} class JvmSource extends Source { - override def sourceName = "jvm" - - override def getMetrics(): Map[String, Metric] = { - val gauges = new JHashMap[String, Metric] - - import scala.collection.JavaConversions._ + val sourceName = "jvm" + val metricRegistry = new MetricRegistry() + val gcMetricSet = new GarbageCollectorMetricSet - gcMetricSet.getMetrics.foreach(kv => gauges.put(kv._1, kv._2)) - val memGaugeSet = new MemoryUsageGaugeSet - memGaugeSet.getMetrics.foreach(kv => gauges.put(kv._1, kv._2)) - - gauges - } -} \ No newline at end of file + + metricRegistry.registerAll(gcMetricSet) + metricRegistry.registerAll(memGaugeSet) +} diff --git a/core/src/main/scala/spark/metrics/source/Source.scala b/core/src/main/scala/spark/metrics/source/Source.scala index edd59de46a..17cbe2f85a 100644 --- a/core/src/main/scala/spark/metrics/source/Source.scala +++ b/core/src/main/scala/spark/metrics/source/Source.scala @@ -1,7 +1,9 @@ package spark.metrics.source import com.codahale.metrics.MetricSet +import com.codahale.metrics.MetricRegistry -trait Source extends MetricSet { +trait Source { def sourceName: String -} \ No newline at end of file + def metricRegistry: MetricRegistry +} -- cgit v1.2.3 From 7fb574bf666661fdf8a786de779f85efe2f15f0c Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 10:14:30 +0800 Subject: Code clean and remarshal --- core/src/main/scala/spark/deploy/master/Master.scala | 9 +++++---- .../spark/deploy/master/MasterInstrumentation.scala | 11 ++--------- .../spark/deploy/worker/WorkerInstrumentation.scala | 18 ++++++------------ core/src/main/scala/spark/metrics/MetricsSystem.scala | 5 ++--- .../main/scala/spark/metrics/sink/ConsoleSink.scala | 1 - core/src/main/scala/spark/metrics/sink/CsvSink.scala | 1 - core/src/main/scala/spark/metrics/sink/Sink.scala | 1 - 7 files changed, 15 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index e44f5e3168..cc0b2d4295 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -78,6 +78,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) Master.metricsSystem.registerSource(masterInstrumentation) + Master.metricsSystem.start() } override def postStop() { @@ -321,22 +322,22 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act removeWorker(worker) } } + + override def postStop() { + Master.metricsSystem.stop() + } } private[spark] object Master { private val systemName = "sparkMaster" private val actorName = "Master" private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r - private val metricsSystem = MetricsSystem.createMetricsSystem("master") def main(argStrings: Array[String]) { val args = new MasterArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort) - - metricsSystem.start() actorSystem.awaitTermination() - metricsSystem.stop() } /** Returns an `akka://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */ diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index 46c90b94d2..61a561c955 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -1,19 +1,12 @@ package spark.deploy.master -import java.util.{Map, HashMap => JHashMap} - -import com.codahale.metrics.{Gauge, Metric} - -import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} +import com.codahale.metrics.{Gauge,MetricRegistry} import spark.metrics.source.Source -import spark.Logging private[spark] class MasterInstrumentation(val master: Master) extends Source { - val className = classOf[Master].getName() - val instrumentationName = "master" val metricRegistry = new MetricRegistry() - val sourceName = instrumentationName + val sourceName = "master" metricRegistry.register(MetricRegistry.name("workers","number"), new Gauge[Int] { diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 5ce29cf04c..94c20a98c1 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -1,44 +1,38 @@ package spark.deploy.worker -import com.codahale.metrics.{Gauge, Metric} +import com.codahale.metrics.{Gauge, MetricRegistry} -import java.util.{Map, HashMap => JHashMap} - -import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import spark.metrics.source.Source private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { - val className = classOf[Worker].getName() - val sourceName = "worker" - val metricRegistry = new MetricRegistry() - metricRegistry.register(MetricRegistry.name(classOf[Worker], "executor", "number"), + metricRegistry.register(MetricRegistry.name("executor", "number"), new Gauge[Int] { override def getValue: Int = worker.executors.size }) // Gauge for cores used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_used", "number"), + metricRegistry.register(MetricRegistry.name("core_used", "number"), new Gauge[Int] { override def getValue: Int = worker.coresUsed }) // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_used", "MBytes"), + metricRegistry.register(MetricRegistry.name("mem_used", "MBytes"), new Gauge[Int] { override def getValue: Int = worker.memoryUsed }) // Gauge for cores free of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "core_free", "number"), + metricRegistry.register(MetricRegistry.name("core_free", "number"), new Gauge[Int] { override def getValue: Int = worker.coresFree }) // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name(classOf[Worker], "mem_free", "MBytes"), + metricRegistry.register(MetricRegistry.name("mem_free", "MBytes"), new Gauge[Int] { override def getValue: Int = worker.memoryFree }) diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index a23ccd2692..5bfdc00eaf 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -5,7 +5,6 @@ import scala.collection.mutable import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import java.util.Properties -//import java.util._ import java.util.concurrent.TimeUnit import spark.Logging @@ -20,7 +19,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val sinks = new mutable.ArrayBuffer[Sink] val sources = new mutable.ArrayBuffer[Source] - var registry = new MetricRegistry() + val registry = new MetricRegistry() registerSources() registerSinks() @@ -35,7 +34,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin def registerSource(source: Source) { sources += source - registry.register(source.sourceName,source.metricRegistry) + registry.register(source.sourceName, source.metricRegistry) } def registerSources() { diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index 9cd17556fa..e2e4197d1d 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -23,7 +23,6 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend .convertRatesTo(TimeUnit.SECONDS) .build() - override def start() { reporter.start(pollPeriod, pollUnit) } diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index 62e51be0dc..c2d645331c 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -30,7 +30,6 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si .convertRatesTo(TimeUnit.SECONDS) .build(new File(pollDir)) - override def start() { reporter.start(pollPeriod, pollUnit) } diff --git a/core/src/main/scala/spark/metrics/sink/Sink.scala b/core/src/main/scala/spark/metrics/sink/Sink.scala index 9fef894fde..3ffdcbdaba 100644 --- a/core/src/main/scala/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/spark/metrics/sink/Sink.scala @@ -2,6 +2,5 @@ package spark.metrics.sink trait Sink { def start: Unit - def stop: Unit } \ No newline at end of file -- cgit v1.2.3 From 576528f0f916cc0434972cbbc9321e4dcec45628 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 10:25:48 +0800 Subject: Add dependency of Codahale's metrics library --- project/SparkBuild.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f3f67b57c8..825319d3de 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -180,6 +180,8 @@ object SparkBuild extends Build { "org.apache.mesos" % "mesos" % "0.9.0-incubating", "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test" + "com.codahale.metrics" % "metrics-core" % "3.0.0", + "com.codahale.metrics" % "metrics-jvm" % "3.0.0" ) ++ ( if (HADOOP_MAJOR_VERSION == "2") { if (HADOOP_YARN) { -- cgit v1.2.3 From 871bc1687eaeb59df24b4778c5992a5f7f105cc8 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 14:46:24 +0800 Subject: Add Executor instrumentation --- .../spark/executor/ExecutorInstrumentation.scala | 35 ++++++++++++++++++++++ .../spark/executor/MesosExecutorBackend.scala | 10 ++++++- .../spark/executor/StandaloneExecutorBackend.scala | 12 ++++++++ 3 files changed, 56 insertions(+), 1 deletion(-) create mode 100644 core/src/main/scala/spark/executor/ExecutorInstrumentation.scala diff --git a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala new file mode 100644 index 0000000000..80aadb66b0 --- /dev/null +++ b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala @@ -0,0 +1,35 @@ +package spark.executor + +import com.codahale.metrics.{Gauge, MetricRegistry} + +import spark.metrics.source.Source + +class ExecutorInstrumentation(val executor: Option[Executor]) extends Source{ + val metricRegistry = new MetricRegistry() + val sourceName = "executor" + + // Gauge for executor thread pool's actively executing task counts + metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), + new Gauge[Int] { + override def getValue: Int = executor.map(_.threadPool.getActiveCount()).getOrElse(0) + }) + + // Gauge for executor thread pool's approximate total number of tasks that have been completed + metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), + new Gauge[Long] { + override def getValue: Long = executor.map(_.threadPool.getCompletedTaskCount()).getOrElse(0) + }) + + // Gauge for executor thread pool's current number of threads + metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), + new Gauge[Int] { + override def getValue: Int = executor.map(_.threadPool.getPoolSize()).getOrElse(0) + }) + + // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool + metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), + new Gauge[Int] { + override def getValue: Int = executor.map(_.threadPool.getMaximumPoolSize()).getOrElse(0) + }) + +} \ No newline at end of file diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala index 4961c42fad..8b6ab0c391 100644 --- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala @@ -24,6 +24,7 @@ import spark.TaskState.TaskState import com.google.protobuf.ByteString import spark.{Utils, Logging} import spark.TaskState +import spark.metrics.MetricsSystem private[spark] class MesosExecutorBackend extends MesosExecutor @@ -32,6 +33,9 @@ private[spark] class MesosExecutorBackend var executor: Executor = null var driver: ExecutorDriver = null + + val executorInstrumentation = new ExecutorInstrumentation(Option(executor)) + MesosExecutorBackend.metricsSystem.start() override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { val mesosTaskId = TaskID.newBuilder().setValue(taskId.toString).build() @@ -79,13 +83,17 @@ private[spark] class MesosExecutorBackend override def frameworkMessage(d: ExecutorDriver, data: Array[Byte]) {} - override def shutdown(d: ExecutorDriver) {} + override def shutdown(d: ExecutorDriver) { + MesosExecutorBackend.metricsSystem.stop() + } } /** * Entry point for Mesos executor. */ private[spark] object MesosExecutorBackend { + private val metricsSystem = MetricsSystem.createMetricsSystem("executor") + def main(args: Array[String]) { MesosNativeLibrary.load() // Create a new Executor and start it running diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index f4003da732..6ef74cd2ff 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -24,6 +24,7 @@ import spark.util.AkkaUtils import akka.actor.{ActorRef, Actor, Props, Terminated} import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import java.util.concurrent.{TimeUnit, ThreadPoolExecutor, SynchronousQueue} +import spark.metrics.MetricsSystem import spark.scheduler.cluster._ import spark.scheduler.cluster.RegisteredExecutor import spark.scheduler.cluster.LaunchTask @@ -45,6 +46,8 @@ private[spark] class StandaloneExecutorBackend( var executor: Executor = null var driver: ActorRef = null + + val executorInstrumentation = new ExecutorInstrumentation(Option(executor)) override def preStart() { logInfo("Connecting to driver: " + driverUrl) @@ -52,6 +55,9 @@ private[spark] class StandaloneExecutorBackend( driver ! RegisterExecutor(executorId, hostPort, cores) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) context.watch(driver) // Doesn't work with remote actors, but useful for testing + + StandaloneExecutorBackend.metricsSystem.registerSource(executorInstrumentation) + StandaloneExecutorBackend.metricsSystem.start() } override def receive = { @@ -81,9 +87,15 @@ private[spark] class StandaloneExecutorBackend( override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { driver ! StatusUpdate(executorId, taskId, state, data) } + + override def postStop() { + StandaloneExecutorBackend.metricsSystem.stop() + } } private[spark] object StandaloneExecutorBackend { + private val metricsSystem = MetricsSystem.createMetricsSystem("executor") + def run(driverUrl: String, executorId: String, hostname: String, cores: Int) { SparkHadoopUtil.runAsUser(run0, Tuple4[Any, Any, Any, Any] (driverUrl, executorId, hostname, cores)) } -- cgit v1.2.3 From 5ce5dc9fcd7acf5c58dd3d456a629b01d57514e4 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 14:48:21 +0800 Subject: Add default properties to deal with no configure file situation --- .../main/scala/spark/metrics/MetricsConfig.scala | 28 ++++++++++++++-------- .../main/scala/spark/metrics/MetricsSystem.scala | 9 ++----- .../scala/spark/metrics/sink/ConsoleSink.scala | 6 ++--- .../main/scala/spark/metrics/sink/CsvSink.scala | 6 ++--- .../main/scala/spark/metrics/sink/JmxSink.scala | 6 +++-- 5 files changed, 30 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index be4f670918..7405192058 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -1,20 +1,25 @@ package spark.metrics import java.util.Properties -import java.io.FileInputStream +import java.io.{File, FileInputStream} import scala.collection.mutable import scala.util.matching.Regex private [spark] class MetricsConfig(val configFile: String) { val properties = new Properties() - var fis: FileInputStream = _ + // Add default properties in case there's no properties file + MetricsConfig.setDefaultProperties(properties) - try { - fis = new FileInputStream(configFile) - properties.load(fis) - } finally { - fis.close() + val confFile = new File(configFile) + if (confFile.exists()) { + var fis: FileInputStream = null + try { + fis = new FileInputStream(configFile) + properties.load(fis) + } finally { + fis.close() + } } val propertyCategories = MetricsConfig.subProperties(properties, MetricsConfig.INSTANCE_REGEX) @@ -35,11 +40,15 @@ private [spark] class MetricsConfig(val configFile: String) { } } -object MetricsConfig { - val DEFAULT_CONFIG_FILE = "conf/metrics.properties" +private[spark] object MetricsConfig { val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r + def setDefaultProperties(prop: Properties) { + prop.setProperty("*.sink.jmx.enabled", "default") + prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") + } + def subProperties(prop: Properties, regex: Regex) = { val subProperties = new mutable.HashMap[String, Properties] @@ -48,7 +57,6 @@ object MetricsConfig { if (regex.findPrefixOf(kv._1) != None) { val regex(prefix, suffix) = kv._1 subProperties.getOrElseUpdate(prefix, new Properties).setProperty(suffix, kv._2) - println(">>>>>subProperties added " + prefix + " " + suffix + " " + kv._2) } } diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index 5bfdc00eaf..6e448cb2a5 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -14,7 +14,7 @@ import spark.metrics.source._ private[spark] class MetricsSystem private (val instance: String) extends Logging { initLogging() - val confFile = System.getProperty("spark.metrics.conf.file", MetricsConfig.DEFAULT_CONFIG_FILE) + val confFile = System.getProperty("spark.metrics.conf.file", "unsupported") val metricsConfig = new MetricsConfig(confFile) val sinks = new mutable.ArrayBuffer[Sink] @@ -58,9 +58,6 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val instConfig = metricsConfig.getInstance(instance) val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) - // Register JMX sink as a default sink - sinks += new JmxSink(registry) - // Register other sinks according to conf sinkConfigs.foreach { kv => val classPath = if (MetricsSystem.DEFAULT_SINKS.contains(kv._1)) { @@ -81,9 +78,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin } private[spark] object MetricsSystem { - val DEFAULT_SINKS = Map( - "console" -> "spark.metrics.sink.ConsoleSink", - "csv" -> "spark.metrics.sink.CsvSink") + val DEFAULT_SINKS = Map("jmx" -> "spark.metrics.sink.JmxSink") val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index e2e4197d1d..d7b7a9e501 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -1,10 +1,10 @@ package spark.metrics.sink +import com.codahale.metrics.{ConsoleReporter, MetricRegistry} + import java.util.Properties import java.util.concurrent.TimeUnit -import com.codahale.metrics.{ConsoleReporter, MetricRegistry} - import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { @@ -18,7 +18,7 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry) extend case None => MetricsSystem.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) } - var reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) + val reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) .build() diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index c2d645331c..e6c5bffd3c 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -1,11 +1,11 @@ package spark.metrics.sink +import com.codahale.metrics.{CsvReporter, MetricRegistry} + import java.io.File import java.util.{Locale, Properties} import java.util.concurrent.TimeUnit -import com.codahale.metrics.{CsvReporter, MetricRegistry} - import spark.metrics.MetricsSystem class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { @@ -24,7 +24,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si case None => CsvSink.CSV_DEFAULT_DIR } - var reporter: CsvReporter = CsvReporter.forRegistry(registry) + val reporter: CsvReporter = CsvReporter.forRegistry(registry) .formatFor(Locale.US) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala index 98b55f7b7f..f097a631c0 100644 --- a/core/src/main/scala/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -1,9 +1,11 @@ package spark.metrics.sink +import java.util.Properties + import com.codahale.metrics.{JmxReporter, MetricRegistry} -class JmxSink(registry: MetricRegistry) extends Sink { - var reporter: JmxReporter = JmxReporter.forRegistry(registry).build() +class JmxSink(val property: Properties, val registry: MetricRegistry) extends Sink { + val reporter: JmxReporter = JmxReporter.forRegistry(registry).build() override def start() { reporter.start() -- cgit v1.2.3 From e080588f7396d9612ea5d909e59f2364c139103a Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 1 Jul 2013 17:04:06 +0800 Subject: Add metrics system unit test --- .../scala/spark/metrics/MetricsConfigSuite.scala | 92 ++++++++++++++++++++++ .../scala/spark/metrics/MetricsSystemSuite.scala | 65 +++++++++++++++ 2 files changed, 157 insertions(+) create mode 100644 core/src/test/scala/spark/metrics/MetricsConfigSuite.scala create mode 100644 core/src/test/scala/spark/metrics/MetricsSystemSuite.scala diff --git a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala new file mode 100644 index 0000000000..0c7142c418 --- /dev/null +++ b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala @@ -0,0 +1,92 @@ +package spark.metrics + +import java.util.Properties +import java.io.{File, FileOutputStream} + +import org.scalatest.{BeforeAndAfter, FunSuite} + +import spark.metrics._ + +class MetricsConfigSuite extends FunSuite with BeforeAndAfter { + var filePath: String = _ + + before { + val prop = new Properties() + + prop.setProperty("*.sink.console.period", "10") + prop.setProperty("*.sink.console.unit", "second") + prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") + prop.setProperty("master.sink.console.period", "20") + prop.setProperty("master.sink.console.unit", "minute") + + val dir = new File("/tmp") + filePath = if (dir.isDirectory() && dir.exists() && dir.canWrite()) { + "/tmp/test_metrics.properties" + } else { + "./test_metrics.properties" + } + + val os = new FileOutputStream(new File(filePath)) + prop.store(os, "for test") + os.close() + } + + test("MetricsConfig with default properties") { + val conf = new MetricsConfig("dummy-file") + assert(conf.properties.size() === 2) + assert(conf.properties.getProperty("*.sink.jmx.enabled") === "default") + assert(conf.properties.getProperty("*.source.jvm.class") === "spark.metrics.source.JvmSource") + assert(conf.properties.getProperty("test-for-dummy") === null) + + val property = conf.getInstance("random") + assert(property.size() === 2) + assert(property.getProperty("sink.jmx.enabled") === "default") + assert(property.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") + } + + test("MetricsConfig with properties set") { + val conf = new MetricsConfig(filePath) + + val masterProp = conf.getInstance("master") + assert(masterProp.size() === 4) + assert(masterProp.getProperty("sink.console.period") === "20") + assert(masterProp.getProperty("sink.console.unit") === "minute") + assert(masterProp.getProperty("sink.jmx.enabled") === "default") + assert(masterProp.getProperty("source.jvm.class") == "spark.metrics.source.JvmSource") + + val workerProp = conf.getInstance("worker") + assert(workerProp.size() === 4) + assert(workerProp.getProperty("sink.console.period") === "10") + assert(workerProp.getProperty("sink.console.unit") === "second") + } + + test("MetricsConfig with subProperties") { + val conf = new MetricsConfig(filePath) + + val propCategories = conf.propertyCategories + assert(propCategories.size === 2) + + val masterProp = conf.getInstance("master") + val sourceProps = MetricsConfig.subProperties(masterProp, MetricsSystem.SOURCE_REGEX) + assert(sourceProps.size === 1) + assert(sourceProps("jvm").getProperty("class") === "spark.metrics.source.JvmSource") + + val sinkProps = MetricsConfig.subProperties(masterProp, MetricsSystem.SINK_REGEX) + assert(sinkProps.size === 2) + assert(sinkProps.contains("console")) + assert(sinkProps.contains("jmx")) + + val consoleProps = sinkProps("console") + assert(consoleProps.size() === 2) + + val jmxProps = sinkProps("jmx") + assert(jmxProps.size() === 1) + } + + after { + val file = new File(filePath) + if (file.exists()) { + file.delete() + } + } +} \ No newline at end of file diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala new file mode 100644 index 0000000000..5e8f8fcf80 --- /dev/null +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -0,0 +1,65 @@ +package spark.metrics + +import java.util.Properties +import java.io.{File, FileOutputStream} + +import org.scalatest.{BeforeAndAfter, FunSuite} + +import spark.metrics._ + +class MetricsSystemSuite extends FunSuite with BeforeAndAfter { + var filePath: String = _ + + before { + val props = new Properties() + props.setProperty("*.sink.console.period", "10") + props.setProperty("*.sink.console.unit", "second") + props.setProperty("test.sink.console.class", "spark.metrics.sink.ConsoleSink") + props.setProperty("test.sink.dummy.class", "spark.metrics.sink.DummySink") + props.setProperty("test.source.dummy.class", "spark.metrics.source.DummySource") + props.setProperty("test.sink.console.period", "20") + props.setProperty("test.sink.console.unit", "minute") + + val dir = new File("/tmp") + filePath = if (dir.isDirectory() && dir.exists() && dir.canWrite()) { + "/tmp/test_metrics.properties" + } else { + "./test_metrics.properties" + } + + val os = new FileOutputStream(new File(filePath)) + props.store(os, "for test") + os.close() + System.setProperty("spark.metrics.conf.file", filePath) + } + + test("MetricsSystem with default config") { + val metricsSystem = MetricsSystem.createMetricsSystem("default") + val sources = metricsSystem.sources + val sinks = metricsSystem.sinks + + assert(sources.length === 1) + assert(sinks.length === 1) + assert(sources(0).sourceName === "jvm") + } + + test("MetricsSystem with sources add") { + val metricsSystem = MetricsSystem.createMetricsSystem("test") + val sources = metricsSystem.sources + val sinks = metricsSystem.sinks + + assert(sources.length === 1) + assert(sinks.length === 2) + + val source = new spark.deploy.master.MasterInstrumentation(null) + metricsSystem.registerSource(source) + assert(sources.length === 2) + } + + after { + val file = new File(filePath) + if (file.exists()) { + file.delete() + } + } +} \ No newline at end of file -- cgit v1.2.3 From e9ac88754d4c5d58aedd4de8768787300b15eada Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 1 Jul 2013 17:05:18 +0800 Subject: Remove twice add Source bug and code clean --- core/src/main/scala/spark/metrics/MetricsSystem.scala | 10 ++++------ core/src/main/scala/spark/metrics/source/JvmSource.scala | 12 ++++++------ 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index 6e448cb2a5..bf4487e0fc 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -1,15 +1,15 @@ package spark.metrics -import scala.collection.mutable - import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import java.util.Properties import java.util.concurrent.TimeUnit +import scala.collection.mutable + import spark.Logging -import spark.metrics.sink._ -import spark.metrics.source._ +import spark.metrics.sink.Sink +import spark.metrics.source.Source private[spark] class MetricsSystem private (val instance: String) extends Logging { initLogging() @@ -46,7 +46,6 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val classPath = kv._2.getProperty("class") try { val source = Class.forName(classPath).newInstance() - sources += source.asInstanceOf[Source] registerSource(source.asInstanceOf[Source]) } catch { case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) @@ -58,7 +57,6 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val instConfig = metricsConfig.getInstance(instance) val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) - // Register other sinks according to conf sinkConfigs.foreach { kv => val classPath = if (MetricsSystem.DEFAULT_SINKS.contains(kv._1)) { MetricsSystem.DEFAULT_SINKS(kv._1) diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala index 13270dae3c..8f0870c1a0 100644 --- a/core/src/main/scala/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -1,17 +1,17 @@ package spark.metrics.source -import java.util.{Map, HashMap => JHashMap} - import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} +import java.util.{Map, HashMap => JHashMap} + class JvmSource extends Source { val sourceName = "jvm" val metricRegistry = new MetricRegistry() - val gcMetricSet = new GarbageCollectorMetricSet - val memGaugeSet = new MemoryUsageGaugeSet + val gcMetricSet = new GarbageCollectorMetricSet + val memGaugeSet = new MemoryUsageGaugeSet - metricRegistry.registerAll(gcMetricSet) - metricRegistry.registerAll(memGaugeSet) + metricRegistry.registerAll(gcMetricSet) + metricRegistry.registerAll(memGaugeSet) } -- cgit v1.2.3 From 7d2eada451686824bd467641bf1763e82011f2a6 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Mon, 1 Jul 2013 21:37:21 +0800 Subject: Add metrics source of DAGScheduler and blockManager Conflicts: core/src/main/scala/spark/SparkContext.scala core/src/main/scala/spark/SparkEnv.scala --- core/src/main/scala/spark/SparkContext.scala | 13 ++++++-- core/src/main/scala/spark/SparkEnv.scala | 6 ++++ .../main/scala/spark/scheduler/DAGScheduler.scala | 3 +- .../scala/spark/scheduler/DAGSchedulerSource.scala | 31 +++++++++++++++++++ .../scala/spark/storage/BlockManagerSource.scala | 35 ++++++++++++++++++++++ 5 files changed, 84 insertions(+), 4 deletions(-) create mode 100644 core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala create mode 100644 core/src/main/scala/spark/storage/BlockManagerSource.scala diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 24ba605646..a6128a9f30 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -60,11 +60,11 @@ import org.apache.mesos.MesosNativeLibrary import spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import spark.partial.{ApproximateEvaluator, PartialResult} import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD} -import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler} +import spark.scheduler.{DAGScheduler, DAGSchedulerSource, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler} import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler} import spark.scheduler.local.LocalScheduler import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import spark.storage.{StorageStatus, StorageUtils, RDDInfo} +import spark.storage.{StorageStatus, StorageUtils, RDDInfo, BlockManagerSource} import spark.util.{MetadataCleaner, TimeStampedHashMap} import ui.{SparkUI} @@ -270,6 +270,15 @@ class SparkContext( // Post init taskScheduler.postStartHook() + val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) + val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) + def initDriverMetrics() = { + SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) + SparkEnv.get.metricsSystem.registerSource(blockManagerSource) + } + + initDriverMetrics() + // Methods for creating RDDs /** Distribute a local Scala collection to form an RDD. */ diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index f2bdc11bdb..d34dafecc5 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -30,6 +30,7 @@ import spark.network.ConnectionManager import spark.serializer.{Serializer, SerializerManager} import spark.util.AkkaUtils import spark.api.python.PythonWorkerFactory +import spark.metrics._ /** @@ -53,6 +54,7 @@ class SparkEnv ( val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, + val metricsSystem: MetricsSystem, // To be set only as part of initialization of SparkContext. // (executorId, defaultHostPort) => executorHostPort // If executorId is NOT found, return defaultHostPort @@ -184,6 +186,9 @@ object SparkEnv extends Logging { httpFileServer.initialize() System.setProperty("spark.fileserver.uri", httpFileServer.serverUri) + val metricsSystem = MetricsSystem.createMetricsSystem("driver") + metricsSystem.start() + // Set the sparkFiles directory, used when downloading dependencies. In local mode, // this is a temporary directory; in distributed mode, this is the executor's current working // directory. @@ -213,6 +218,7 @@ object SparkEnv extends Logging { connectionManager, httpFileServer, sparkFilesDir, + metricsSystem, None) } } diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 9b45fc2938..781e49bdec 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -30,7 +30,7 @@ import spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialRe import spark.scheduler.cluster.TaskInfo import spark.storage.{BlockManager, BlockManagerMaster} import spark.util.{MetadataCleaner, TimeStampedHashMap} - +import spark.metrics.MetricsSystem /** * A Scheduler subclass that implements stage-oriented scheduling. It computes a DAG of stages for * each job, keeps track of which RDDs and stage outputs are materialized, and computes a minimal @@ -126,7 +126,6 @@ class DAGScheduler( val resultStageToJob = new HashMap[Stage, ActiveJob] val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup) - // Start a thread to run the DAGScheduler event loop def start() { new Thread("DAGScheduler") { diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala new file mode 100644 index 0000000000..57aa74512c --- /dev/null +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala @@ -0,0 +1,31 @@ +package spark.scheduler + +import com.codahale.metrics.{Gauge,MetricRegistry} + +import spark.metrics.source.Source + +private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends Source { + val metricRegistry = new MetricRegistry() + val sourceName = "DAGScheduler" + + + metricRegistry.register(MetricRegistry.name("stage","failedStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.failed.size + }) + + metricRegistry.register(MetricRegistry.name("stage","runningStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.running.size + }) + + metricRegistry.register(MetricRegistry.name("stage","waitingStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.waiting.size + }) + + metricRegistry.register(MetricRegistry.name("job","allJobs"), new Gauge[Int] { + override def getValue: Int = dagScheduler.nextRunId.get() + }) + + metricRegistry.register(MetricRegistry.name("job","ActiveJobs"), new Gauge[Int] { + override def getValue: Int = dagScheduler.activeJobs.size + }) +} diff --git a/core/src/main/scala/spark/storage/BlockManagerSource.scala b/core/src/main/scala/spark/storage/BlockManagerSource.scala new file mode 100644 index 0000000000..c0ce9259c8 --- /dev/null +++ b/core/src/main/scala/spark/storage/BlockManagerSource.scala @@ -0,0 +1,35 @@ +package spark.storage + +import com.codahale.metrics.{Gauge,MetricRegistry} + +import spark.metrics.source.Source +import spark.storage._ + +private[spark] class BlockManagerSource(val blockManager: BlockManager) extends Source { + val metricRegistry = new MetricRegistry() + val sourceName = "BlockManager" + + metricRegistry.register(MetricRegistry.name("memory","maxMem"), new Gauge[Long] { + override def getValue: Long = { + val storageStatusList = blockManager.master.getStorageStatus + val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) + maxMem + } + }) + + metricRegistry.register(MetricRegistry.name("memory","remainingMem"), new Gauge[Long] { + override def getValue: Long = { + val storageStatusList = blockManager.master.getStorageStatus + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + remainingMem + } + }) + + metricRegistry.register(MetricRegistry.name("disk","diskSpaceUsed"), new Gauge[Long] { + override def getValue: Long = { + val storageStatusList = blockManager.master.getStorageStatus + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).reduceOption(_+_).getOrElse(0L) + diskSpaceUsed + } + }) +} -- cgit v1.2.3 From 9cea0c28184c86625f8281eea2af77eff15acb73 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Mon, 1 Jul 2013 22:22:29 +0800 Subject: Refactor metricsSystem unit test, add resource files. --- .../test/resources/test_metrics_config.properties | 6 +++++ .../test/resources/test_metrics_system.properties | 7 +++++ .../scala/spark/metrics/MetricsConfigSuite.scala | 30 +++------------------ .../scala/spark/metrics/MetricsSystemSuite.scala | 31 +++------------------- 4 files changed, 19 insertions(+), 55 deletions(-) create mode 100644 core/src/test/resources/test_metrics_config.properties create mode 100644 core/src/test/resources/test_metrics_system.properties diff --git a/core/src/test/resources/test_metrics_config.properties b/core/src/test/resources/test_metrics_config.properties new file mode 100644 index 0000000000..2011940003 --- /dev/null +++ b/core/src/test/resources/test_metrics_config.properties @@ -0,0 +1,6 @@ +*.sink.console.period = 10 +*.sink.console.unit = second +*.source.jvm.class = spark.metrics.source.JvmSource +master.sink.console.period = 20 +master.sink.console.unit = minute + diff --git a/core/src/test/resources/test_metrics_system.properties b/core/src/test/resources/test_metrics_system.properties new file mode 100644 index 0000000000..06afbc6625 --- /dev/null +++ b/core/src/test/resources/test_metrics_system.properties @@ -0,0 +1,7 @@ +*.sink.console.period = 10 +*.sink.console.unit = second +test.sink.console.class = spark.metrics.sink.ConsoleSink +test.sink.dummy.class = spark.metrics.sink.DummySink +test.source.dummy.class = spark.metrics.source.DummySource +test.sink.console.period = 20 +test.sink.console.unit = minute diff --git a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala index 0c7142c418..f4c83cb644 100644 --- a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala @@ -11,24 +11,7 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ before { - val prop = new Properties() - - prop.setProperty("*.sink.console.period", "10") - prop.setProperty("*.sink.console.unit", "second") - prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") - prop.setProperty("master.sink.console.period", "20") - prop.setProperty("master.sink.console.unit", "minute") - - val dir = new File("/tmp") - filePath = if (dir.isDirectory() && dir.exists() && dir.canWrite()) { - "/tmp/test_metrics.properties" - } else { - "./test_metrics.properties" - } - - val os = new FileOutputStream(new File(filePath)) - prop.store(os, "for test") - os.close() + filePath = getClass.getClassLoader.getResource("test_metrics_config.properties").getFile() } test("MetricsConfig with default properties") { @@ -81,12 +64,5 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { val jmxProps = sinkProps("jmx") assert(jmxProps.size() === 1) - } - - after { - val file = new File(filePath) - if (file.exists()) { - file.delete() - } - } -} \ No newline at end of file + } +} diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala index 5e8f8fcf80..967be6ec47 100644 --- a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -11,25 +11,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ before { - val props = new Properties() - props.setProperty("*.sink.console.period", "10") - props.setProperty("*.sink.console.unit", "second") - props.setProperty("test.sink.console.class", "spark.metrics.sink.ConsoleSink") - props.setProperty("test.sink.dummy.class", "spark.metrics.sink.DummySink") - props.setProperty("test.source.dummy.class", "spark.metrics.source.DummySource") - props.setProperty("test.sink.console.period", "20") - props.setProperty("test.sink.console.unit", "minute") - - val dir = new File("/tmp") - filePath = if (dir.isDirectory() && dir.exists() && dir.canWrite()) { - "/tmp/test_metrics.properties" - } else { - "./test_metrics.properties" - } - - val os = new FileOutputStream(new File(filePath)) - props.store(os, "for test") - os.close() + filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile() System.setProperty("spark.metrics.conf.file", filePath) } @@ -54,12 +36,5 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { val source = new spark.deploy.master.MasterInstrumentation(null) metricsSystem.registerSource(source) assert(sources.length === 2) - } - - after { - val file = new File(filePath) - if (file.exists()) { - file.delete() - } - } -} \ No newline at end of file + } +} -- cgit v1.2.3 From 5f8802c1fb106cc04c30b6aca0a6ce98fa5c0e15 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Tue, 2 Jul 2013 10:41:43 +0800 Subject: Register and init metricsSystem in SparkContext Conflicts: core/src/main/scala/spark/SparkContext.scala core/src/main/scala/spark/SparkEnv.scala --- core/src/main/scala/spark/SparkContext.scala | 10 ++++++++-- core/src/main/scala/spark/SparkEnv.scala | 4 ---- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index a6128a9f30..1255d0c72e 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -67,6 +67,9 @@ import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend import spark.storage.{StorageStatus, StorageUtils, RDDInfo, BlockManagerSource} import spark.util.{MetadataCleaner, TimeStampedHashMap} import ui.{SparkUI} +import spark.metrics._ + +import scala.util.DynamicVariable /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -272,9 +275,12 @@ class SparkContext( val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) + val metricsSystem = MetricsSystem.createMetricsSystem("driver") + def initDriverMetrics() = { - SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) - SparkEnv.get.metricsSystem.registerSource(blockManagerSource) + metricsSystem.registerSource(dagSchedulerSource) + metricsSystem.registerSource(blockManagerSource) + metricsSystem.start() } initDriverMetrics() diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index d34dafecc5..125dc55bd8 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -54,7 +54,6 @@ class SparkEnv ( val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, - val metricsSystem: MetricsSystem, // To be set only as part of initialization of SparkContext. // (executorId, defaultHostPort) => executorHostPort // If executorId is NOT found, return defaultHostPort @@ -186,8 +185,6 @@ object SparkEnv extends Logging { httpFileServer.initialize() System.setProperty("spark.fileserver.uri", httpFileServer.serverUri) - val metricsSystem = MetricsSystem.createMetricsSystem("driver") - metricsSystem.start() // Set the sparkFiles directory, used when downloading dependencies. In local mode, // this is a temporary directory; in distributed mode, this is the executor's current working @@ -218,7 +215,6 @@ object SparkEnv extends Logging { connectionManager, httpFileServer, sparkFilesDir, - metricsSystem, None) } } -- cgit v1.2.3 From 1daff54b2ed92d0bcee7030d7d3ab5c274f80d2f Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 2 Jul 2013 11:28:32 +0800 Subject: Change Executor MetricsSystem initialize code to SparkEnv --- core/src/main/scala/spark/SparkContext.scala | 6 ++---- core/src/main/scala/spark/SparkEnv.scala | 9 +++++++++ core/src/main/scala/spark/executor/Executor.scala | 6 +++++- .../main/scala/spark/executor/ExecutorInstrumentation.scala | 10 +++++----- .../src/main/scala/spark/executor/MesosExecutorBackend.scala | 10 +--------- .../scala/spark/executor/StandaloneExecutorBackend.scala | 12 ------------ 6 files changed, 22 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 1255d0c72e..f1d9d5e442 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -275,12 +275,10 @@ class SparkContext( val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) - val metricsSystem = MetricsSystem.createMetricsSystem("driver") def initDriverMetrics() = { - metricsSystem.registerSource(dagSchedulerSource) - metricsSystem.registerSource(blockManagerSource) - metricsSystem.start() + SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) + SparkEnv.get.metricsSystem.registerSource(blockManagerSource) } initDriverMetrics() diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 125dc55bd8..7b3dc69348 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -54,6 +54,7 @@ class SparkEnv ( val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, + val metricsSystem: metricsSystem, // To be set only as part of initialization of SparkContext. // (executorId, defaultHostPort) => executorHostPort // If executorId is NOT found, return defaultHostPort @@ -69,6 +70,7 @@ class SparkEnv ( broadcastManager.stop() blockManager.stop() blockManager.master.stop() + metricsSystem.stop() actorSystem.shutdown() // Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut // down, but let's call it anyway in case it gets fixed in a later release @@ -185,6 +187,12 @@ object SparkEnv extends Logging { httpFileServer.initialize() System.setProperty("spark.fileserver.uri", httpFileServer.serverUri) + val metricsSystem = if (isDriver) { + MetricsSystem.createMetricsSystem("driver") + } else { + MetricsSystem.createMetricsSystem("executor") + } + metricsSystem.start() // Set the sparkFiles directory, used when downloading dependencies. In local mode, // this is a temporary directory; in distributed mode, this is the executor's current working @@ -215,6 +223,7 @@ object SparkEnv extends Logging { connectionManager, httpFileServer, sparkFilesDir, + metricsSystem, None) } } diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 2e81151882..7179ed84a8 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -86,10 +86,14 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert } } ) + + val executorInstrumentation = new ExecutorInstrumentation(this) // Initialize Spark environment (using system properties read above) val env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, false, false) - SparkEnv.set(env) + SparkEnv.set(env) + env.metricsSystem.registerSource(executorInstrumentation) + private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size") // Start worker thread pool diff --git a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala index 80aadb66b0..ebbcbee742 100644 --- a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala +++ b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala @@ -4,32 +4,32 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source -class ExecutorInstrumentation(val executor: Option[Executor]) extends Source{ +class ExecutorInstrumentation(val executor: Executor) extends Source{ val metricRegistry = new MetricRegistry() val sourceName = "executor" // Gauge for executor thread pool's actively executing task counts metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), new Gauge[Int] { - override def getValue: Int = executor.map(_.threadPool.getActiveCount()).getOrElse(0) + override def getValue: Int = executor.threadPool.getActiveCount() }) // Gauge for executor thread pool's approximate total number of tasks that have been completed metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), new Gauge[Long] { - override def getValue: Long = executor.map(_.threadPool.getCompletedTaskCount()).getOrElse(0) + override def getValue: Long = executor.threadPool.getCompletedTaskCount() }) // Gauge for executor thread pool's current number of threads metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), new Gauge[Int] { - override def getValue: Int = executor.map(_.threadPool.getPoolSize()).getOrElse(0) + override def getValue: Int = executor.threadPool.getPoolSize() }) // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), new Gauge[Int] { - override def getValue: Int = executor.map(_.threadPool.getMaximumPoolSize()).getOrElse(0) + override def getValue: Int = executor.threadPool.getMaximumPoolSize() }) } \ No newline at end of file diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala index 8b6ab0c391..4961c42fad 100644 --- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala @@ -24,7 +24,6 @@ import spark.TaskState.TaskState import com.google.protobuf.ByteString import spark.{Utils, Logging} import spark.TaskState -import spark.metrics.MetricsSystem private[spark] class MesosExecutorBackend extends MesosExecutor @@ -33,9 +32,6 @@ private[spark] class MesosExecutorBackend var executor: Executor = null var driver: ExecutorDriver = null - - val executorInstrumentation = new ExecutorInstrumentation(Option(executor)) - MesosExecutorBackend.metricsSystem.start() override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { val mesosTaskId = TaskID.newBuilder().setValue(taskId.toString).build() @@ -83,17 +79,13 @@ private[spark] class MesosExecutorBackend override def frameworkMessage(d: ExecutorDriver, data: Array[Byte]) {} - override def shutdown(d: ExecutorDriver) { - MesosExecutorBackend.metricsSystem.stop() - } + override def shutdown(d: ExecutorDriver) {} } /** * Entry point for Mesos executor. */ private[spark] object MesosExecutorBackend { - private val metricsSystem = MetricsSystem.createMetricsSystem("executor") - def main(args: Array[String]) { MesosNativeLibrary.load() // Create a new Executor and start it running diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index 6ef74cd2ff..f4003da732 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -24,7 +24,6 @@ import spark.util.AkkaUtils import akka.actor.{ActorRef, Actor, Props, Terminated} import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import java.util.concurrent.{TimeUnit, ThreadPoolExecutor, SynchronousQueue} -import spark.metrics.MetricsSystem import spark.scheduler.cluster._ import spark.scheduler.cluster.RegisteredExecutor import spark.scheduler.cluster.LaunchTask @@ -46,8 +45,6 @@ private[spark] class StandaloneExecutorBackend( var executor: Executor = null var driver: ActorRef = null - - val executorInstrumentation = new ExecutorInstrumentation(Option(executor)) override def preStart() { logInfo("Connecting to driver: " + driverUrl) @@ -55,9 +52,6 @@ private[spark] class StandaloneExecutorBackend( driver ! RegisterExecutor(executorId, hostPort, cores) context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) context.watch(driver) // Doesn't work with remote actors, but useful for testing - - StandaloneExecutorBackend.metricsSystem.registerSource(executorInstrumentation) - StandaloneExecutorBackend.metricsSystem.start() } override def receive = { @@ -87,15 +81,9 @@ private[spark] class StandaloneExecutorBackend( override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { driver ! StatusUpdate(executorId, taskId, state, data) } - - override def postStop() { - StandaloneExecutorBackend.metricsSystem.stop() - } } private[spark] object StandaloneExecutorBackend { - private val metricsSystem = MetricsSystem.createMetricsSystem("executor") - def run(driverUrl: String, executorId: String, hostname: String, cores: Int) { SparkHadoopUtil.runAsUser(run0, Tuple4[Any, Any, Any, Any] (driverUrl, executorId, hostname, cores)) } -- cgit v1.2.3 From a79f6077f0b3bc0110f6e8e15de3068f296f3b81 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 2 Jul 2013 11:29:26 +0800 Subject: Add Maven metrics library dependency and code changes --- conf/metrics.properties | 4 ++-- core/pom.xml | 8 ++++++++ core/src/main/scala/spark/SparkEnv.scala | 3 ++- pom.xml | 8 ++++++++ 4 files changed, 20 insertions(+), 3 deletions(-) diff --git a/conf/metrics.properties b/conf/metrics.properties index 0bbb6b5229..2ea3aaf345 100644 --- a/conf/metrics.properties +++ b/conf/metrics.properties @@ -1,10 +1,10 @@ -# syntax: [prefix].[sink|source].[instance].[options] +# syntax: [instance].[sink|source].[name].[options] *.sink.console.period=10 *.sink.console.unit=second -master.source.jvm.class=spark.metrics.source.JvmSource +*.sink.console.class=spark.metrics.sink.ConsoleSink master.sink.console.period=10 diff --git a/core/pom.xml b/core/pom.xml index 6329b2fbd8..237d988712 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -108,6 +108,14 @@ log4j log4j + + com.codahale.metrics + metrics-core + + + com.codahale.metrics + metrics-jvm + org.apache.derby diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 7b3dc69348..204049d927 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -24,6 +24,7 @@ import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem} import akka.remote.RemoteActorRefProvider import spark.broadcast.BroadcastManager +import spark.metrics.MetricsSystem import spark.storage.BlockManager import spark.storage.BlockManagerMaster import spark.network.ConnectionManager @@ -54,7 +55,7 @@ class SparkEnv ( val connectionManager: ConnectionManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, - val metricsSystem: metricsSystem, + val metricsSystem: MetricsSystem, // To be set only as part of initialization of SparkContext. // (executorId, defaultHostPort) => executorHostPort // If executorId is NOT found, return defaultHostPort diff --git a/pom.xml b/pom.xml index eb7bd7e9df..3fbd93c7dd 100644 --- a/pom.xml +++ b/pom.xml @@ -268,6 +268,14 @@ org.scala-lang scalap ${scala.version} + com.codahale.metrics + metrics-core + 3.0.0 + + + com.codahale.metrics + metrics-jvm + 3.0.0 -- cgit v1.2.3 From 5730193e0c8639b654f489c5956e31451d81b7db Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 2 Jul 2013 13:48:09 +0800 Subject: Fix some typos --- core/src/main/scala/spark/SparkContext.scala | 2 -- core/src/main/scala/spark/SparkEnv.scala | 1 - core/src/main/scala/spark/metrics/sink/JmxSink.scala | 10 +++++----- core/src/main/scala/spark/metrics/source/JvmSource.scala | 6 ++---- core/src/main/scala/spark/metrics/source/Source.scala | 3 +-- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 3 ++- project/SparkBuild.scala | 2 +- 7 files changed, 11 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index f1d9d5e442..1e59a4d47d 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -69,8 +69,6 @@ import spark.util.{MetadataCleaner, TimeStampedHashMap} import ui.{SparkUI} import spark.metrics._ -import scala.util.DynamicVariable - /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 204049d927..4a1d341f5d 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -31,7 +31,6 @@ import spark.network.ConnectionManager import spark.serializer.{Serializer, SerializerManager} import spark.util.AkkaUtils import spark.api.python.PythonWorkerFactory -import spark.metrics._ /** diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/spark/metrics/sink/JmxSink.scala index f097a631c0..6a40885b78 100644 --- a/core/src/main/scala/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/spark/metrics/sink/JmxSink.scala @@ -1,18 +1,18 @@ package spark.metrics.sink -import java.util.Properties - import com.codahale.metrics.{JmxReporter, MetricRegistry} +import java.util.Properties + class JmxSink(val property: Properties, val registry: MetricRegistry) extends Sink { val reporter: JmxReporter = JmxReporter.forRegistry(registry).build() - + override def start() { reporter.start() } - + override def stop() { reporter.stop() } - + } diff --git a/core/src/main/scala/spark/metrics/source/JvmSource.scala b/core/src/main/scala/spark/metrics/source/JvmSource.scala index 8f0870c1a0..79f505079c 100644 --- a/core/src/main/scala/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/spark/metrics/source/JvmSource.scala @@ -3,15 +3,13 @@ package spark.metrics.source import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} -import java.util.{Map, HashMap => JHashMap} - class JvmSource extends Source { val sourceName = "jvm" val metricRegistry = new MetricRegistry() - + val gcMetricSet = new GarbageCollectorMetricSet val memGaugeSet = new MemoryUsageGaugeSet - + metricRegistry.registerAll(gcMetricSet) metricRegistry.registerAll(memGaugeSet) } diff --git a/core/src/main/scala/spark/metrics/source/Source.scala b/core/src/main/scala/spark/metrics/source/Source.scala index 17cbe2f85a..5607e2c40a 100644 --- a/core/src/main/scala/spark/metrics/source/Source.scala +++ b/core/src/main/scala/spark/metrics/source/Source.scala @@ -1,9 +1,8 @@ package spark.metrics.source -import com.codahale.metrics.MetricSet import com.codahale.metrics.MetricRegistry trait Source { def sourceName: String - def metricRegistry: MetricRegistry + def metricRegistry: MetricRegistry } diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 781e49bdec..9b45fc2938 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -30,7 +30,7 @@ import spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialRe import spark.scheduler.cluster.TaskInfo import spark.storage.{BlockManager, BlockManagerMaster} import spark.util.{MetadataCleaner, TimeStampedHashMap} -import spark.metrics.MetricsSystem + /** * A Scheduler subclass that implements stage-oriented scheduling. It computes a DAG of stages for * each job, keeps track of which RDDs and stage outputs are materialized, and computes a minimal @@ -126,6 +126,7 @@ class DAGScheduler( val resultStageToJob = new HashMap[Stage, ActiveJob] val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup) + // Start a thread to run the DAGScheduler event loop def start() { new Thread("DAGScheduler") { diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 825319d3de..d4d70afdd5 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -179,7 +179,7 @@ object SparkBuild extends Build { "net.liftweb" % "lift-json_2.9.2" % "2.5", "org.apache.mesos" % "mesos" % "0.9.0-incubating", "io.netty" % "netty-all" % "4.0.0.Beta2", - "org.apache.derby" % "derby" % "10.4.2.0" % "test" + "org.apache.derby" % "derby" % "10.4.2.0" % "test", "com.codahale.metrics" % "metrics-core" % "3.0.0", "com.codahale.metrics" % "metrics-jvm" % "3.0.0" ) ++ ( -- cgit v1.2.3 From ed1a3bc206c01974eedd0b1fb1deec183376b5c6 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Wed, 3 Jul 2013 20:11:10 +0800 Subject: continue to refactor code style and functions --- core/src/main/scala/spark/SparkContext.scala | 6 +- .../main/scala/spark/deploy/master/Master.scala | 6 +- .../deploy/master/MasterInstrumentation.scala | 19 ++--- .../deploy/worker/WorkerInstrumentation.scala | 35 ++++----- core/src/main/scala/spark/executor/Executor.scala | 8 +- .../spark/executor/ExecutorInstrumentation.scala | 35 ++++----- .../main/scala/spark/metrics/MetricsConfig.scala | 88 +++++++++++----------- .../main/scala/spark/metrics/MetricsSystem.scala | 40 +++++----- .../scala/spark/metrics/sink/ConsoleSink.scala | 30 ++++---- .../main/scala/spark/metrics/sink/CsvSink.scala | 40 +++++----- .../scala/spark/scheduler/DAGSchedulerSource.scala | 23 +++--- .../scala/spark/storage/BlockManagerSource.scala | 14 ++-- .../scala/spark/metrics/MetricsConfigSuite.scala | 32 ++++---- .../scala/spark/metrics/MetricsSystemSuite.scala | 14 ++-- 14 files changed, 189 insertions(+), 201 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 1e59a4d47d..77cb0ee0cd 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -274,9 +274,9 @@ class SparkContext( val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) - def initDriverMetrics() = { - SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) - SparkEnv.get.metricsSystem.registerSource(blockManagerSource) + def initDriverMetrics() { + SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) + SparkEnv.get.metricsSystem.registerSource(blockManagerSource) } initDriverMetrics() diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index cc0b2d4295..5f67366eb6 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -57,7 +57,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act val webUi = new MasterWebUI(self, webUiPort) Utils.checkHost(host, "Expected hostname") - + val masterInstrumentation = new MasterInstrumentation(this) val masterPublicAddress = { @@ -76,7 +76,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) - + Master.metricsSystem.registerSource(masterInstrumentation) Master.metricsSystem.start() } @@ -322,7 +322,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act removeWorker(worker) } } - + override def postStop() { Master.metricsSystem.stop() } diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala index 61a561c955..4c3708cc4c 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala @@ -5,24 +5,21 @@ import com.codahale.metrics.{Gauge,MetricRegistry} import spark.metrics.source.Source private[spark] class MasterInstrumentation(val master: Master) extends Source { - val metricRegistry = new MetricRegistry() + val metricRegistry = new MetricRegistry() val sourceName = "master" - metricRegistry.register(MetricRegistry.name("workers","number"), - new Gauge[Int] { + // Gauge for worker numbers in cluster + metricRegistry.register(MetricRegistry.name("workers","number"), new Gauge[Int] { override def getValue: Int = master.workers.size }) - + // Gauge for application numbers in cluster - metricRegistry.register(MetricRegistry.name("apps", "number"), - new Gauge[Int] { - override def getValue: Int = master.apps.size + metricRegistry.register(MetricRegistry.name("apps", "number"), new Gauge[Int] { + override def getValue: Int = master.apps.size }) // Gauge for waiting application numbers in cluster - metricRegistry.register(MetricRegistry.name("waiting_apps", "number"), - new Gauge[Int] { - override def getValue: Int = master.waitingApps.size + metricRegistry.register(MetricRegistry.name("waiting_apps", "number"), new Gauge[Int] { + override def getValue: Int = master.waitingApps.size }) - } diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala index 94c20a98c1..c76c0b4711 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala @@ -8,32 +8,27 @@ private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { val sourceName = "worker" val metricRegistry = new MetricRegistry() - metricRegistry.register(MetricRegistry.name("executor", "number"), - new Gauge[Int] { - override def getValue: Int = worker.executors.size + metricRegistry.register(MetricRegistry.name("executor", "number"), new Gauge[Int] { + override def getValue: Int = worker.executors.size }) - + // Gauge for cores used of this worker - metricRegistry.register(MetricRegistry.name("core_used", "number"), - new Gauge[Int] { - override def getValue: Int = worker.coresUsed + metricRegistry.register(MetricRegistry.name("core_used", "number"), new Gauge[Int] { + override def getValue: Int = worker.coresUsed }) - + // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name("mem_used", "MBytes"), - new Gauge[Int] { - override def getValue: Int = worker.memoryUsed + metricRegistry.register(MetricRegistry.name("mem_used", "MBytes"), new Gauge[Int] { + override def getValue: Int = worker.memoryUsed }) - + // Gauge for cores free of this worker - metricRegistry.register(MetricRegistry.name("core_free", "number"), - new Gauge[Int] { - override def getValue: Int = worker.coresFree + metricRegistry.register(MetricRegistry.name("core_free", "number"), new Gauge[Int] { + override def getValue: Int = worker.coresFree }) - + // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name("mem_free", "MBytes"), - new Gauge[Int] { - override def getValue: Int = worker.memoryFree - }) + metricRegistry.register(MetricRegistry.name("mem_free", "MBytes"), new Gauge[Int] { + override def getValue: Int = worker.memoryFree + }) } diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 7179ed84a8..4ea05dec1c 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -69,7 +69,7 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert override def uncaughtException(thread: Thread, exception: Throwable) { try { logError("Uncaught exception in thread " + thread, exception) - + // We may have been called from a shutdown hook. If so, we must not call System.exit(). // (If we do, we will deadlock.) if (!Utils.inShutdown()) { @@ -86,14 +86,14 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert } } ) - + val executorInstrumentation = new ExecutorInstrumentation(this) // Initialize Spark environment (using system properties read above) val env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, false, false) - SparkEnv.set(env) + SparkEnv.set(env) env.metricsSystem.registerSource(executorInstrumentation) - + private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size") // Start worker thread pool diff --git a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala index ebbcbee742..ad406f41b4 100644 --- a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala +++ b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala @@ -4,32 +4,27 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source -class ExecutorInstrumentation(val executor: Executor) extends Source{ +class ExecutorInstrumentation(val executor: Executor) extends Source { val metricRegistry = new MetricRegistry() val sourceName = "executor" - + // Gauge for executor thread pool's actively executing task counts - metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), - new Gauge[Int] { - override def getValue: Int = executor.threadPool.getActiveCount() + metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), new Gauge[Int] { + override def getValue: Int = executor.threadPool.getActiveCount() }) - + // Gauge for executor thread pool's approximate total number of tasks that have been completed - metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), - new Gauge[Long] { - override def getValue: Long = executor.threadPool.getCompletedTaskCount() + metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), new Gauge[Long] { + override def getValue: Long = executor.threadPool.getCompletedTaskCount() }) - + // Gauge for executor thread pool's current number of threads - metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), - new Gauge[Int] { - override def getValue: Int = executor.threadPool.getPoolSize() + metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), new Gauge[Int] { + override def getValue: Int = executor.threadPool.getPoolSize() }) - - // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool - metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), - new Gauge[Int] { - override def getValue: Int = executor.threadPool.getMaximumPoolSize() - }) -} \ No newline at end of file + // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool + metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), new Gauge[Int] { + override def getValue: Int = executor.threadPool.getMaximumPoolSize() + }) +} diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index 7405192058..b1f6a1e596 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -3,63 +3,63 @@ package spark.metrics import java.util.Properties import java.io.{File, FileInputStream} -import scala.collection.mutable +import scala.collection.mutable.HashMap import scala.util.matching.Regex -private [spark] class MetricsConfig(val configFile: String) { +private[spark] class MetricsConfig(val configFile: String) { val properties = new Properties() - // Add default properties in case there's no properties file - MetricsConfig.setDefaultProperties(properties) - - val confFile = new File(configFile) - if (confFile.exists()) { - var fis: FileInputStream = null - try { - fis = new FileInputStream(configFile) - properties.load(fis) - } finally { - fis.close() - } - } - - val propertyCategories = MetricsConfig.subProperties(properties, MetricsConfig.INSTANCE_REGEX) - if (propertyCategories.contains(MetricsConfig.DEFAULT_PREFIX)) { - import scala.collection.JavaConversions._ - val defaultProperty = propertyCategories(MetricsConfig.DEFAULT_PREFIX) - for ((inst, prop) <- propertyCategories; p <- defaultProperty - if inst != MetricsConfig.DEFAULT_PREFIX; if prop.getProperty(p._1) == null) { - prop.setProperty(p._1, p._2) - } - } - - def getInstance(inst: String) = { - propertyCategories.get(inst) match { - case Some(s) => s - case None => propertyCategories(MetricsConfig.DEFAULT_PREFIX) - } - } -} - -private[spark] object MetricsConfig { val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r - - def setDefaultProperties(prop: Properties) { + var propertyCategories: HashMap[String, Properties] = null + + private def setDefaultProperties(prop: Properties) { prop.setProperty("*.sink.jmx.enabled", "default") prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") } - - def subProperties(prop: Properties, regex: Regex) = { - val subProperties = new mutable.HashMap[String, Properties] - + + def initilize() { + //Add default properties in case there's no properties file + setDefaultProperties(properties) + + val confFile = new File(configFile) + if (confFile.exists()) { + var fis: FileInputStream = null + try { + fis = new FileInputStream(configFile) + properties.load(fis) + } finally { + fis.close() + } + } + + propertyCategories = subProperties(properties, INSTANCE_REGEX) + if (propertyCategories.contains(DEFAULT_PREFIX)) { + import scala.collection.JavaConversions._ + val defaultProperty = propertyCategories(DEFAULT_PREFIX) + for ((inst, prop) <- propertyCategories; p <- defaultProperty + if inst != DEFAULT_PREFIX; if prop.getProperty(p._1) == null) { + prop.setProperty(p._1, p._2) + } + } + } + + def subProperties(prop: Properties, regex: Regex): HashMap[String, Properties] = { + val subProperties = new HashMap[String, Properties] import scala.collection.JavaConversions._ - prop.foreach { kv => + prop.foreach { kv => if (regex.findPrefixOf(kv._1) != None) { val regex(prefix, suffix) = kv._1 subProperties.getOrElseUpdate(prefix, new Properties).setProperty(suffix, kv._2) } } - subProperties } -} \ No newline at end of file + + def getInstance(inst: String): Properties = { + propertyCategories.get(inst) match { + case Some(s) => s + case None => propertyCategories(DEFAULT_PREFIX) + } + } +} + diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index bf4487e0fc..54f6c6e4da 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -13,34 +13,37 @@ import spark.metrics.source.Source private[spark] class MetricsSystem private (val instance: String) extends Logging { initLogging() - + val confFile = System.getProperty("spark.metrics.conf.file", "unsupported") val metricsConfig = new MetricsConfig(confFile) - + val sinks = new mutable.ArrayBuffer[Sink] val sources = new mutable.ArrayBuffer[Source] val registry = new MetricRegistry() - + + val DEFAULT_SINKS = Map("jmx" -> "spark.metrics.sink.JmxSink") + + metricsConfig.initilize() registerSources() registerSinks() - + def start() { sinks.foreach(_.start) } - + def stop() { sinks.foreach(_.stop) } - + def registerSource(source: Source) { sources += source registry.register(source.sourceName, source.metricRegistry) } - + def registerSources() { val instConfig = metricsConfig.getInstance(instance) - val sourceConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) - + val sourceConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) + // Register all the sources related to instance sourceConfigs.foreach { kv => val classPath = kv._2.getProperty("class") @@ -52,14 +55,14 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin } } } - + def registerSinks() { val instConfig = metricsConfig.getInstance(instance) - val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) - + val sinkConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) + sinkConfigs.foreach { kv => - val classPath = if (MetricsSystem.DEFAULT_SINKS.contains(kv._1)) { - MetricsSystem.DEFAULT_SINKS(kv._1) + val classPath = if (DEFAULT_SINKS.contains(kv._1)) { + DEFAULT_SINKS(kv._1) } else { // For non-default sink, a property class should be set and create using reflection kv._2.getProperty("class") @@ -76,17 +79,14 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin } private[spark] object MetricsSystem { - val DEFAULT_SINKS = Map("jmx" -> "spark.metrics.sink.JmxSink") - val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r - val timeUnits = Map( - "millisecond" -> TimeUnit.MILLISECONDS, + "illisecond" -> TimeUnit.MILLISECONDS, "second" -> TimeUnit.SECONDS, "minute" -> TimeUnit.MINUTES, "hour" -> TimeUnit.HOURS, "day" -> TimeUnit.DAYS) - - def createMetricsSystem(instance: String) = new MetricsSystem(instance) + + def createMetricsSystem(instance: String): MetricsSystem = new MetricsSystem(instance) } diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index d7b7a9e501..c67c0ee912 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -8,34 +8,34 @@ import java.util.concurrent.TimeUnit import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { - val pollPeriod = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_PERIOD)) match { + + val CONSOLE_DEFAULT_PERIOD = "10" + val CONSOLE_DEFAULT_UNIT = "second" + + val CONSOLE_KEY_PERIOD = "period" + val CONSOLE_KEY_UNIT = "unit" + + val pollPeriod = Option(property.getProperty(CONSOLE_KEY_PERIOD)) match { case Some(s) => s.toInt - case None => ConsoleSink.CONSOLE_DEFAULT_PERIOD.toInt + case None => CONSOLE_DEFAULT_PERIOD.toInt } - - val pollUnit = Option(property.getProperty(ConsoleSink.CONSOLE_KEY_UNIT)) match { + + val pollUnit = Option(property.getProperty(CONSOLE_KEY_UNIT)) match { case Some(s) => MetricsSystem.timeUnits(s) - case None => MetricsSystem.timeUnits(ConsoleSink.CONSOLE_DEFAULT_UNIT) + case None => MetricsSystem.timeUnits(CONSOLE_DEFAULT_UNIT) } - + val reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) .build() override def start() { - reporter.start(pollPeriod, pollUnit) + reporter.start(pollPeriod, pollUnit) } - + override def stop() { reporter.stop() } } -object ConsoleSink { - val CONSOLE_DEFAULT_PERIOD = "10" - val CONSOLE_DEFAULT_UNIT = "second" - - val CONSOLE_KEY_PERIOD = "period" - val CONSOLE_KEY_UNIT = "unit" -} diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index e6c5bffd3c..a8ca819e87 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -9,21 +9,29 @@ import java.util.concurrent.TimeUnit import spark.metrics.MetricsSystem class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { - val pollPeriod = Option(property.getProperty(CsvSink.CSV_KEY_PERIOD)) match { + val CSV_KEY_PERIOD = "period" + val CSV_KEY_UNIT = "unit" + val CSV_KEY_DIR = "directory" + + val CSV_DEFAULT_PERIOD = "10" + val CSV_DEFAULT_UNIT = "second" + val CSV_DEFAULT_DIR = "/tmp/" + + val pollPeriod = Option(property.getProperty(CSV_KEY_PERIOD)) match { case Some(s) => s.toInt - case None => CsvSink.CSV_DEFAULT_PERIOD.toInt + case None => CSV_DEFAULT_PERIOD.toInt } - - val pollUnit = Option(property.getProperty(CsvSink.CSV_KEY_UNIT)) match { + + val pollUnit = Option(property.getProperty(CSV_KEY_UNIT)) match { case Some(s) => MetricsSystem.timeUnits(s) - case None => MetricsSystem.timeUnits(CsvSink.CSV_DEFAULT_UNIT) + case None => MetricsSystem.timeUnits(CSV_DEFAULT_UNIT) } - - val pollDir = Option(property.getProperty(CsvSink.CSV_KEY_DIR)) match { + + val pollDir = Option(property.getProperty(CSV_KEY_DIR)) match { case Some(s) => s - case None => CsvSink.CSV_DEFAULT_DIR + case None => CSV_DEFAULT_DIR } - + val reporter: CsvReporter = CsvReporter.forRegistry(registry) .formatFor(Locale.US) .convertDurationsTo(TimeUnit.MILLISECONDS) @@ -31,21 +39,11 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si .build(new File(pollDir)) override def start() { - reporter.start(pollPeriod, pollUnit) + reporter.start(pollPeriod, pollUnit) } - + override def stop() { reporter.stop() } } -object CsvSink { - val CSV_KEY_PERIOD = "period" - val CSV_KEY_UNIT = "unit" - val CSV_KEY_DIR = "directory" - - val CSV_DEFAULT_PERIOD = "10" - val CSV_DEFAULT_UNIT = "second" - val CSV_DEFAULT_DIR = "/tmp/" -} - diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala index 57aa74512c..38158b8a2b 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala @@ -5,27 +5,26 @@ import com.codahale.metrics.{Gauge,MetricRegistry} import spark.metrics.source.Source private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends Source { - val metricRegistry = new MetricRegistry() + val metricRegistry = new MetricRegistry() val sourceName = "DAGScheduler" - - metricRegistry.register(MetricRegistry.name("stage","failedStage"), new Gauge[Int] { - override def getValue: Int = dagScheduler.failed.size + metricRegistry.register(MetricRegistry.name("stage", "failedStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.failed.size }) - metricRegistry.register(MetricRegistry.name("stage","runningStage"), new Gauge[Int] { - override def getValue: Int = dagScheduler.running.size + metricRegistry.register(MetricRegistry.name("stage", "runningStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.running.size }) - metricRegistry.register(MetricRegistry.name("stage","waitingStage"), new Gauge[Int] { - override def getValue: Int = dagScheduler.waiting.size + metricRegistry.register(MetricRegistry.name("stage", "waitingStage"), new Gauge[Int] { + override def getValue: Int = dagScheduler.waiting.size }) - metricRegistry.register(MetricRegistry.name("job","allJobs"), new Gauge[Int] { - override def getValue: Int = dagScheduler.nextRunId.get() + metricRegistry.register(MetricRegistry.name("job", "allJobs"), new Gauge[Int] { + override def getValue: Int = dagScheduler.nextRunId.get() }) - metricRegistry.register(MetricRegistry.name("job","ActiveJobs"), new Gauge[Int] { - override def getValue: Int = dagScheduler.activeJobs.size + metricRegistry.register(MetricRegistry.name("job", "ActiveJobs"), new Gauge[Int] { + override def getValue: Int = dagScheduler.activeJobs.size }) } diff --git a/core/src/main/scala/spark/storage/BlockManagerSource.scala b/core/src/main/scala/spark/storage/BlockManagerSource.scala index c0ce9259c8..f964827102 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/spark/storage/BlockManagerSource.scala @@ -6,29 +6,29 @@ import spark.metrics.source.Source import spark.storage._ private[spark] class BlockManagerSource(val blockManager: BlockManager) extends Source { - val metricRegistry = new MetricRegistry() + val metricRegistry = new MetricRegistry() val sourceName = "BlockManager" - metricRegistry.register(MetricRegistry.name("memory","maxMem"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "maxMem"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) + val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) maxMem } }) - metricRegistry.register(MetricRegistry.name("memory","remainingMem"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "remainingMem"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) remainingMem } }) - metricRegistry.register(MetricRegistry.name("disk","diskSpaceUsed"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).reduceOption(_+_).getOrElse(0L) + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).reduceOption(_+_).getOrElse(0L) diskSpaceUsed } }) diff --git a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala index f4c83cb644..bb1be4f4fc 100644 --- a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala @@ -9,60 +9,64 @@ import spark.metrics._ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ - + before { filePath = getClass.getClassLoader.getResource("test_metrics_config.properties").getFile() } test("MetricsConfig with default properties") { val conf = new MetricsConfig("dummy-file") + conf.initilize() + assert(conf.properties.size() === 2) assert(conf.properties.getProperty("*.sink.jmx.enabled") === "default") assert(conf.properties.getProperty("*.source.jvm.class") === "spark.metrics.source.JvmSource") assert(conf.properties.getProperty("test-for-dummy") === null) - + val property = conf.getInstance("random") assert(property.size() === 2) assert(property.getProperty("sink.jmx.enabled") === "default") assert(property.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") } - + test("MetricsConfig with properties set") { val conf = new MetricsConfig(filePath) - + conf.initilize() + val masterProp = conf.getInstance("master") assert(masterProp.size() === 4) assert(masterProp.getProperty("sink.console.period") === "20") assert(masterProp.getProperty("sink.console.unit") === "minute") assert(masterProp.getProperty("sink.jmx.enabled") === "default") assert(masterProp.getProperty("source.jvm.class") == "spark.metrics.source.JvmSource") - + val workerProp = conf.getInstance("worker") assert(workerProp.size() === 4) assert(workerProp.getProperty("sink.console.period") === "10") assert(workerProp.getProperty("sink.console.unit") === "second") } - + test("MetricsConfig with subProperties") { val conf = new MetricsConfig(filePath) - + conf.initilize() + val propCategories = conf.propertyCategories assert(propCategories.size === 2) - + val masterProp = conf.getInstance("master") - val sourceProps = MetricsConfig.subProperties(masterProp, MetricsSystem.SOURCE_REGEX) + val sourceProps = conf.subProperties(masterProp, MetricsSystem.SOURCE_REGEX) assert(sourceProps.size === 1) assert(sourceProps("jvm").getProperty("class") === "spark.metrics.source.JvmSource") - - val sinkProps = MetricsConfig.subProperties(masterProp, MetricsSystem.SINK_REGEX) + + val sinkProps = conf.subProperties(masterProp, MetricsSystem.SINK_REGEX) assert(sinkProps.size === 2) assert(sinkProps.contains("console")) assert(sinkProps.contains("jmx")) - + val consoleProps = sinkProps("console") assert(consoleProps.size() === 2) - + val jmxProps = sinkProps("jmx") assert(jmxProps.size() === 1) - } + } } diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala index 967be6ec47..f29bb9db67 100644 --- a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -9,32 +9,32 @@ import spark.metrics._ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ - + before { filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile() System.setProperty("spark.metrics.conf.file", filePath) } - + test("MetricsSystem with default config") { val metricsSystem = MetricsSystem.createMetricsSystem("default") val sources = metricsSystem.sources val sinks = metricsSystem.sinks - + assert(sources.length === 1) assert(sinks.length === 1) assert(sources(0).sourceName === "jvm") } - + test("MetricsSystem with sources add") { val metricsSystem = MetricsSystem.createMetricsSystem("test") val sources = metricsSystem.sources val sinks = metricsSystem.sinks - + assert(sources.length === 1) assert(sinks.length === 2) - + val source = new spark.deploy.master.MasterInstrumentation(null) metricsSystem.registerSource(source) assert(sources.length === 2) - } + } } -- cgit v1.2.3 From 5b4a2f2094258920613f50c0d337e1e92c08d012 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Wed, 3 Jul 2013 20:31:53 +0800 Subject: Add metrics config template file --- conf/metrics.properties | 15 --------------- conf/metrics.properties.template | 14 ++++++++++++++ 2 files changed, 14 insertions(+), 15 deletions(-) delete mode 100644 conf/metrics.properties create mode 100644 conf/metrics.properties.template diff --git a/conf/metrics.properties b/conf/metrics.properties deleted file mode 100644 index 2ea3aaf345..0000000000 --- a/conf/metrics.properties +++ /dev/null @@ -1,15 +0,0 @@ -# syntax: [instance].[sink|source].[name].[options] - -*.sink.console.period=10 - -*.sink.console.unit=second - -*.sink.console.class=spark.metrics.sink.ConsoleSink - -master.sink.console.period=10 - -master.sink.console.unit=second - -worker.sink.console.period=20 - -worker.sink.console.unit=second diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template new file mode 100644 index 0000000000..b4204e1deb --- /dev/null +++ b/conf/metrics.properties.template @@ -0,0 +1,14 @@ +# syntax: [instance].[sink|source].[name].[options] + +#*.sink.console.period=10 + +#*.sink.console.unit=second + +#master.source.jvm.class=spark.metrics.source.JvmSource + +#worker.source.jvm.class=spark.metrics.source.JvmSource + +#driver.source.jvm.class=spark.metrics.source.JvmSource + +#executor.source.jvm.class=spark.metrics.source.JvmSource + -- cgit v1.2.3 From 05637de8423da85c5934cdfa8f07254133a58474 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Wed, 3 Jul 2013 21:16:39 +0800 Subject: Change class xxxInstrumentation to class xxxSource --- .../main/scala/spark/deploy/master/Master.scala | 4 +-- .../deploy/master/MasterInstrumentation.scala | 25 ---------------- .../scala/spark/deploy/master/MasterSource.scala | 25 ++++++++++++++++ .../main/scala/spark/deploy/worker/Worker.scala | 4 +-- .../deploy/worker/WorkerInstrumentation.scala | 34 ---------------------- .../scala/spark/deploy/worker/WorkerSource.scala | 34 ++++++++++++++++++++++ core/src/main/scala/spark/executor/Executor.scala | 4 +-- .../spark/executor/ExecutorInstrumentation.scala | 30 ------------------- .../main/scala/spark/executor/ExecutorSource.scala | 30 +++++++++++++++++++ .../scala/spark/metrics/MetricsSystemSuite.scala | 2 +- 10 files changed, 96 insertions(+), 96 deletions(-) delete mode 100644 core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala create mode 100644 core/src/main/scala/spark/deploy/master/MasterSource.scala delete mode 100644 core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala create mode 100644 core/src/main/scala/spark/deploy/worker/WorkerSource.scala delete mode 100644 core/src/main/scala/spark/executor/ExecutorInstrumentation.scala create mode 100644 core/src/main/scala/spark/executor/ExecutorSource.scala diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 5f67366eb6..1d592206c0 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -58,7 +58,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act Utils.checkHost(host, "Expected hostname") - val masterInstrumentation = new MasterInstrumentation(this) + val masterSource = new MasterSource(this) val masterPublicAddress = { val envVar = System.getenv("SPARK_PUBLIC_DNS") @@ -77,7 +77,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) - Master.metricsSystem.registerSource(masterInstrumentation) + Master.metricsSystem.registerSource(masterSource) Master.metricsSystem.start() } diff --git a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala b/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala deleted file mode 100644 index 4c3708cc4c..0000000000 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ /dev/null @@ -1,25 +0,0 @@ -package spark.deploy.master - -import com.codahale.metrics.{Gauge,MetricRegistry} - -import spark.metrics.source.Source - -private[spark] class MasterInstrumentation(val master: Master) extends Source { - val metricRegistry = new MetricRegistry() - val sourceName = "master" - - // Gauge for worker numbers in cluster - metricRegistry.register(MetricRegistry.name("workers","number"), new Gauge[Int] { - override def getValue: Int = master.workers.size - }) - - // Gauge for application numbers in cluster - metricRegistry.register(MetricRegistry.name("apps", "number"), new Gauge[Int] { - override def getValue: Int = master.apps.size - }) - - // Gauge for waiting application numbers in cluster - metricRegistry.register(MetricRegistry.name("waiting_apps", "number"), new Gauge[Int] { - override def getValue: Int = master.waitingApps.size - }) -} diff --git a/core/src/main/scala/spark/deploy/master/MasterSource.scala b/core/src/main/scala/spark/deploy/master/MasterSource.scala new file mode 100644 index 0000000000..f94e5b2c34 --- /dev/null +++ b/core/src/main/scala/spark/deploy/master/MasterSource.scala @@ -0,0 +1,25 @@ +package spark.deploy.master + +import com.codahale.metrics.{Gauge,MetricRegistry} + +import spark.metrics.source.Source + +private[spark] class MasterSource(val master: Master) extends Source { + val metricRegistry = new MetricRegistry() + val sourceName = "master" + + // Gauge for worker numbers in cluster + metricRegistry.register(MetricRegistry.name("workers","number"), new Gauge[Int] { + override def getValue: Int = master.workers.size + }) + + // Gauge for application numbers in cluster + metricRegistry.register(MetricRegistry.name("apps", "number"), new Gauge[Int] { + override def getValue: Int = master.apps.size + }) + + // Gauge for waiting application numbers in cluster + metricRegistry.register(MetricRegistry.name("waiting_apps", "number"), new Gauge[Int] { + override def getValue: Int = master.waitingApps.size + }) +} diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index eaa1c1806f..5c0f77fd75 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -68,7 +68,7 @@ private[spark] class Worker( var coresUsed = 0 var memoryUsed = 0 - val workerInstrumentation = new WorkerInstrumentation(this) + val workerSource = new WorkerSource(this) def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed @@ -102,7 +102,7 @@ private[spark] class Worker( connectToMaster() startWebUi() - Worker.metricsSystem.registerSource(workerInstrumentation) + Worker.metricsSystem.registerSource(workerSource) Worker.metricsSystem.start() } diff --git a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala b/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala deleted file mode 100644 index c76c0b4711..0000000000 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ /dev/null @@ -1,34 +0,0 @@ -package spark.deploy.worker - -import com.codahale.metrics.{Gauge, MetricRegistry} - -import spark.metrics.source.Source - -private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { - val sourceName = "worker" - val metricRegistry = new MetricRegistry() - - metricRegistry.register(MetricRegistry.name("executor", "number"), new Gauge[Int] { - override def getValue: Int = worker.executors.size - }) - - // Gauge for cores used of this worker - metricRegistry.register(MetricRegistry.name("core_used", "number"), new Gauge[Int] { - override def getValue: Int = worker.coresUsed - }) - - // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name("mem_used", "MBytes"), new Gauge[Int] { - override def getValue: Int = worker.memoryUsed - }) - - // Gauge for cores free of this worker - metricRegistry.register(MetricRegistry.name("core_free", "number"), new Gauge[Int] { - override def getValue: Int = worker.coresFree - }) - - // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name("mem_free", "MBytes"), new Gauge[Int] { - override def getValue: Int = worker.memoryFree - }) -} diff --git a/core/src/main/scala/spark/deploy/worker/WorkerSource.scala b/core/src/main/scala/spark/deploy/worker/WorkerSource.scala new file mode 100644 index 0000000000..539eac71bd --- /dev/null +++ b/core/src/main/scala/spark/deploy/worker/WorkerSource.scala @@ -0,0 +1,34 @@ +package spark.deploy.worker + +import com.codahale.metrics.{Gauge, MetricRegistry} + +import spark.metrics.source.Source + +private[spark] class WorkerSource(val worker: Worker) extends Source { + val sourceName = "worker" + val metricRegistry = new MetricRegistry() + + metricRegistry.register(MetricRegistry.name("executor", "number"), new Gauge[Int] { + override def getValue: Int = worker.executors.size + }) + + // Gauge for cores used of this worker + metricRegistry.register(MetricRegistry.name("core_used", "number"), new Gauge[Int] { + override def getValue: Int = worker.coresUsed + }) + + // Gauge for memory used of this worker + metricRegistry.register(MetricRegistry.name("mem_used", "MBytes"), new Gauge[Int] { + override def getValue: Int = worker.memoryUsed + }) + + // Gauge for cores free of this worker + metricRegistry.register(MetricRegistry.name("core_free", "number"), new Gauge[Int] { + override def getValue: Int = worker.coresFree + }) + + // Gauge for memory used of this worker + metricRegistry.register(MetricRegistry.name("mem_free", "MBytes"), new Gauge[Int] { + override def getValue: Int = worker.memoryFree + }) +} diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 4ea05dec1c..8a74a8d853 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -87,12 +87,12 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert } ) - val executorInstrumentation = new ExecutorInstrumentation(this) + val executorSource = new ExecutorSource(this) // Initialize Spark environment (using system properties read above) val env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, false, false) SparkEnv.set(env) - env.metricsSystem.registerSource(executorInstrumentation) + env.metricsSystem.registerSource(executorSource) private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size") diff --git a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala b/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala deleted file mode 100644 index ad406f41b4..0000000000 --- a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala +++ /dev/null @@ -1,30 +0,0 @@ -package spark.executor - -import com.codahale.metrics.{Gauge, MetricRegistry} - -import spark.metrics.source.Source - -class ExecutorInstrumentation(val executor: Executor) extends Source { - val metricRegistry = new MetricRegistry() - val sourceName = "executor" - - // Gauge for executor thread pool's actively executing task counts - metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), new Gauge[Int] { - override def getValue: Int = executor.threadPool.getActiveCount() - }) - - // Gauge for executor thread pool's approximate total number of tasks that have been completed - metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), new Gauge[Long] { - override def getValue: Long = executor.threadPool.getCompletedTaskCount() - }) - - // Gauge for executor thread pool's current number of threads - metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), new Gauge[Int] { - override def getValue: Int = executor.threadPool.getPoolSize() - }) - - // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool - metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), new Gauge[Int] { - override def getValue: Int = executor.threadPool.getMaximumPoolSize() - }) -} diff --git a/core/src/main/scala/spark/executor/ExecutorSource.scala b/core/src/main/scala/spark/executor/ExecutorSource.scala new file mode 100644 index 0000000000..d8b531cb58 --- /dev/null +++ b/core/src/main/scala/spark/executor/ExecutorSource.scala @@ -0,0 +1,30 @@ +package spark.executor + +import com.codahale.metrics.{Gauge, MetricRegistry} + +import spark.metrics.source.Source + +class ExecutorSource(val executor: Executor) extends Source { + val metricRegistry = new MetricRegistry() + val sourceName = "executor" + + // Gauge for executor thread pool's actively executing task counts + metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), new Gauge[Int] { + override def getValue: Int = executor.threadPool.getActiveCount() + }) + + // Gauge for executor thread pool's approximate total number of tasks that have been completed + metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), new Gauge[Long] { + override def getValue: Long = executor.threadPool.getCompletedTaskCount() + }) + + // Gauge for executor thread pool's current number of threads + metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), new Gauge[Int] { + override def getValue: Int = executor.threadPool.getPoolSize() + }) + + // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool + metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), new Gauge[Int] { + override def getValue: Int = executor.threadPool.getMaximumPoolSize() + }) +} diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala index f29bb9db67..462c28e894 100644 --- a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -33,7 +33,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { assert(sources.length === 1) assert(sinks.length === 2) - val source = new spark.deploy.master.MasterInstrumentation(null) + val source = new spark.deploy.master.MasterSource(null) metricsSystem.registerSource(source) assert(sources.length === 2) } -- cgit v1.2.3 From 8d1ef7f2dfc453137b8dbbb72a7f1ca93b57bb15 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 4 Jul 2013 10:11:30 +0800 Subject: Code style changes --- conf/metrics.properties.template | 2 ++ core/src/main/scala/spark/deploy/master/MasterSource.scala | 4 ++-- core/src/main/scala/spark/metrics/MetricsConfig.scala | 8 ++++---- core/src/main/scala/spark/metrics/sink/ConsoleSink.scala | 1 - 4 files changed, 8 insertions(+), 7 deletions(-) diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template index b4204e1deb..c7e24aa36c 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -1,5 +1,7 @@ # syntax: [instance].[sink|source].[name].[options] +#*.sink.console.class=spark.metrics.sink.ConsoleSink + #*.sink.console.period=10 #*.sink.console.unit=second diff --git a/core/src/main/scala/spark/deploy/master/MasterSource.scala b/core/src/main/scala/spark/deploy/master/MasterSource.scala index f94e5b2c34..65c22320d6 100644 --- a/core/src/main/scala/spark/deploy/master/MasterSource.scala +++ b/core/src/main/scala/spark/deploy/master/MasterSource.scala @@ -1,6 +1,6 @@ package spark.deploy.master -import com.codahale.metrics.{Gauge,MetricRegistry} +import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source @@ -10,7 +10,7 @@ private[spark] class MasterSource(val master: Master) extends Source { // Gauge for worker numbers in cluster metricRegistry.register(MetricRegistry.name("workers","number"), new Gauge[Int] { - override def getValue: Int = master.workers.size + override def getValue: Int = master.workers.size }) // Gauge for application numbers in cluster diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index b1f6a1e596..5066b7ac22 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -3,14 +3,14 @@ package spark.metrics import java.util.Properties import java.io.{File, FileInputStream} -import scala.collection.mutable.HashMap +import scala.collection.mutable import scala.util.matching.Regex private[spark] class MetricsConfig(val configFile: String) { val properties = new Properties() val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r - var propertyCategories: HashMap[String, Properties] = null + var propertyCategories: mutable.HashMap[String, Properties] = null private def setDefaultProperties(prop: Properties) { prop.setProperty("*.sink.jmx.enabled", "default") @@ -43,8 +43,8 @@ private[spark] class MetricsConfig(val configFile: String) { } } - def subProperties(prop: Properties, regex: Regex): HashMap[String, Properties] = { - val subProperties = new HashMap[String, Properties] + def subProperties(prop: Properties, regex: Regex): mutable.HashMap[String, Properties] = { + val subProperties = new mutable.HashMap[String, Properties] import scala.collection.JavaConversions._ prop.foreach { kv => if (regex.findPrefixOf(kv._1) != None) { diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index c67c0ee912..437f24a575 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -8,7 +8,6 @@ import java.util.concurrent.TimeUnit import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { - val CONSOLE_DEFAULT_PERIOD = "10" val CONSOLE_DEFAULT_UNIT = "second" -- cgit v1.2.3 From 31ec72b243459e0d1d9c89033328e1d79f3aa76f Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 16 Jul 2013 10:57:19 +0800 Subject: Code refactor according to comments --- conf/metrics.properties.template | 73 ++++++++++++++++++- .../main/scala/spark/deploy/master/Master.scala | 11 ++- .../scala/spark/deploy/master/MasterSource.scala | 2 +- .../main/scala/spark/deploy/worker/Worker.scala | 11 ++- .../scala/spark/deploy/worker/WorkerSource.scala | 12 ++-- .../main/scala/spark/executor/ExecutorSource.scala | 8 +-- .../main/scala/spark/metrics/MetricsConfig.scala | 50 ++++++++----- .../main/scala/spark/metrics/MetricsSystem.scala | 83 ++++++++++++++++------ .../scala/spark/metrics/sink/ConsoleSink.scala | 12 ++-- .../main/scala/spark/metrics/sink/CsvSink.scala | 12 ++-- .../scala/spark/scheduler/DAGSchedulerSource.scala | 10 +-- .../scala/spark/storage/BlockManagerSource.scala | 31 +++++--- .../test/resources/test_metrics_config.properties | 6 +- .../test/resources/test_metrics_system.properties | 4 +- .../scala/spark/metrics/MetricsConfigSuite.scala | 38 ++++------ .../scala/spark/metrics/MetricsSystemSuite.scala | 13 ++-- 16 files changed, 250 insertions(+), 126 deletions(-) diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template index c7e24aa36c..0486ca4c79 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -1,11 +1,82 @@ # syntax: [instance].[sink|source].[name].[options] +# "instance" specify "who" (the role) use metrics system. In spark there are +# several roles like master, worker, executor, driver, these roles will +# create metrics system for monitoring. So instance represents these roles. +# Currently in Spark, several instances have already implemented: master, +# worker, executor, driver. +# +# [instance] field can be "master", "worker", "executor", "driver", which means +# only the specified instance has this property. +# a wild card "*" can be used to represent instance name, which means all the +# instances will have this property. +# +# "source" specify "where" (source) to collect metrics data. In metrics system, +# there exists two kinds of source: +# 1. Spark internal source, like MasterSource, WorkerSource, etc, which will +# collect Spark component's internal state, these sources are related to +# instance and will be added after specific metrics system is created. +# 2. Common source, like JvmSource, which will collect low level state, is +# configured by configuration and loaded through reflection. +# +# "sink" specify "where" (destination) to output metrics data to. Several sinks +# can be coexisted and flush metrics to all these sinks. +# +# [sink|source] field specify this property is source related or sink, this +# field can only be source or sink. +# +# [name] field specify the name of source or sink, this is custom defined. +# +# [options] field is the specific property of this source or sink, this source +# or sink is responsible for parsing this property. +# +# Notes: +# 1. Sinks should be added through configuration, like console sink, class +# full name should be specified by class property. +# 2. Some sinks can specify polling period, like console sink, which is 10 seconds, +# it should be attention minimal polling period is 1 seconds, any period +# below than 1s is illegal. +# 3. Wild card property can be overlapped by specific instance property, for +# example, *.sink.console.period can be overlapped by master.sink.console.period. +# 4. A metrics specific configuration +# "spark.metrics.conf=${SPARK_HOME}/conf/metrics.properties" should be +# added to Java property using -Dspark.metrics.conf=xxx if you want to +# customize metrics system, or you can put it in ${SPARK_HOME}/conf, +# metrics system will search and load it automatically. + +# Enable JmxSink for all instances by class name +#*.sink.jmx.class=spark.metrics.sink.JmxSink + +# Enable ConsoleSink for all instances by class name #*.sink.console.class=spark.metrics.sink.ConsoleSink +# Polling period for ConsoleSink #*.sink.console.period=10 -#*.sink.console.unit=second +#*.sink.console.unit=seconds + +# Master instance overlap polling period +#master.sink.console.period=15 + +#master.sink.console.unit=seconds + +# Enable CsvSink for all instances +#*.sink.csv.class=spark.metrics.sink.CsvSink + +# Polling period for CsvSink +#*.sink.csv.period=1 + +#*.sink.csv.unit=minutes + +# Polling directory for CsvSink +#*.sink.csv.directory=/tmp/ + +# Worker instance overlap polling period +#worker.sink.csv.period=10 + +#worker.sink.csv.unit=minutes +# Enable jvm source for instance master, worker, driver and executor #master.source.jvm.class=spark.metrics.source.JvmSource #worker.source.jvm.class=spark.metrics.source.JvmSource diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 1d592206c0..9692af5295 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -58,6 +58,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act Utils.checkHost(host, "Expected hostname") + val metricsSystem = MetricsSystem.createMetricsSystem("master") val masterSource = new MasterSource(this) val masterPublicAddress = { @@ -77,12 +78,13 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act webUi.start() context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers()) - Master.metricsSystem.registerSource(masterSource) - Master.metricsSystem.start() + metricsSystem.registerSource(masterSource) + metricsSystem.start() } override def postStop() { webUi.stop() + metricsSystem.stop() } override def receive = { @@ -322,17 +324,12 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act removeWorker(worker) } } - - override def postStop() { - Master.metricsSystem.stop() - } } private[spark] object Master { private val systemName = "sparkMaster" private val actorName = "Master" private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r - private val metricsSystem = MetricsSystem.createMetricsSystem("master") def main(argStrings: Array[String]) { val args = new MasterArguments(argStrings) diff --git a/core/src/main/scala/spark/deploy/master/MasterSource.scala b/core/src/main/scala/spark/deploy/master/MasterSource.scala index 65c22320d6..b8cfa6a773 100644 --- a/core/src/main/scala/spark/deploy/master/MasterSource.scala +++ b/core/src/main/scala/spark/deploy/master/MasterSource.scala @@ -19,7 +19,7 @@ private[spark] class MasterSource(val master: Master) extends Source { }) // Gauge for waiting application numbers in cluster - metricRegistry.register(MetricRegistry.name("waiting_apps", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("waitingApps", "number"), new Gauge[Int] { override def getValue: Int = master.waitingApps.size }) } diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 5c0f77fd75..8fa0d12b82 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -68,6 +68,7 @@ private[spark] class Worker( var coresUsed = 0 var memoryUsed = 0 + val metricsSystem = MetricsSystem.createMetricsSystem("worker") val workerSource = new WorkerSource(this) def coresFree: Int = cores - coresUsed @@ -100,10 +101,9 @@ private[spark] class Worker( webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) webUi.start() connectToMaster() - startWebUi() - Worker.metricsSystem.registerSource(workerSource) - Worker.metricsSystem.start() + metricsSystem.registerSource(workerSource) + metricsSystem.start() } def connectToMaster() { @@ -185,14 +185,11 @@ private[spark] class Worker( override def postStop() { executors.values.foreach(_.kill()) webUi.stop() - - Worker.metricsSystem.stop() + metricsSystem.stop() } } private[spark] object Worker { - private val metricsSystem = MetricsSystem.createMetricsSystem("worker") - def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, diff --git a/core/src/main/scala/spark/deploy/worker/WorkerSource.scala b/core/src/main/scala/spark/deploy/worker/WorkerSource.scala index 539eac71bd..39cb8e5690 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerSource.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerSource.scala @@ -8,27 +8,27 @@ private[spark] class WorkerSource(val worker: Worker) extends Source { val sourceName = "worker" val metricRegistry = new MetricRegistry() - metricRegistry.register(MetricRegistry.name("executor", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("executors", "number"), new Gauge[Int] { override def getValue: Int = worker.executors.size }) // Gauge for cores used of this worker - metricRegistry.register(MetricRegistry.name("core_used", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("coresUsed", "number"), new Gauge[Int] { override def getValue: Int = worker.coresUsed }) // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name("mem_used", "MBytes"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("memUsed", "MBytes"), new Gauge[Int] { override def getValue: Int = worker.memoryUsed }) // Gauge for cores free of this worker - metricRegistry.register(MetricRegistry.name("core_free", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("coresFree", "number"), new Gauge[Int] { override def getValue: Int = worker.coresFree }) - // Gauge for memory used of this worker - metricRegistry.register(MetricRegistry.name("mem_free", "MBytes"), new Gauge[Int] { + // Gauge for memory free of this worker + metricRegistry.register(MetricRegistry.name("memFree", "MBytes"), new Gauge[Int] { override def getValue: Int = worker.memoryFree }) } diff --git a/core/src/main/scala/spark/executor/ExecutorSource.scala b/core/src/main/scala/spark/executor/ExecutorSource.scala index d8b531cb58..94116edfcf 100644 --- a/core/src/main/scala/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/spark/executor/ExecutorSource.scala @@ -9,22 +9,22 @@ class ExecutorSource(val executor: Executor) extends Source { val sourceName = "executor" // Gauge for executor thread pool's actively executing task counts - metricRegistry.register(MetricRegistry.name("threadpool", "active_task", "number"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("threadpool", "activeTask", "count"), new Gauge[Int] { override def getValue: Int = executor.threadPool.getActiveCount() }) // Gauge for executor thread pool's approximate total number of tasks that have been completed - metricRegistry.register(MetricRegistry.name("threadpool", "complete_task", "count"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("threadpool", "completeTask", "count"), new Gauge[Long] { override def getValue: Long = executor.threadPool.getCompletedTaskCount() }) // Gauge for executor thread pool's current number of threads - metricRegistry.register(MetricRegistry.name("threadpool", "current_pool", "size"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("threadpool", "currentPool", "size"), new Gauge[Int] { override def getValue: Int = executor.threadPool.getPoolSize() }) // Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool - metricRegistry.register(MetricRegistry.name("threadpool", "max_pool", "size"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("threadpool", "maxPool", "size"), new Gauge[Int] { override def getValue: Int = executor.threadPool.getMaximumPoolSize() }) } diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/spark/metrics/MetricsConfig.scala index 5066b7ac22..ed505b0aa7 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/spark/metrics/MetricsConfig.scala @@ -1,44 +1,58 @@ package spark.metrics import java.util.Properties -import java.io.{File, FileInputStream} +import java.io.{File, FileInputStream, InputStream, IOException} import scala.collection.mutable import scala.util.matching.Regex -private[spark] class MetricsConfig(val configFile: String) { - val properties = new Properties() +import spark.Logging + +private[spark] class MetricsConfig(val configFile: Option[String]) extends Logging { + initLogging() + val DEFAULT_PREFIX = "*" val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r + val METRICS_CONF = "metrics.properties" + + val properties = new Properties() var propertyCategories: mutable.HashMap[String, Properties] = null private def setDefaultProperties(prop: Properties) { - prop.setProperty("*.sink.jmx.enabled", "default") - prop.setProperty("*.source.jvm.class", "spark.metrics.source.JvmSource") + // empty function, any default property can be set here } - def initilize() { + def initialize() { //Add default properties in case there's no properties file setDefaultProperties(properties) - val confFile = new File(configFile) - if (confFile.exists()) { - var fis: FileInputStream = null - try { - fis = new FileInputStream(configFile) - properties.load(fis) - } finally { - fis.close() + // If spark.metrics.conf is not set, try to get file in class path + var is: InputStream = null + try { + is = configFile match { + case Some(f) => new FileInputStream(f) + case None => getClass.getClassLoader.getResourceAsStream(METRICS_CONF) + } + + if (is != null) { + properties.load(is) } + } catch { + case e: Exception => logError("Error loading configure file", e) + } finally { + if (is != null) is.close() } propertyCategories = subProperties(properties, INSTANCE_REGEX) if (propertyCategories.contains(DEFAULT_PREFIX)) { import scala.collection.JavaConversions._ + val defaultProperty = propertyCategories(DEFAULT_PREFIX) - for ((inst, prop) <- propertyCategories; p <- defaultProperty - if inst != DEFAULT_PREFIX; if prop.getProperty(p._1) == null) { - prop.setProperty(p._1, p._2) + for { (inst, prop) <- propertyCategories + if (inst != DEFAULT_PREFIX) + (k, v) <- defaultProperty + if (prop.getProperty(k) == null) } { + prop.setProperty(k, v) } } } @@ -58,7 +72,7 @@ private[spark] class MetricsConfig(val configFile: String) { def getInstance(inst: String): Properties = { propertyCategories.get(inst) match { case Some(s) => s - case None => propertyCategories(DEFAULT_PREFIX) + case None => propertyCategories.getOrElse(DEFAULT_PREFIX, new Properties) } } } diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index 54f6c6e4da..2f87577ff3 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -11,19 +11,51 @@ import spark.Logging import spark.metrics.sink.Sink import spark.metrics.source.Source +/** + * Spark Metrics System, created by specific "instance", combined by source, + * sink, periodically poll source metrics data to sink destinations. + * + * "instance" specify "who" (the role) use metrics system. In spark there are several roles + * like master, worker, executor, client driver, these roles will create metrics system + * for monitoring. So instance represents these roles. Currently in Spark, several instances + * have already implemented: master, worker, executor, driver. + * + * "source" specify "where" (source) to collect metrics data. In metrics system, there exists + * two kinds of source: + * 1. Spark internal source, like MasterSource, WorkerSource, etc, which will collect + * Spark component's internal state, these sources are related to instance and will be + * added after specific metrics system is created. + * 2. Common source, like JvmSource, which will collect low level state, is configured by + * configuration and loaded through reflection. + * + * "sink" specify "where" (destination) to output metrics data to. Several sinks can be + * coexisted and flush metrics to all these sinks. + * + * Metrics configuration format is like below: + * [instance].[sink|source].[name].[options] = xxxx + * + * [instance] can be "master", "worker", "executor", "driver", which means only the specified + * instance has this property. + * wild card "*" can be used to replace instance name, which means all the instances will have + * this property. + * + * [sink|source] means this property belongs to source or sink. This field can only be source or sink. + * + * [name] specify the name of sink or source, it is custom defined. + * + * [options] is the specific property of this source or sink. + */ private[spark] class MetricsSystem private (val instance: String) extends Logging { initLogging() - val confFile = System.getProperty("spark.metrics.conf.file", "unsupported") - val metricsConfig = new MetricsConfig(confFile) + val confFile = System.getProperty("spark.metrics.conf") + val metricsConfig = new MetricsConfig(Option(confFile)) val sinks = new mutable.ArrayBuffer[Sink] val sources = new mutable.ArrayBuffer[Source] val registry = new MetricRegistry() - val DEFAULT_SINKS = Map("jmx" -> "spark.metrics.sink.JmxSink") - - metricsConfig.initilize() + metricsConfig.initialize() registerSources() registerSinks() @@ -37,7 +69,11 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin def registerSource(source: Source) { sources += source - registry.register(source.sourceName, source.metricRegistry) + try { + registry.register(source.sourceName, source.metricRegistry) + } catch { + case e: IllegalArgumentException => logInfo("Metrics already registered", e) + } } def registerSources() { @@ -51,7 +87,7 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val source = Class.forName(classPath).newInstance() registerSource(source.asInstanceOf[Source]) } catch { - case e: Exception => logError("source class " + classPath + " cannot be instantialized", e) + case e: Exception => logError("Source class " + classPath + " cannot be instantialized", e) } } } @@ -61,18 +97,14 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val sinkConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) sinkConfigs.foreach { kv => - val classPath = if (DEFAULT_SINKS.contains(kv._1)) { - DEFAULT_SINKS(kv._1) - } else { - // For non-default sink, a property class should be set and create using reflection - kv._2.getProperty("class") - } + val classPath = kv._2.getProperty("class") try { - val sink = Class.forName(classPath).getConstructor(classOf[Properties], classOf[MetricRegistry]) + val sink = Class.forName(classPath) + .getConstructor(classOf[Properties], classOf[MetricRegistry]) .newInstance(kv._2, registry) sinks += sink.asInstanceOf[Sink] } catch { - case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e) + case e: Exception => logError("Sink class " + classPath + " cannot be instantialized", e) } } } @@ -81,12 +113,17 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin private[spark] object MetricsSystem { val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r - val timeUnits = Map( - "illisecond" -> TimeUnit.MILLISECONDS, - "second" -> TimeUnit.SECONDS, - "minute" -> TimeUnit.MINUTES, - "hour" -> TimeUnit.HOURS, - "day" -> TimeUnit.DAYS) - - def createMetricsSystem(instance: String): MetricsSystem = new MetricsSystem(instance) + + val MINIMAL_POLL_UNIT = TimeUnit.SECONDS + val MINIMAL_POLL_PERIOD = 1 + + def checkMinimalPollingPeriod(pollUnit: TimeUnit, pollPeriod: Int) { + val period = MINIMAL_POLL_UNIT.convert(pollPeriod, pollUnit) + if (period < MINIMAL_POLL_PERIOD) { + throw new IllegalArgumentException("Polling period " + pollPeriod + " " + pollUnit + + " below than minimal polling period ") + } + } + + def createMetricsSystem(instance: String): MetricsSystem = new MetricsSystem(instance) } diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala index 437f24a575..eaaac5d153 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala @@ -8,22 +8,24 @@ import java.util.concurrent.TimeUnit import spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { - val CONSOLE_DEFAULT_PERIOD = "10" - val CONSOLE_DEFAULT_UNIT = "second" + val CONSOLE_DEFAULT_PERIOD = 10 + val CONSOLE_DEFAULT_UNIT = "SECONDS" val CONSOLE_KEY_PERIOD = "period" val CONSOLE_KEY_UNIT = "unit" val pollPeriod = Option(property.getProperty(CONSOLE_KEY_PERIOD)) match { case Some(s) => s.toInt - case None => CONSOLE_DEFAULT_PERIOD.toInt + case None => CONSOLE_DEFAULT_PERIOD } val pollUnit = Option(property.getProperty(CONSOLE_KEY_UNIT)) match { - case Some(s) => MetricsSystem.timeUnits(s) - case None => MetricsSystem.timeUnits(CONSOLE_DEFAULT_UNIT) + case Some(s) => TimeUnit.valueOf(s.toUpperCase()) + case None => TimeUnit.valueOf(CONSOLE_DEFAULT_UNIT) } + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) + val reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) .convertDurationsTo(TimeUnit.MILLISECONDS) .convertRatesTo(TimeUnit.SECONDS) diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/spark/metrics/sink/CsvSink.scala index a8ca819e87..aa5bff0d34 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/spark/metrics/sink/CsvSink.scala @@ -13,19 +13,21 @@ class CsvSink(val property: Properties, val registry: MetricRegistry) extends Si val CSV_KEY_UNIT = "unit" val CSV_KEY_DIR = "directory" - val CSV_DEFAULT_PERIOD = "10" - val CSV_DEFAULT_UNIT = "second" + val CSV_DEFAULT_PERIOD = 10 + val CSV_DEFAULT_UNIT = "SECONDS" val CSV_DEFAULT_DIR = "/tmp/" val pollPeriod = Option(property.getProperty(CSV_KEY_PERIOD)) match { case Some(s) => s.toInt - case None => CSV_DEFAULT_PERIOD.toInt + case None => CSV_DEFAULT_PERIOD } val pollUnit = Option(property.getProperty(CSV_KEY_UNIT)) match { - case Some(s) => MetricsSystem.timeUnits(s) - case None => MetricsSystem.timeUnits(CSV_DEFAULT_UNIT) + case Some(s) => TimeUnit.valueOf(s.toUpperCase()) + case None => TimeUnit.valueOf(CSV_DEFAULT_UNIT) } + + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) val pollDir = Option(property.getProperty(CSV_KEY_DIR)) match { case Some(s) => s diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala index 38158b8a2b..87d27cc70d 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala @@ -8,23 +8,23 @@ private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends val metricRegistry = new MetricRegistry() val sourceName = "DAGScheduler" - metricRegistry.register(MetricRegistry.name("stage", "failedStage"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("stage", "failedStages", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.failed.size }) - metricRegistry.register(MetricRegistry.name("stage", "runningStage"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("stage", "runningStages", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.running.size }) - metricRegistry.register(MetricRegistry.name("stage", "waitingStage"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("stage", "waitingStages", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.waiting.size }) - metricRegistry.register(MetricRegistry.name("job", "allJobs"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("job", "allJobs", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.nextRunId.get() }) - metricRegistry.register(MetricRegistry.name("job", "ActiveJobs"), new Gauge[Int] { + metricRegistry.register(MetricRegistry.name("job", "activeJobs", "number"), new Gauge[Int] { override def getValue: Int = dagScheduler.activeJobs.size }) } diff --git a/core/src/main/scala/spark/storage/BlockManagerSource.scala b/core/src/main/scala/spark/storage/BlockManagerSource.scala index f964827102..4faa715c94 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/spark/storage/BlockManagerSource.scala @@ -9,27 +9,40 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager) extends val metricRegistry = new MetricRegistry() val sourceName = "BlockManager" - metricRegistry.register(MetricRegistry.name("memory", "maxMem"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "maxMem", "MBytes"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) - maxMem + val maxMem = storageStatusList.map(_.maxMem).reduce(_ + _) + maxMem / 1024 / 1024 } }) - metricRegistry.register(MetricRegistry.name("memory", "remainingMem"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "remainingMem", "MBytes"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_) - remainingMem + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_ + _) + remainingMem / 1024 / 1024 } }) - metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed"), new Gauge[Long] { + metricRegistry.register(MetricRegistry.name("memory", "memUsed", "MBytes"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).reduceOption(_+_).getOrElse(0L) - diskSpaceUsed + val maxMem = storageStatusList.map(_.maxMem).reduce(_ + _) + val remainingMem = storageStatusList.map(_.memRemaining).reduce(_ + _) + (maxMem - remainingMem) / 1024 / 1024 + } + }) + + metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed", "MBytes"), new Gauge[Long] { + override def getValue: Long = { + val storageStatusList = blockManager.master.getStorageStatus + val diskSpaceUsed = storageStatusList + .flatMap(_.blocks.values.map(_.diskSize)) + .reduceOption(_ + _) + .getOrElse(0L) + + diskSpaceUsed / 1024 / 1024 } }) } diff --git a/core/src/test/resources/test_metrics_config.properties b/core/src/test/resources/test_metrics_config.properties index 2011940003..2b31ddf2eb 100644 --- a/core/src/test/resources/test_metrics_config.properties +++ b/core/src/test/resources/test_metrics_config.properties @@ -1,6 +1,6 @@ *.sink.console.period = 10 -*.sink.console.unit = second +*.sink.console.unit = seconds *.source.jvm.class = spark.metrics.source.JvmSource master.sink.console.period = 20 -master.sink.console.unit = minute - +master.sink.console.unit = minutes + diff --git a/core/src/test/resources/test_metrics_system.properties b/core/src/test/resources/test_metrics_system.properties index 06afbc6625..d5479f0298 100644 --- a/core/src/test/resources/test_metrics_system.properties +++ b/core/src/test/resources/test_metrics_system.properties @@ -1,7 +1,7 @@ *.sink.console.period = 10 -*.sink.console.unit = second +*.sink.console.unit = seconds test.sink.console.class = spark.metrics.sink.ConsoleSink test.sink.dummy.class = spark.metrics.sink.DummySink test.source.dummy.class = spark.metrics.source.DummySource test.sink.console.period = 20 -test.sink.console.unit = minute +test.sink.console.unit = minutes diff --git a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala index bb1be4f4fc..87cd2ffad2 100644 --- a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala @@ -15,40 +15,36 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { } test("MetricsConfig with default properties") { - val conf = new MetricsConfig("dummy-file") - conf.initilize() + val conf = new MetricsConfig(Option("dummy-file")) + conf.initialize() - assert(conf.properties.size() === 2) - assert(conf.properties.getProperty("*.sink.jmx.enabled") === "default") - assert(conf.properties.getProperty("*.source.jvm.class") === "spark.metrics.source.JvmSource") + assert(conf.properties.size() === 0) assert(conf.properties.getProperty("test-for-dummy") === null) val property = conf.getInstance("random") - assert(property.size() === 2) - assert(property.getProperty("sink.jmx.enabled") === "default") - assert(property.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") + assert(property.size() === 0) } test("MetricsConfig with properties set") { - val conf = new MetricsConfig(filePath) - conf.initilize() + val conf = new MetricsConfig(Option(filePath)) + conf.initialize() val masterProp = conf.getInstance("master") - assert(masterProp.size() === 4) + assert(masterProp.size() === 3) assert(masterProp.getProperty("sink.console.period") === "20") - assert(masterProp.getProperty("sink.console.unit") === "minute") - assert(masterProp.getProperty("sink.jmx.enabled") === "default") - assert(masterProp.getProperty("source.jvm.class") == "spark.metrics.source.JvmSource") + assert(masterProp.getProperty("sink.console.unit") === "minutes") + assert(masterProp.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") val workerProp = conf.getInstance("worker") - assert(workerProp.size() === 4) + assert(workerProp.size() === 3) assert(workerProp.getProperty("sink.console.period") === "10") - assert(workerProp.getProperty("sink.console.unit") === "second") + assert(workerProp.getProperty("sink.console.unit") === "seconds") + assert(masterProp.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") } test("MetricsConfig with subProperties") { - val conf = new MetricsConfig(filePath) - conf.initilize() + val conf = new MetricsConfig(Option(filePath)) + conf.initialize() val propCategories = conf.propertyCategories assert(propCategories.size === 2) @@ -59,14 +55,10 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { assert(sourceProps("jvm").getProperty("class") === "spark.metrics.source.JvmSource") val sinkProps = conf.subProperties(masterProp, MetricsSystem.SINK_REGEX) - assert(sinkProps.size === 2) + assert(sinkProps.size === 1) assert(sinkProps.contains("console")) - assert(sinkProps.contains("jmx")) val consoleProps = sinkProps("console") assert(consoleProps.size() === 2) - - val jmxProps = sinkProps("jmx") - assert(jmxProps.size() === 1) } } diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala index 462c28e894..c189996417 100644 --- a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -12,7 +12,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { before { filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile() - System.setProperty("spark.metrics.conf.file", filePath) + System.setProperty("spark.metrics.conf", filePath) } test("MetricsSystem with default config") { @@ -20,9 +20,8 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { val sources = metricsSystem.sources val sinks = metricsSystem.sinks - assert(sources.length === 1) - assert(sinks.length === 1) - assert(sources(0).sourceName === "jvm") + assert(sources.length === 0) + assert(sinks.length === 0) } test("MetricsSystem with sources add") { @@ -30,11 +29,11 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { val sources = metricsSystem.sources val sinks = metricsSystem.sinks - assert(sources.length === 1) - assert(sinks.length === 2) + assert(sources.length === 0) + assert(sinks.length === 1) val source = new spark.deploy.master.MasterSource(null) metricsSystem.registerSource(source) - assert(sources.length === 2) + assert(sources.length === 1) } } -- cgit v1.2.3 From 93c6015f82d4d27a6f09686b1e849be1cbbd0615 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 24 Jul 2013 10:53:02 -0700 Subject: Shows task status and running tasks on Stage Page: fixes SPARK-804 and 811 --- .../main/scala/spark/scheduler/cluster/TaskInfo.scala | 12 ++++++++++++ core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 5 +++++ core/src/main/scala/spark/ui/jobs/JobProgressUI.scala | 16 ++++++++++++++++ core/src/main/scala/spark/ui/jobs/StagePage.scala | 19 +++++++++++++------ 4 files changed, 46 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index a1ebd48b01..f840100eca 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -51,6 +51,18 @@ class TaskInfo( def running: Boolean = !finished + def status: String = { + if (running) + return "RUNNING" + if (failed) + return "FAILED" + if (successful) + return "SUCCESSFUL" + if (finished) + return "FINISHED" + "UNKNOWN" + } + def duration: Long = { if (!finished) { throw new UnsupportedOperationException("duration() called on unfinished tasks") diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index bad5c442ab..db1c902955 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -124,6 +124,10 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { if (!executorToTasksActive.contains(eid)) executorToTasksActive(eid) = HashSet[Long]() executorToTasksActive(eid) += taskStart.taskInfo.taskId + val taskList = executorToTaskInfos.getOrElse( + eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) + taskList += ((taskStart.taskInfo, None, None)) + executorToTaskInfos(eid) = taskList } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { @@ -142,6 +146,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } val taskList = executorToTaskInfos.getOrElse( eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) + taskList -= ((taskEnd.taskInfo, None, None)) taskList += ((taskEnd.taskInfo, metrics, failureInfo)) executorToTaskInfos(eid) = taskList } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 44dcf82d11..6e332415db 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -65,6 +65,7 @@ private[spark] class JobProgressListener extends SparkListener { val completedStages = ListBuffer[Stage]() val failedStages = ListBuffer[Stage]() + val stageToTasksActive = HashMap[Int, HashSet[Long]]() val stageToTasksComplete = HashMap[Int, Int]() val stageToTasksFailed = HashMap[Int, Int]() val stageToTaskInfos = @@ -93,8 +94,22 @@ private[spark] class JobProgressListener extends SparkListener { override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = activeStages += stageSubmitted.stage + override def onTaskStart(taskStart: SparkListenerTaskStart) { + val sid = taskStart.task.stageId + if (!stageToTasksActive.contains(sid)) + stageToTasksActive(sid) = HashSet[Long]() + stageToTasksActive(sid) += taskStart.taskInfo.taskId + val taskList = stageToTaskInfos.getOrElse( + sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) + taskList += ((taskStart.taskInfo, None, None)) + stageToTaskInfos(sid) = taskList + } + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId + if (!stageToTasksActive.contains(sid)) + stageToTasksActive(sid) = HashSet[Long]() + stageToTasksActive(sid) -= taskEnd.taskInfo.taskId val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => @@ -106,6 +121,7 @@ private[spark] class JobProgressListener extends SparkListener { } val taskList = stageToTaskInfos.getOrElse( sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) + taskList -= ((taskEnd.taskInfo, None, None)) taskList += ((taskEnd.taskInfo, metrics, failureInfo)) stageToTaskInfos(sid) = taskList } diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 292966f23a..b77b29045e 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -41,8 +41,8 @@ private[spark] class StagePage(parent: JobProgressUI) { if (!listener.stageToTaskInfos.contains(stageId)) { val content =
        -

        Summary Metrics

        No tasks have finished yet -

        Tasks

        No tasks have finished yet +

        Summary Metrics

        No tasks have started yet +

        Tasks

        No tasks have started yet
        return headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId), Jobs) } @@ -53,7 +53,7 @@ private[spark] class StagePage(parent: JobProgressUI) { val shuffleWrite = listener.hasShuffleWrite(stageId) val taskHeaders: Seq[String] = - Seq("Task ID", "Duration", "Locality Level", "Worker", "Launch Time") ++ + Seq("Task ID", "Status", "Duration", "Locality Level", "Worker", "Launch Time") ++ {if (shuffleRead) Seq("Shuffle Read") else Nil} ++ {if (shuffleWrite) Seq("Shuffle Write") else Nil} ++ Seq("Details") @@ -61,7 +61,7 @@ private[spark] class StagePage(parent: JobProgressUI) { val taskTable = listingTable(taskHeaders, taskRow, tasks) // Excludes tasks which failed and have incomplete metrics - val validTasks = tasks.filter(t => Option(t._2).isDefined) + val validTasks = tasks.filter(t => t._1.status == "SUCCESSFUL" && (Option(t._2).isDefined)) val summaryTable: Option[Seq[Node]] = if (validTasks.size == 0) { @@ -108,10 +108,17 @@ private[spark] class StagePage(parent: JobProgressUI) { def fmtStackTrace(trace: Seq[StackTraceElement]): Seq[Node] = trace.map(e => {e.toString}) val (info, metrics, exception) = taskData + + val duration = if (info.status == "RUNNING") info.timeRunning(System.currentTimeMillis()) + else metrics.map(m => m.executorRunTime).getOrElse(1) + val formatDuration = if (info.status == "RUNNING") parent.formatDuration(duration) + else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") + {info.taskId} - m.executorRunTime.toString}.getOrElse("1")}> - {metrics.map{m => parent.formatDuration(m.executorRunTime)}.getOrElse("")} + {info.status} + + {formatDuration} {info.taskLocality} {info.hostPort} -- cgit v1.2.3 From bd3931c87415679fef8b91390d50188c3da98430 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 24 Jul 2013 11:27:17 -0700 Subject: Changed ifs with returns to if/else --- .../main/scala/spark/scheduler/cluster/TaskInfo.scala | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index f840100eca..afcfe96a09 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -53,14 +53,15 @@ class TaskInfo( def status: String = { if (running) - return "RUNNING" - if (failed) - return "FAILED" - if (successful) - return "SUCCESSFUL" - if (finished) - return "FINISHED" - "UNKNOWN" + "RUNNING" + else if (failed) + "FAILED" + else if (successful) + "SUCCESSFUL" + else if (finished) + "FINISHED" + else + "UNKNOWN" } def duration: Long = { -- cgit v1.2.3 From 4280e1768d09e43b55f26556e14a8eb70d9b9fad Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 24 Jul 2013 12:48:48 -0700 Subject: Removed finished status for task info, changed name of success case --- core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index afcfe96a09..c693b722ac 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -57,9 +57,7 @@ class TaskInfo( else if (failed) "FAILED" else if (successful) - "SUCCESSFUL" - else if (finished) - "FINISHED" + "SUCCESS" else "UNKNOWN" } -- cgit v1.2.3 From 57009eef9099aa3b35b9262d9637af3dc94266a1 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 24 Jul 2013 13:43:09 -0700 Subject: Fixed consistency of "success" status string --- core/src/main/scala/spark/ui/jobs/StagePage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index b77b29045e..654f347723 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -61,7 +61,7 @@ private[spark] class StagePage(parent: JobProgressUI) { val taskTable = listingTable(taskHeaders, taskRow, tasks) // Excludes tasks which failed and have incomplete metrics - val validTasks = tasks.filter(t => t._1.status == "SUCCESSFUL" && (Option(t._2).isDefined)) + val validTasks = tasks.filter(t => t._1.status == "SUCCESS" && (Option(t._2).isDefined)) val summaryTable: Option[Seq[Node]] = if (validTasks.size == 0) { -- cgit v1.2.3 From 1d101928069c31aec039bb8b69c9bf4c204eead4 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Mon, 22 Jul 2013 13:11:35 -0700 Subject: Fix setting of SPARK_EXAMPLES_JAR --- bin/compute-classpath.sh | 11 ----------- run | 13 +++++++++++++ 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index eb270a5428..e4ce1ca848 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -76,17 +76,6 @@ function dev_classpath { CLASSPATH="$CLASSPATH:$jar" done - # Figure out the JAR file that our examples were packaged into. This includes a bit of a hack - # to avoid the -sources and -doc packages that are built by publish-local. - if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then - # Use the JAR from the SBT build - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` - fi - if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then - # Use the JAR from the Maven build - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` - fi - # Add Scala standard library if [ -z "$SCALA_LIBRARY_PATH" ]; then if [ -z "$SCALA_HOME" ]; then diff --git a/run b/run index f49aa92796..4cffda4708 100755 --- a/run +++ b/run @@ -17,6 +17,8 @@ # limitations under the License. # +SCALA_VERSION=2.9.3 + # Figure out where the Scala framework is installed FWDIR="$(cd `dirname $0`; pwd)" @@ -136,6 +138,17 @@ if [ ! -f "$FWDIR/RELEASE" ]; then echo "You need to compile Spark repl module before running this program" >&2 exit 1 fi + + # Figure out the JAR file that our examples were packaged into. This includes a bit of a hack + # to avoid the -sources and -doc packages that are built by publish-local. + if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then + # Use the JAR from the SBT build + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar` + fi + if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then + # Use the JAR from the Maven build + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar` + fi fi # Compute classpath using external script -- cgit v1.2.3 From c258718606a2960649dde0a4925fcf385d617c37 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 24 Jul 2013 16:12:32 -0700 Subject: Fix Maven build errors after previous commits --- pom.xml | 18 ++++---- tools/pom.xml | 133 +++++++++++++++++++++++++++++++++++++++++++++++++++------- 2 files changed, 129 insertions(+), 22 deletions(-) diff --git a/pom.xml b/pom.xml index 1fe288b92c..4cf9589b07 100644 --- a/pom.xml +++ b/pom.xml @@ -248,6 +248,16 @@ lift-json_2.9.2 2.5
        + + com.codahale.metrics + metrics-core + 3.0.0 + + + com.codahale.metrics + metrics-jvm + 3.0.0 + org.scala-lang scala-compiler @@ -267,14 +277,6 @@ org.scala-lang scalap ${scala.version} - com.codahale.metrics - metrics-core - 3.0.0 - - - com.codahale.metrics - metrics-jvm - 3.0.0 diff --git a/tools/pom.xml b/tools/pom.xml index 3dfba5808c..1125aba4f1 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,28 +20,16 @@ org.spark-project spark-parent - 0.7.4-SNAPSHOT + 0.8.0-SNAPSHOT ../pom.xml org.spark-project - spark-tools_2.9.3 + spark-tools jar Spark Project Tools http://spark-project.org/ - - - org.spark-project - spark-core_2.9.3 - ${project.version} - - - org.spark-project - spark-streaming_2.9.3 - ${project.version} - - target/scala-${scala.version}/classes target/scala-${scala.version}/test-classes @@ -60,4 +48,121 @@ + + + + hadoop1 + + + org.spark-project + spark-core + ${project.version} + hadoop1 + + + org.spark-project + spark-streaming + ${project.version} + hadoop1 + + + org.apache.hadoop + hadoop-core + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop1 + + + + + + + hadoop2 + + + org.spark-project + spark-core + ${project.version} + hadoop2 + + + org.spark-project + spark-streaming + ${project.version} + hadoop2 + + + org.apache.hadoop + hadoop-core + provided + + + org.apache.hadoop + hadoop-client + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop2 + + + + + + + hadoop2-yarn + + + org.spark-project + spark-core + ${project.version} + hadoop2-yarn + + + org.spark-project + spark-streaming + ${project.version} + hadoop2-yarn + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.hadoop + hadoop-yarn-api + provided + + + org.apache.hadoop + hadoop-yarn-common + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop2-yarn + + + + + + -- cgit v1.2.3 From 8e0939f5a9c0823d51d0e94185b55d4a02628a53 Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 20:43:57 -0700 Subject: refactor Kryo serializer support to use chill/chill-java --- core/src/main/scala/spark/KryoSerializer.scala | 143 ++++----------------- .../src/test/scala/spark/KryoSerializerSuite.scala | 32 +---- project/SparkBuild.scala | 5 +- 3 files changed, 37 insertions(+), 143 deletions(-) diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala index ee37da7948..2ea90d41c3 100644 --- a/core/src/main/scala/spark/KryoSerializer.scala +++ b/core/src/main/scala/spark/KryoSerializer.scala @@ -19,24 +19,15 @@ package spark import java.io._ import java.nio.ByteBuffer -import java.nio.channels.Channels - -import scala.collection.immutable -import scala.collection.mutable - -import com.esotericsoftware.kryo._ -import com.esotericsoftware.kryo.{Serializer => KSerializer} +import com.esotericsoftware.kryo.{Kryo, KryoException} import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} -import de.javakaffee.kryoserializers.KryoReflectionFactorySupport - +import com.twitter.chill.ScalaKryoInstantiator import serializer.{SerializerInstance, DeserializationStream, SerializationStream} import spark.broadcast._ import spark.storage._ -private[spark] -class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { - +private[spark] class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { val output = new KryoOutput(outStream) def writeObject[T](t: T): SerializationStream = { @@ -48,9 +39,7 @@ class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends Seria def close() { output.close() } } -private[spark] -class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { - +private[spark] class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { val input = new KryoInput(inStream) def readObject[T](): T = { @@ -58,7 +47,7 @@ class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends Deser kryo.readClassAndObject(input).asInstanceOf[T] } catch { // DeserializationStream uses the EOF exception to indicate stopping condition. - case e: com.esotericsoftware.kryo.KryoException => throw new java.io.EOFException + case _: KryoException => throw new EOFException } } @@ -69,10 +58,9 @@ class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends Deser } private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance { - - val kryo = ks.kryo.get() - val output = ks.output.get() - val input = ks.input.get() + val kryo = ks.newKryo() + val output = ks.newKryoOutput() + val input = ks.newKryoInput() def serialize[T](t: T): ByteBuffer = { output.clear() @@ -108,125 +96,51 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ * serialization. */ trait KryoRegistrator { - def registerClasses(kryo: Kryo): Unit + def registerClasses(kryo: Kryo) } /** * A Spark serializer that uses the [[http://code.google.com/p/kryo/wiki/V1Documentation Kryo 1.x library]]. */ class KryoSerializer extends spark.serializer.Serializer with Logging { + private val bufferSize = System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 - val bufferSize = System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 + def newKryoOutput() = new KryoOutput(bufferSize) - val kryo = new ThreadLocal[Kryo] { - override def initialValue = createKryo() - } - - val output = new ThreadLocal[KryoOutput] { - override def initialValue = new KryoOutput(bufferSize) - } - - val input = new ThreadLocal[KryoInput] { - override def initialValue = new KryoInput(bufferSize) - } + def newKryoInput() = new KryoInput(bufferSize) - def createKryo(): Kryo = { - val kryo = new KryoReflectionFactorySupport() + def newKryo(): Kryo = { + val instantiator = new ScalaKryoInstantiator + val kryo = instantiator.newKryo() + val classLoader = Thread.currentThread.getContextClassLoader // Register some commonly used classes val toRegister: Seq[AnyRef] = Seq( - // Arrays - Array(1), Array(1.0), Array(1.0f), Array(1L), Array(""), Array(("", "")), - Array(new java.lang.Object), Array(1.toByte), Array(true), Array('c'), - // Specialized Tuple2s - ("", ""), ("", 1), (1, 1), (1.0, 1.0), (1L, 1L), - (1, 1.0), (1.0, 1), (1L, 1.0), (1.0, 1L), (1, 1L), (1L, 1), - // Scala collections - List(1), mutable.ArrayBuffer(1), - // Options and Either - Some(1), Left(1), Right(1), - // Higher-dimensional tuples - (1, 1, 1), (1, 1, 1, 1), (1, 1, 1, 1, 1), - None, ByteBuffer.allocate(1), StorageLevel.MEMORY_ONLY, PutBlock("1", ByteBuffer.allocate(1), StorageLevel.MEMORY_ONLY), GotBlock("1", ByteBuffer.allocate(1)), GetBlock("1") ) - for (obj <- toRegister) { - kryo.register(obj.getClass) - } + + for (obj <- toRegister) kryo.register(obj.getClass) // Allow sending SerializableWritable kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer()) kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer()) - // Register some commonly used Scala singleton objects. Because these - // are singletons, we must return the exact same local object when we - // deserialize rather than returning a clone as FieldSerializer would. - class SingletonSerializer[T](obj: T) extends KSerializer[T] { - override def write(kryo: Kryo, output: KryoOutput, obj: T) {} - override def read(kryo: Kryo, input: KryoInput, cls: java.lang.Class[T]): T = obj - } - kryo.register(None.getClass, new SingletonSerializer[AnyRef](None)) - kryo.register(Nil.getClass, new SingletonSerializer[AnyRef](Nil)) - - // Register maps with a special serializer since they have complex internal structure - class ScalaMapSerializer(buildMap: Array[(Any, Any)] => scala.collection.Map[Any, Any]) - extends KSerializer[Array[(Any, Any)] => scala.collection.Map[Any, Any]] { - - //hack, look at https://groups.google.com/forum/#!msg/kryo-users/Eu5V4bxCfws/k-8UQ22y59AJ - private final val FAKE_REFERENCE = new Object() - override def write( - kryo: Kryo, - output: KryoOutput, - obj: Array[(Any, Any)] => scala.collection.Map[Any, Any]) { - val map = obj.asInstanceOf[scala.collection.Map[Any, Any]] - output.writeInt(map.size) - for ((k, v) <- map) { - kryo.writeClassAndObject(output, k) - kryo.writeClassAndObject(output, v) - } - } - override def read ( - kryo: Kryo, - input: KryoInput, - cls: Class[Array[(Any, Any)] => scala.collection.Map[Any, Any]]) - : Array[(Any, Any)] => scala.collection.Map[Any, Any] = { - kryo.reference(FAKE_REFERENCE) - val size = input.readInt() - val elems = new Array[(Any, Any)](size) - for (i <- 0 until size) { - val k = kryo.readClassAndObject(input) - val v = kryo.readClassAndObject(input) - elems(i)=(k,v) - } - buildMap(elems).asInstanceOf[Array[(Any, Any)] => scala.collection.Map[Any, Any]] + // Allow the user to register their own classes by setting spark.kryo.registrator + try { + Option(System.getProperty("spark.kryo.registrator")).foreach { regCls => + logDebug("Running user registrator: " + regCls) + val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] + reg.registerClasses(kryo) } + } catch { + case _: Exception => println("Failed to register spark.kryo.registrator") } - kryo.register(mutable.HashMap().getClass, new ScalaMapSerializer(mutable.HashMap() ++ _)) - // TODO: add support for immutable maps too; this is more annoying because there are many - // subclasses of immutable.Map for small maps (with <= 4 entries) - val map1 = Map[Any, Any](1 -> 1) - val map2 = Map[Any, Any](1 -> 1, 2 -> 2) - val map3 = Map[Any, Any](1 -> 1, 2 -> 2, 3 -> 3) - val map4 = Map[Any, Any](1 -> 1, 2 -> 2, 3 -> 3, 4 -> 4) - val map5 = Map[Any, Any](1 -> 1, 2 -> 2, 3 -> 3, 4 -> 4, 5 -> 5) - kryo.register(map1.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap)) - kryo.register(map2.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap)) - kryo.register(map3.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap)) - kryo.register(map4.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap)) - kryo.register(map5.getClass, new ScalaMapSerializer(mutable.HashMap() ++ _ toMap)) - // Allow the user to register their own classes by setting spark.kryo.registrator - val regCls = System.getProperty("spark.kryo.registrator") - if (regCls != null) { - logInfo("Running user registrator: " + regCls) - val classLoader = Thread.currentThread.getContextClassLoader - val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator] - reg.registerClasses(kryo) - } + kryo.setClassLoader(classLoader) // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops kryo.setReferences(System.getProperty("spark.kryo.referenceTracking", "true").toBoolean) @@ -235,7 +149,6 @@ class KryoSerializer extends spark.serializer.Serializer with Logging { } def newInstance(): SerializerInstance = { - this.kryo.get().setClassLoader(Thread.currentThread().getContextClassLoader) new KryoSerializerInstance(this) } -} +} \ No newline at end of file diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/spark/KryoSerializerSuite.scala index c3323dcbb3..cb59bb9b65 100644 --- a/core/src/test/scala/spark/KryoSerializerSuite.scala +++ b/core/src/test/scala/spark/KryoSerializerSuite.scala @@ -1,30 +1,10 @@ -/* - * 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 import scala.collection.mutable -import scala.collection.immutable import org.scalatest.FunSuite import com.esotericsoftware.kryo._ -import SparkContext._ - class KryoSerializerSuite extends FunSuite { test("basic types") { val ser = (new KryoSerializer).newInstance() @@ -53,6 +33,7 @@ class KryoSerializerSuite extends FunSuite { check(Array(true, false, true)) check(Array('a', 'b', 'c')) check(Array[Int]()) + check(Array(Array("1", "2"), Array("1", "2", "3", "4"))) } test("pairs") { @@ -99,11 +80,10 @@ class KryoSerializerSuite extends FunSuite { check(mutable.HashMap(1 -> "one", 2 -> "two")) check(mutable.HashMap("one" -> 1, "two" -> 2)) check(List(Some(mutable.HashMap(1->1, 2->2)), None, Some(mutable.HashMap(3->4)))) - check(List(mutable.HashMap("one" -> 1, "two" -> 2),mutable.HashMap(1->"one",2->"two",3->"three"))) } test("custom registrator") { - import spark.test._ + import KryoTest._ System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName) val ser = (new KryoSerializer).newInstance() @@ -123,14 +103,14 @@ class KryoSerializerSuite extends FunSuite { val hashMap = new java.util.HashMap[String, String] hashMap.put("foo", "bar") check(hashMap) - + System.clearProperty("spark.kryo.registrator") } } -package test { +object KryoTest { case class CaseClass(i: Int, s: String) {} - + class ClassWithNoArgConstructor { var x: Int = 0 override def equals(other: Any) = other match { @@ -154,4 +134,4 @@ package test { k.register(classOf[java.util.HashMap[_, _]]) } } -} +} \ No newline at end of file diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d4d70afdd5..9920e00a67 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -170,7 +170,6 @@ object SparkBuild extends Build { "com.ning" % "compress-lzf" % "0.8.4", "org.ow2.asm" % "asm" % "4.0", "com.google.protobuf" % "protobuf-java" % "2.4.1", - "de.javakaffee" % "kryo-serializers" % "0.22", "com.typesafe.akka" % "akka-actor" % "2.0.5" excludeAll(excludeNetty), "com.typesafe.akka" % "akka-remote" % "2.0.5" excludeAll(excludeNetty), "com.typesafe.akka" % "akka-slf4j" % "2.0.5" excludeAll(excludeNetty), @@ -181,7 +180,9 @@ object SparkBuild extends Build { "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test", "com.codahale.metrics" % "metrics-core" % "3.0.0", - "com.codahale.metrics" % "metrics-jvm" % "3.0.0" + "com.codahale.metrics" % "metrics-jvm" % "3.0.0", + "com.twitter" % "chill_2.9.3" % "0.3.0", + "com.twitter" % "chill-java" % "0.3.0" ) ++ ( if (HADOOP_MAJOR_VERSION == "2") { if (HADOOP_YARN) { -- cgit v1.2.3 From a1c515fb02646c857c607949120c7d3cb29f4dce Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 20:50:32 -0700 Subject: add copyright back in --- core/src/test/scala/spark/KryoSerializerSuite.scala | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/spark/KryoSerializerSuite.scala index cb59bb9b65..f6e3eec546 100644 --- a/core/src/test/scala/spark/KryoSerializerSuite.scala +++ b/core/src/test/scala/spark/KryoSerializerSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 import scala.collection.mutable -- cgit v1.2.3 From fc4b025314232fbeeaacb84ec3e2491e9c4ddd1b Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 20:53:15 -0700 Subject: add test --- core/src/test/scala/spark/KryoSerializerSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/spark/KryoSerializerSuite.scala index f6e3eec546..30d2d5282b 100644 --- a/core/src/test/scala/spark/KryoSerializerSuite.scala +++ b/core/src/test/scala/spark/KryoSerializerSuite.scala @@ -97,6 +97,7 @@ class KryoSerializerSuite extends FunSuite { check(mutable.HashMap(1 -> "one", 2 -> "two")) check(mutable.HashMap("one" -> 1, "two" -> 2)) check(List(Some(mutable.HashMap(1->1, 2->2)), None, Some(mutable.HashMap(3->4)))) + check(List(mutable.HashMap("one" -> 1, "two" -> 2),mutable.HashMap(1->"one",2->"two",3->"three"))) } test("custom registrator") { -- cgit v1.2.3 From 30a369a808cdc16effe9c5755c74fc5a3f3d7cac Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 20:55:48 -0700 Subject: update pom.xml --- pom.xml | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 4cf9589b07..469dc64329 100644 --- a/pom.xml +++ b/pom.xml @@ -193,9 +193,14 @@ 2.4.1 - de.javakaffee - kryo-serializers - 0.22 + com.twitter + chill + 0.3.0 + + + com.twitter + chill-java + 0.3.0 com.typesafe.akka -- cgit v1.2.3 From e56aa75de0f3c00e9942f0863c0fb8c57aab5321 Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 22:08:09 -0700 Subject: fix wrapping --- core/src/main/scala/spark/KryoSerializer.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala index 2ea90d41c3..eeb2993d8a 100644 --- a/core/src/main/scala/spark/KryoSerializer.scala +++ b/core/src/main/scala/spark/KryoSerializer.scala @@ -27,7 +27,8 @@ import serializer.{SerializerInstance, DeserializationStream, SerializationStrea import spark.broadcast._ import spark.storage._ -private[spark] class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { +private[spark] +class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { val output = new KryoOutput(outStream) def writeObject[T](t: T): SerializationStream = { @@ -39,7 +40,8 @@ private[spark] class KryoSerializationStream(kryo: Kryo, outStream: OutputStream def close() { output.close() } } -private[spark] class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { +private[spark] +class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { val input = new KryoInput(inStream) def readObject[T](): T = { -- cgit v1.2.3 From e2421c13117c7d9cc950da8c9de4903d0ea49c43 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 25 Jul 2013 00:05:43 -0700 Subject: Update Chill reference in pom.xml too --- core/pom.xml | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 237d988712..8c740e3887 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -57,8 +57,14 @@ protobuf-java - de.javakaffee - kryo-serializers + com.twitter + chill + 0.3.0 + + + com.twitter + chill-java + 0.3.0 com.typesafe.akka -- cgit v1.2.3 From 8eb8b52997da56acb3500fa48aac1ab9a3dfdddf Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 25 Jul 2013 08:58:02 -0700 Subject: Fix Chill version in Maven --- core/pom.xml | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 8c740e3887..f0c936c86a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -58,7 +58,7 @@ com.twitter - chill + chill_2.9.3 0.3.0 diff --git a/pom.xml b/pom.xml index 469dc64329..44729bd422 100644 --- a/pom.xml +++ b/pom.xml @@ -194,7 +194,7 @@ com.twitter - chill + chill_2.9.3 0.3.0 -- cgit v1.2.3 From a6de90c927e5569e30a7bc6188253bd14c32c3a9 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Thu, 25 Jul 2013 01:01:50 -0700 Subject: For standalone mode, get JAVA_HOME, SPARK_JAVA_OPTS, SPARK_LIBRARY_PATH from application env, not worker env --- core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 8b51ff1c3a..47d3390928 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -51,6 +51,9 @@ private[spark] class ExecutorRunner( var process: Process = null var shutdownHook: Thread = null + private def getAppEnv(key: String): Option[String] = + appDesc.command.environment.get(key).orElse(Option(getenv(key))) + def start() { workerThread = new Thread("ExecutorRunner for " + fullId) { override def run() { fetchAndRunExecutor() } @@ -95,7 +98,7 @@ private[spark] class ExecutorRunner( def buildCommandSeq(): Seq[String] = { val command = appDesc.command - val runner = Option(getenv("JAVA_HOME")).map(_ + "/bin/java").getOrElse("java") + val runner = getAppEnv("JAVA_HOME").map(_ + "/bin/java").getOrElse("java") // SPARK-698: do not call the run.cmd script, as process.destroy() // fails to kill a process tree on Windows Seq(runner) ++ buildJavaOpts() ++ Seq(command.mainClass) ++ @@ -107,10 +110,10 @@ private[spark] class ExecutorRunner( * the way the JAVA_OPTS are assembled there. */ def buildJavaOpts(): Seq[String] = { - val libraryOpts = Option(getenv("SPARK_LIBRARY_PATH")) + val libraryOpts = getAppEnv("SPARK_LIBRARY_PATH") .map(p => List("-Djava.library.path=" + p)) .getOrElse(Nil) - val userOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString).getOrElse(Nil) + val userOpts = getAppEnv("SPARK_JAVA_OPTS").map(Utils.splitCommandString).getOrElse(Nil) val memoryOpts = Seq("-Xms" + memory + "M", "-Xmx" + memory + "M") // Figure out our classpath with the external compute-classpath script -- cgit v1.2.3