From 25c0739bad7222d45b4818c7bf6987521a3509d2 Mon Sep 17 00:00:00 2001 From: folone Date: Mon, 14 Jan 2013 09:52:11 +0100 Subject: [PATCH 0001/2521] =?UTF-8?q?Moved=20to=20scala=202.10.0.=20Notabl?= =?UTF-8?q?e=20changes=20are:=20-=20akka=20=202.0.3=20=20=E2=86=92=202.1.0?= =?UTF-8?q?=20-=20spray=201.0-M1=20=E2=86=92=201.1-M7=20For=20now=20the=20?= =?UTF-8?q?repl=20subproject=20is=20commented=20out,=20as=20scala=20reflec?= =?UTF-8?q?tion=20api=20changed=20very=20much=20since=20the=20introduction?= =?UTF-8?q?=20of=20macros.?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- core/src/main/scala/spark/CacheTracker.scala | 6 +- .../main/scala/spark/MapOutputTracker.scala | 6 +- .../spark/api/java/function/Function.java | 8 +-- .../spark/api/java/function/Function2.java | 9 +-- .../java/function/PairFlatMapFunction.java | 12 ++-- .../spark/api/java/function/PairFunction.java | 12 ++-- .../scala/spark/deploy/JsonProtocol.scala | 2 +- .../scala/spark/deploy/client/Client.scala | 4 +- .../scala/spark/deploy/master/Master.scala | 2 +- .../spark/deploy/master/MasterWebUI.scala | 45 ++++++------ .../scala/spark/deploy/worker/Worker.scala | 2 +- .../spark/deploy/worker/WorkerWebUI.scala | 43 ++++++------ .../spark/network/ConnectionManager.scala | 6 +- .../spark/network/ConnectionManagerTest.scala | 4 +- .../cluster/StandaloneSchedulerBackend.scala | 4 +- .../scala/spark/storage/BlockManager.scala | 6 +- .../spark/storage/BlockManagerMaster.scala | 5 +- .../storage/BlockManagerMasterActor.scala | 4 +- .../src/main/scala/spark/util/AkkaUtils.scala | 37 +++++----- .../test/scala/spark/CacheTrackerSuite.scala | 11 ++- project/SparkBuild.scala | 70 ++++++++++--------- project/build.properties | 2 +- project/plugins.sbt | 6 +- 23 files changed, 155 insertions(+), 151 deletions(-) diff --git a/core/src/main/scala/spark/CacheTracker.scala b/core/src/main/scala/spark/CacheTracker.scala index 04c26b2e40f..a73438208aa 100644 --- a/core/src/main/scala/spark/CacheTracker.scala +++ b/core/src/main/scala/spark/CacheTracker.scala @@ -5,12 +5,12 @@ import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet import akka.actor._ -import akka.dispatch._ +import scala.concurrent.Await import akka.pattern.ask import akka.remote._ -import akka.util.Duration +import scala.concurrent.duration.Duration import akka.util.Timeout -import akka.util.duration._ +import scala.concurrent.duration._ import spark.storage.BlockManager import spark.storage.StorageLevel diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala index 70eb9f702e6..08d2956782b 100644 --- a/core/src/main/scala/spark/MapOutputTracker.scala +++ b/core/src/main/scala/spark/MapOutputTracker.scala @@ -8,12 +8,12 @@ import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet import akka.actor._ -import akka.dispatch._ +import scala.concurrent.Await import akka.pattern.ask import akka.remote._ -import akka.util.Duration +import scala.concurrent.duration.Duration import akka.util.Timeout -import akka.util.duration._ +import scala.concurrent.duration._ import spark.scheduler.MapStatus import spark.storage.BlockManagerId 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 dae8295f213..9f6ab9a5928 100644 --- a/core/src/main/scala/spark/api/java/function/Function.java +++ b/core/src/main/scala/spark/api/java/function/Function.java @@ -1,7 +1,7 @@ package spark.api.java.function; -import scala.reflect.ClassManifest; -import scala.reflect.ClassManifest$; +import scala.reflect.ClassTag; +import scala.reflect.ClassTag$; import scala.runtime.AbstractFunction1; import java.io.Serializable; @@ -15,8 +15,8 @@ public abstract class Function extends WrappedFunction1 implements Serializable { public abstract R call(T t) throws Exception; - public ClassManifest returnType() { - return (ClassManifest) ClassManifest$.MODULE$.fromClass(Object.class); + public ClassTag returnType() { + return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } } 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 69bf12c8c9d..b32c178aa39 100644 --- a/core/src/main/scala/spark/api/java/function/Function2.java +++ b/core/src/main/scala/spark/api/java/function/Function2.java @@ -1,7 +1,7 @@ package spark.api.java.function; -import scala.reflect.ClassManifest; -import scala.reflect.ClassManifest$; +import scala.reflect.ClassTag; +import scala.reflect.ClassTag$; import scala.runtime.AbstractFunction2; import java.io.Serializable; @@ -14,8 +14,9 @@ public abstract class Function2 extends WrappedFunction2 public abstract R call(T1 t1, T2 t2) throws Exception; - public ClassManifest returnType() { - return (ClassManifest) ClassManifest$.MODULE$.fromClass(Object.class); + public ClassTag returnType() { + return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } + } 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 b3cc4df6aa4..296a3b5044a 100644 --- a/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java +++ b/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java @@ -1,8 +1,8 @@ package spark.api.java.function; import scala.Tuple2; -import scala.reflect.ClassManifest; -import scala.reflect.ClassManifest$; +import scala.reflect.ClassTag; +import scala.reflect.ClassTag$; import scala.runtime.AbstractFunction1; import java.io.Serializable; @@ -19,11 +19,11 @@ public abstract class PairFlatMapFunction public abstract Iterable> call(T t) throws Exception; - public ClassManifest keyType() { - return (ClassManifest) ClassManifest$.MODULE$.fromClass(Object.class); + public ClassTag keyType() { + return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } - public ClassManifest valueType() { - return (ClassManifest) ClassManifest$.MODULE$.fromClass(Object.class); + public ClassTag valueType() { + return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } } 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 9fc6df4b888..e3f94788e26 100644 --- a/core/src/main/scala/spark/api/java/function/PairFunction.java +++ b/core/src/main/scala/spark/api/java/function/PairFunction.java @@ -1,8 +1,8 @@ package spark.api.java.function; import scala.Tuple2; -import scala.reflect.ClassManifest; -import scala.reflect.ClassManifest$; +import scala.reflect.ClassTag; +import scala.reflect.ClassTag$; import scala.runtime.AbstractFunction1; import java.io.Serializable; @@ -18,11 +18,11 @@ public abstract class PairFunction public abstract Tuple2 call(T t) throws Exception; - public ClassManifest keyType() { - return (ClassManifest) ClassManifest$.MODULE$.fromClass(Object.class); + public ClassTag keyType() { + return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } - public ClassManifest valueType() { - return (ClassManifest) ClassManifest$.MODULE$.fromClass(Object.class); + public ClassTag valueType() { + return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } } diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index 732fa080645..12069947fc5 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -2,7 +2,7 @@ package spark.deploy import master.{JobInfo, WorkerInfo} import worker.ExecutorRunner -import cc.spray.json._ +import spray.json._ /** * spray-json helper class containing implicit conversion to json for marshalling responses diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index 90fe9508cdb..0aee7597962 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -3,7 +3,7 @@ package spark.deploy.client import spark.deploy._ import akka.actor._ import akka.pattern.ask -import akka.util.duration._ +import scala.concurrent.duration._ import akka.pattern.AskTimeoutException import spark.{SparkException, Logging} import akka.remote.RemoteClientLifeCycleEvent @@ -11,7 +11,7 @@ import akka.remote.RemoteClientShutdown import spark.deploy.RegisterJob import akka.remote.RemoteClientDisconnected import akka.actor.Terminated -import akka.dispatch.Await +import scala.concurrent.Await /** * The main class used to talk to a Spark deploy cluster. Takes a master URL, a job description, diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 6ecebe626a7..e034312c12b 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -49,7 +49,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor } def startWebUi() { - val webUi = new MasterWebUI(context.system, self) + val webUi = new MasterWebUI(self) try { AkkaUtils.startSprayServer(context.system, "0.0.0.0", webUiPort, webUi.handler) } catch { diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 458ee2d6658..a4dadb6ef94 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -1,39 +1,42 @@ package spark.deploy.master -import akka.actor.{ActorRef, ActorSystem} -import akka.dispatch.Await +import akka.actor.{ActorRef, ActorContext, ActorRefFactory} +import scala.concurrent.Await import akka.pattern.ask import akka.util.Timeout -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.concurrent.duration._ +import spray.routing.Directives +import spray.routing.directives._ +import spray.httpx.TwirlSupport._ +import spray.httpx.SprayJsonSupport._ +import spray.http.MediaTypes._ import spark.deploy._ import spark.deploy.JsonProtocol._ private[spark] -class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Directives { +class MasterWebUI(master: ActorRef)(implicit val context: ActorContext) extends Directives { + import context.dispatcher + + val actorSystem = context.system val RESOURCE_DIR = "spark/deploy/master/webui" val STATIC_RESOURCE_DIR = "spark/deploy/static" - + implicit val timeout = Timeout(1 seconds) - + val handler = { get { (path("") & parameters('format ?)) { case Some(js) if js.equalsIgnoreCase("json") => - val future = master ? RequestMasterState - respondWithMediaType(MediaTypes.`application/json`) { ctx => + val future = (master ? RequestMasterState).mapTo[MasterState] + respondWithMediaType(`application/json`) { ctx => ctx.complete(future.mapTo[MasterState]) } case _ => - completeWith { - val future = master ? RequestMasterState + complete { + val future = (master ? RequestMasterState).mapTo[MasterState] future.map { - masterState => spark.deploy.master.html.index.render(masterState.asInstanceOf[MasterState]) + masterState => spark.deploy.master.html.index.render(masterState) } } } ~ @@ -50,15 +53,13 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct } } } - respondWithMediaType(MediaTypes.`application/json`) { ctx => + respondWithMediaType(`application/json`) { ctx => ctx.complete(jobInfo.mapTo[JobInfo]) } case (jobId, _) => - completeWith { - val future = master ? RequestMasterState - future.map { state => - val masterState = state.asInstanceOf[MasterState] - + complete { + val future = (master ? RequestMasterState).mapTo[MasterState] + future.map { masterState => masterState.activeJobs.find(_.id == jobId) match { case Some(job) => spark.deploy.master.html.job_details.render(job) case _ => masterState.completedJobs.find(_.id == jobId) match { diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 7c9e588ea2d..ec25a19e7b9 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -100,7 +100,7 @@ private[spark] class Worker( } def startWebUi() { - val webUi = new WorkerWebUI(context.system, self) + val webUi = new WorkerWebUI(self) try { AkkaUtils.startSprayServer(context.system, "0.0.0.0", webUiPort, webUi.handler) } catch { diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index f9489d99fc1..7dd1781900e 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -1,46 +1,49 @@ package spark.deploy.worker -import akka.actor.{ActorRef, ActorSystem} -import akka.dispatch.Await +import akka.actor.{ActorRef, ActorContext} +import scala.concurrent.Await import akka.pattern.ask import akka.util.Timeout -import akka.util.duration._ -import cc.spray.Directives -import cc.spray.typeconversion.TwirlSupport._ -import cc.spray.http.MediaTypes -import cc.spray.typeconversion.SprayJsonSupport._ +import scala.concurrent.duration._ +import spray.routing.Directives +import spray.httpx.TwirlSupport._ +import spray.httpx.SprayJsonSupport._ +import spray.http.MediaTypes._ import spark.deploy.{WorkerState, RequestWorkerState} import spark.deploy.JsonProtocol._ private[spark] -class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Directives { - val RESOURCE_DIR = "spark/deploy/worker/webui" +class WorkerWebUI(worker: ActorRef)(implicit val context: ActorContext) extends Directives { + import context.dispatcher + + val actorSystem = context.system + val RESOURCE_DIR = "spark/deploy/worker/webui" val STATIC_RESOURCE_DIR = "spark/deploy/static" - + implicit val timeout = Timeout(1 seconds) - + val handler = { get { - (path("") & parameters('format ?)) { + (path("") & parameters('format ?)) { case Some(js) if js.equalsIgnoreCase("json") => { - val future = worker ? RequestWorkerState - respondWithMediaType(MediaTypes.`application/json`) { ctx => - ctx.complete(future.mapTo[WorkerState]) + val future = (worker ? RequestWorkerState).mapTo[WorkerState] + respondWithMediaType(`application/json`) { ctx => + ctx.complete(future) } } case _ => - completeWith{ - val future = worker ? RequestWorkerState + complete { + val future = (worker ? RequestWorkerState).mapTo[WorkerState] future.map { workerState => - spark.deploy.worker.html.index(workerState.asInstanceOf[WorkerState]) + spark.deploy.worker.html.index(workerState) } } } ~ path("log") { parameters("jobId", "executorId", "logType") { (jobId, executorId, logType) => - respondWithMediaType(cc.spray.http.MediaTypes.`text/plain`) { - getFromFileName("work/" + jobId + "/" + executorId + "/" + logType) + respondWithMediaType(`text/plain`) { + getFromFile("work/" + jobId + "/" + executorId + "/" + logType) } } } ~ diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 36c01ad629b..04b303afe07 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -14,9 +14,9 @@ import scala.collection.mutable.SynchronizedQueue import scala.collection.mutable.Queue import scala.collection.mutable.ArrayBuffer -import akka.dispatch.{Await, Promise, ExecutionContext, Future} -import akka.util.Duration -import akka.util.duration._ +import scala.concurrent.{Await, Promise, ExecutionContext, Future} +import scala.concurrent.duration.Duration +import scala.concurrent.duration._ private[spark] case class ConnectionManagerId(host: String, port: Int) { def toSocketAddress() = new InetSocketAddress(host, port) diff --git a/core/src/main/scala/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/spark/network/ConnectionManagerTest.scala index 533e4610f3a..1b5b0935c2f 100644 --- a/core/src/main/scala/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/spark/network/ConnectionManagerTest.scala @@ -8,8 +8,8 @@ import scala.io.Source import java.nio.ByteBuffer import java.net.InetAddress -import akka.dispatch.Await -import akka.util.duration._ +import scala.concurrent.Await +import scala.concurrent.duration._ private[spark] object ConnectionManagerTest extends Logging{ def main(args: Array[String]) { diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index eeaae23dc86..03dc5f4b9b4 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -3,11 +3,11 @@ package spark.scheduler.cluster import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import akka.actor._ -import akka.util.duration._ +import scala.concurrent.duration._ import akka.pattern.ask import spark.{SparkException, Logging, TaskState} -import akka.dispatch.Await +import scala.concurrent.Await import java.util.concurrent.atomic.AtomicInteger import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent} diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 7a8ac10cdd8..7a1344668fb 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -8,9 +8,9 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} import scala.collection.JavaConversions._ import akka.actor.{ActorSystem, Cancellable, Props} -import akka.dispatch.{Await, Future} -import akka.util.Duration -import akka.util.duration._ +import scala.concurrent.{Await, Future} +import scala.concurrent.duration.Duration +import scala.concurrent.duration._ import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index a3d8671834d..46e1860d09f 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -4,10 +4,9 @@ import scala.collection.mutable.ArrayBuffer import scala.util.Random import akka.actor.{Actor, ActorRef, ActorSystem, Props} -import akka.dispatch.Await +import scala.concurrent.Await import akka.pattern.ask -import akka.util.{Duration, Timeout} -import akka.util.duration._ +import scala.concurrent.duration._ import spark.{Logging, SparkException, Utils} diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index f4d026da332..9bf9161c312 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -7,8 +7,7 @@ import scala.collection.JavaConversions._ import scala.util.Random import akka.actor.{Actor, ActorRef, Cancellable} -import akka.util.{Duration, Timeout} -import akka.util.duration._ +import scala.concurrent.duration._ import spark.{Logging, Utils} @@ -42,6 +41,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { override def preStart() { if (!BlockManager.getDisableHeartBeatsForTesting) { + import context.dispatcher timeoutCheckingTask = context.system.scheduler.schedule( 0.seconds, checkTimeoutInterval.milliseconds, self, ExpireDeadHosts) } diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala index e67cb0336d0..6fd9aa70fcf 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/spark/util/AkkaUtils.scala @@ -1,16 +1,16 @@ package spark.util -import akka.actor.{Props, ActorSystemImpl, ActorSystem} +import akka.actor.{Props, ActorSystem, ExtendedActorSystem} import com.typesafe.config.ConfigFactory -import akka.util.duration._ +import scala.concurrent.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 spray.routing.Route +import spray.io.IOExtension +import spray.routing.HttpServiceActor +import spray.can.server.{HttpServer, ServerSettings} +import spray.io.SingletonHandler +import scala.concurrent.Await import spark.SparkException import java.util.concurrent.TimeoutException @@ -23,11 +23,11 @@ private[spark] object AkkaUtils { * ActorSystem itself and its port (which is hard to get from Akka). */ def createActorSystem(name: String, host: String, port: Int): (ActorSystem, Int) = { - val akkaThreads = System.getProperty("spark.akka.threads", "4").toInt + 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", "20").toInt val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt - val akkaConf = ConfigFactory.parseString(""" + val akkaConf = ConfigFactory.parseString(""" akka.daemonic = on akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"] akka.actor.provider = "akka.remote.RemoteActorRefProvider" @@ -44,7 +44,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) } @@ -54,14 +54,13 @@ private[spark] object AkkaUtils { * handle requests. Throws a SparkException if this fails. */ def startSprayServer(actorSystem: ActorSystem, ip: String, port: Int, route: Route) { - 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 = "HttpServer") - actorSystem.registerOnTermination { ioWorker.stop() } + val ioWorker = IOExtension(actorSystem).ioBridge() + val httpService = actorSystem.actorOf(Props(HttpServiceActor(route))) + val server = actorSystem.actorOf( + Props(new HttpServer(ioWorker, SingletonHandler(httpService), ServerSettings())), name = "HttpServer") + actorSystem.registerOnTermination { actorSystem.stop(ioWorker) } val timeout = 3.seconds - val future = server.ask(HttpServer.Bind(ip, port))(timeout) + val future = server.ask(HttpServer.Bind(ip, port))(timeout) try { Await.result(future, timeout) match { case bound: HttpServer.Bound => diff --git a/core/src/test/scala/spark/CacheTrackerSuite.scala b/core/src/test/scala/spark/CacheTrackerSuite.scala index 467605981b3..d0c2bd47fc0 100644 --- a/core/src/test/scala/spark/CacheTrackerSuite.scala +++ b/core/src/test/scala/spark/CacheTrackerSuite.scala @@ -5,20 +5,19 @@ import org.scalatest.FunSuite import scala.collection.mutable.HashMap import akka.actor._ -import akka.dispatch._ -import akka.pattern.ask +import scala.concurrent.{Await, Future} import akka.remote._ -import akka.util.Duration +import scala.concurrent.duration.Duration import akka.util.Timeout -import akka.util.duration._ +import scala.concurrent.duration._ class CacheTrackerSuite extends FunSuite { // Send a message to an actor and wait for a reply, in a blocking manner private def ask(actor: ActorRef, message: Any): Any = { try { val timeout = 10.seconds - val future = actor.ask(message)(timeout) - return Await.result(future, timeout) + val future: Future[Any] = akka.pattern.ask(actor, message)(timeout) + Await.result(future, timeout) } catch { case e: Exception => throw new SparkException("Error communicating with actor", e) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 219674028eb..d0b3c350f12 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -17,11 +17,11 @@ object SparkBuild extends Build { //val HADOOP_VERSION = "2.0.0-mr1-cdh4.1.1" //val HADOOP_MAJOR_VERSION = "2" - lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel) + lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, /*repl,*/ examples, bagel) lazy val core = Project("core", file("core"), settings = coreSettings) - lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) +// lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) @@ -32,10 +32,10 @@ object SparkBuild extends Build { lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") def sharedSettings = Defaults.defaultSettings ++ Seq( - organization := "org.spark-project", - version := "0.7.0-SNAPSHOT", - scalaVersion := "2.9.2", - scalacOptions := Seq(/*"-deprecation",*/ "-unchecked", "-optimize"), // -deprecation is too noisy due to usage of old Hadoop API, enable it once that's no longer an issue + organization := "org.spark-project", + version := "0.7.0-SNAPSHOT", + scalaVersion := "2.10.0", + scalacOptions := Seq(/*"-deprecation",*/ "-unchecked", "-optimize"), // -deprecation is too noisy due to usage of old Hadoop API, enable it once that's no longer an issue unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", @@ -87,11 +87,11 @@ object SparkBuild extends Build { */ libraryDependencies ++= Seq( - "org.eclipse.jetty" % "jetty-server" % "7.5.3.v20111011", - "org.scalatest" %% "scalatest" % "1.8" % "test", - "org.scalacheck" %% "scalacheck" % "1.9" % "test", - "com.novocode" % "junit-interface" % "0.8" % "test" - ), + "org.eclipse.jetty" % "jetty-server" % "7.5.3.v20111011", + "org.scalatest" %% "scalatest" % "1.9.1" % "test", + "org.scalacheck" %% "scalacheck" % "1.10.0" % "test", + "com.novocode" % "junit-interface" % "0.8" % "test" + ), parallelExecution := false, /* Workaround for issue #206 (fixed after SBT 0.11.0) */ watchTransitiveSources <<= Defaults.inDependencies[Task[Seq[File]]](watchSources.task, @@ -112,31 +112,33 @@ object SparkBuild extends Build { name := "spark-core", resolvers ++= Seq( "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/", - "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", - "Spray Repository" at "http://repo.spray.cc/", + "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/" ), libraryDependencies ++= Seq( - "com.google.guava" % "guava" % "11.0.1", - "log4j" % "log4j" % "1.2.16", - "org.slf4j" % "slf4j-api" % slf4jVersion, - "org.slf4j" % "slf4j-log4j12" % slf4jVersion, - "com.ning" % "compress-lzf" % "0.8.4", - "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION, - "asm" % "asm-all" % "3.3.1", - "com.google.protobuf" % "protobuf-java" % "2.4.1", - "de.javakaffee" % "kryo-serializers" % "0.20", - "com.typesafe.akka" % "akka-actor" % "2.0.3", - "com.typesafe.akka" % "akka-remote" % "2.0.3", - "com.typesafe.akka" % "akka-slf4j" % "2.0.3", - "it.unimi.dsi" % "fastutil" % "6.4.4", - "colt" % "colt" % "1.2.0", - "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" - ) ++ (if (HADOOP_MAJOR_VERSION == "2") Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION) else None).toSeq, + "com.google.guava" % "guava" % "11.0.1", + "log4j" % "log4j" % "1.2.16", + "org.slf4j" % "slf4j-api" % slf4jVersion, + "org.slf4j" % "slf4j-log4j12" % slf4jVersion, + "com.ning" % "compress-lzf" % "0.8.4", + "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION, + "asm" % "asm-all" % "3.3.1", + "com.google.protobuf" % "protobuf-java" % "2.4.1", + "de.javakaffee" % "kryo-serializers" % "0.20", + "com.typesafe.akka" %% "akka-remote" % "2.1.0", + "com.typesafe.akka" %% "akka-slf4j" % "2.1.0", + "it.unimi.dsi" % "fastutil" % "6.4.4", + "io.spray" % "spray-can" % "1.1-M7", + "io.spray" % "spray-io" % "1.1-M7", + "io.spray" % "spray-routing" % "1.1-M7", + "io.spray" %% "spray-json" % "1.2.3", + "colt" % "colt" % "1.2.0", + "org.apache.mesos" % "mesos" % "0.9.0-incubating", + "org.scala-lang" % "scala-actors" % "2.10.0" + ) ++ (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 @@ -144,10 +146,10 @@ object SparkBuild extends Build { publish := {} ) - def replSettings = sharedSettings ++ Seq( +/* def replSettings = sharedSettings ++ Seq( name := "spark-repl", libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _) - ) + )*/ def examplesSettings = sharedSettings ++ Seq( name := "spark-examples" diff --git a/project/build.properties b/project/build.properties index d4287112c6a..4474a03e1aa 100644 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1 @@ -sbt.version=0.11.3 +sbt.version=0.12.1 diff --git a/project/plugins.sbt b/project/plugins.sbt index 4d0e696a113..0e4eb7085c0 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -6,11 +6,11 @@ resolvers += "Spray Repository" at "http://repo.spray.cc/" addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.3") -addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.0-RC1") +addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.1") -addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.0.0") +addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.2.0") -addSbtPlugin("cc.spray" %% "sbt-twirl" % "0.5.2") +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) From a5403acd4ead9ecabfa9f2ed6d6d58afe912a238 Mon Sep 17 00:00:00 2001 From: folone Date: Sun, 20 Jan 2013 14:42:16 +0100 Subject: [PATCH 0002/2521] Updated maven build for scala 2.10. --- core/pom.xml | 20 ++++++++++-------- pom.xml | 57 ++++++++++++++++++++++++++++------------------------ 2 files changed, 43 insertions(+), 34 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 862d3ec37ac..6316b28a7be 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -44,16 +44,16 @@ kryo-serializers - com.typesafe.akka - akka-actor + org.scala-lang + scala-actors com.typesafe.akka - akka-remote + akka-remote_${scala.version} com.typesafe.akka - akka-slf4j + akka-slf4j_${scala.version} it.unimi.dsi @@ -64,15 +64,19 @@ colt - cc.spray + io.spray spray-can - cc.spray - spray-server + io.spray + spray-routing + + + io.spray + spray-io - cc.spray + io.spray spray-json_${scala.version} diff --git a/pom.xml b/pom.xml index 751189a9d87..756fe8783bd 100644 --- a/pom.xml +++ b/pom.xml @@ -41,8 +41,8 @@ core bagel examples - repl - repl-bin + @@ -50,20 +50,20 @@ UTF-8 1.5 - 2.9.2 + 2.10 0.9.0-incubating - 2.0.3 - 1.0-M2.1 - 1.1.1 + 2.1.0 + 1.1-M7 + 1.2.3 1.6.1 4.1.2 - jboss-repo - JBoss Repository - http://repository.jboss.org/nexus/content/repositories/releases/ + typesafe-repo + Typesafe Repository + http://repo.typesafe.com/typesafe/releases/ true @@ -72,9 +72,9 @@ - cloudera-repo - Cloudera Repository - https://repository.cloudera.com/artifactory/cloudera-repos/ + jboss-repo + JBoss Repository + http://repository.jboss.org/nexus/content/repositories/releases/ true @@ -83,9 +83,9 @@ - typesafe-repo - Typesafe Repository - http://repo.typesafe.com/typesafe/releases/ + cloudera-repo + Cloudera Repository + https://repository.cloudera.com/artifactory/cloudera-repos/ true @@ -189,18 +189,18 @@ 0.20 - com.typesafe.akka - akka-actor - ${akka.version} + org.scala-lang + scala-actors + 2.10.0 com.typesafe.akka - akka-remote + akka-remote_${scala.version} ${akka.version} com.typesafe.akka - akka-slf4j + akka-slf4j_${scala.version} ${akka.version} @@ -214,17 +214,22 @@ 1.2.0 - cc.spray + io.spray spray-can ${spray.version} - cc.spray - spray-server + io.spray + spray-routing + ${spray.version} + + + io.spray + spray-io ${spray.version} - cc.spray + io.spray spray-json_${scala.version} ${spray.json.version} @@ -258,13 +263,13 @@ org.scalatest scalatest_${scala.version} - 1.8 + 1.9.1 test org.scalacheck scalacheck_${scala.version} - 1.9 + 1.10.0 test From fd6e51deec83f01be3db41e84255329eedbe15da Mon Sep 17 00:00:00 2001 From: folone Date: Sun, 20 Jan 2013 17:02:58 +0100 Subject: [PATCH 0003/2521] Fixed the failing test. --- .../test/scala/spark/AccumulatorSuite.scala | 33 +++++++++---------- 1 file changed, 15 insertions(+), 18 deletions(-) diff --git a/core/src/test/scala/spark/AccumulatorSuite.scala b/core/src/test/scala/spark/AccumulatorSuite.scala index d8be99dde71..9f5335978f3 100644 --- a/core/src/test/scala/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/spark/AccumulatorSuite.scala @@ -13,6 +13,20 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter var sc: SparkContext = null + implicit def setAccum[A] = new AccumulableParam[mutable.Set[A], A] { + def addInPlace(t1: mutable.Set[A], t2: mutable.Set[A]) : mutable.Set[A] = { + t1 ++= t2 + t1 + } + def addAccumulator(t1: mutable.Set[A], t2: A) : mutable.Set[A] = { + t1 += t2 + t1 + } + def zero(t: mutable.Set[A]) : mutable.Set[A] = { + new mutable.HashSet[A]() + } + } + after { if (sc != null) { sc.stop() @@ -40,7 +54,6 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } test ("add value to collection accumulators") { - import SetAccum._ val maxI = 1000 for (nThreads <- List(1, 10)) { //test single & multi-threaded sc = new SparkContext("local[" + nThreads + "]", "test") @@ -60,22 +73,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } } - implicit object SetAccum extends AccumulableParam[mutable.Set[Any], Any] { - def addInPlace(t1: mutable.Set[Any], t2: mutable.Set[Any]) : mutable.Set[Any] = { - t1 ++= t2 - t1 - } - def addAccumulator(t1: mutable.Set[Any], t2: Any) : mutable.Set[Any] = { - t1 += t2 - t1 - } - def zero(t: mutable.Set[Any]) : mutable.Set[Any] = { - new mutable.HashSet[Any]() - } - } - test ("value not readable in tasks") { - import SetAccum._ val maxI = 1000 for (nThreads <- List(1, 10)) { //test single & multi-threaded sc = new SparkContext("local[" + nThreads + "]", "test") @@ -123,7 +121,6 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } test ("localValue readable in tasks") { - import SetAccum._ val maxI = 1000 for (nThreads <- List(1, 10)) { //test single & multi-threaded sc = new SparkContext("local[" + nThreads + "]", "test") @@ -135,7 +132,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } acc.value should be ( (0 to maxI).toSet) sc.stop() - sc = null + sc = null } } From 81c4d19c612208b932f8579427cb895385336c6e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 19 Feb 2013 12:43:13 -0800 Subject: [PATCH 0004/2521] Maven and sbt build changes for SparkGraph. --- graph/pom.xml | 106 ++++++++++++++++++ .../src/main/scala/spark/graph/package.scala | 7 ++ pom.xml | 1 + project/SparkBuild.scala | 6 +- run | 2 + run2.cmd | 2 + 6 files changed, 123 insertions(+), 1 deletion(-) create mode 100644 graph/pom.xml create mode 100644 graph/src/main/scala/spark/graph/package.scala diff --git a/graph/pom.xml b/graph/pom.xml new file mode 100644 index 00000000000..1cd9cda98bc --- /dev/null +++ b/graph/pom.xml @@ -0,0 +1,106 @@ + + + 4.0.0 + + org.spark-project + parent + 0.7.0-SNAPSHOT + ../pom.xml + + + org.spark-project + spark-graph + jar + Spark Graph + http://spark-project.org/ + + + + org.eclipse.jetty + jetty-server + + + + org.scalatest + scalatest_${scala.version} + test + + + org.scalacheck + scalacheck_${scala.version} + test + + + + target/scala-${scala.version}/classes + target/scala-${scala.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + + + + + hadoop1 + + + org.spark-project + spark-core + ${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.apache.hadoop + hadoop-core + provided + + + org.apache.hadoop + hadoop-client + provided + + + + + + org.apache.maven.plugins + maven-jar-plugin + + hadoop2 + + + + + + + diff --git a/graph/src/main/scala/spark/graph/package.scala b/graph/src/main/scala/spark/graph/package.scala new file mode 100644 index 00000000000..35b9f244613 --- /dev/null +++ b/graph/src/main/scala/spark/graph/package.scala @@ -0,0 +1,7 @@ +package spark + +package object graph { + type Vid = Int + type Pid = Int + type Status = Boolean +} diff --git a/pom.xml b/pom.xml index 3ea989a082b..11a1f87ed37 100644 --- a/pom.xml +++ b/pom.xml @@ -41,6 +41,7 @@ core bagel examples + graph streaming repl repl-bin diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 03b8094f7dc..3e383b1300b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -17,7 +17,7 @@ object SparkBuild extends Build { //val HADOOP_VERSION = "2.0.0-mr1-cdh4.1.1" //val HADOOP_MAJOR_VERSION = "2" - 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, graph, streaming) lazy val core = Project("core", file("core"), settings = coreSettings) @@ -27,6 +27,8 @@ object SparkBuild extends Build { lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn (core) + lazy val graph = Project("graph", file("graph"), settings = graphSettings) dependsOn (core) + lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn (core) // A configuration to set an alternative publishLocalConfiguration @@ -161,6 +163,8 @@ object SparkBuild extends Build { def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel") + def graphSettings = sharedSettings ++ Seq(name := "spark-graph") + def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", libraryDependencies ++= Seq( diff --git a/run b/run index a0946294497..4ecaa705117 100755 --- a/run +++ b/run @@ -63,6 +63,7 @@ CORE_DIR="$FWDIR/core" REPL_DIR="$FWDIR/repl" EXAMPLES_DIR="$FWDIR/examples" BAGEL_DIR="$FWDIR/bagel" +GRAPH_DIR="$FWDIR/graph" STREAMING_DIR="$FWDIR/streaming" PYSPARK_DIR="$FWDIR/python" @@ -95,6 +96,7 @@ if [ -e repl-bin/target ]; then done fi CLASSPATH+=":$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" +CLASSPATH+=":$GRAPH_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do CLASSPATH+=":$jar" done diff --git a/run2.cmd b/run2.cmd index 67f1e465e47..eb5e283e9f4 100644 --- a/run2.cmd +++ b/run2.cmd @@ -34,6 +34,7 @@ set CORE_DIR=%FWDIR%core set REPL_DIR=%FWDIR%repl set EXAMPLES_DIR=%FWDIR%examples set BAGEL_DIR=%FWDIR%bagel +set GRAPH_DIR=%FWDIR%graph set PYSPARK_DIR=%FWDIR%python rem Build up classpath @@ -45,6 +46,7 @@ for /R "%FWDIR%\lib_managed\bundles" %%j in (*.jar) do set CLASSPATH=!CLASSPATH! for /R "%REPL_DIR%\lib" %%j in (*.jar) do set CLASSPATH=!CLASSPATH!;%%j for /R "%PYSPARK_DIR%\lib" %%j in (*.jar) do set CLASSPATH=!CLASSPATH!;%%j set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes +set CLASSPATH=%CLASSPATH%;%GRAPH_DIR%\target\scala-%SCALA_VERSION%\classes rem Figure out whether to run our class with java or with the scala launcher. rem In most cases, we'd prefer to execute our process with java because scala From 85a35c68401e171df0b72b172a689d8c4e412199 Mon Sep 17 00:00:00 2001 From: Christoph Grothaus Date: Fri, 15 Feb 2013 14:11:34 +0100 Subject: [PATCH 0005/2521] Fix SPARK-698. From ExecutorRunner, launch java directly instead via the run scripts. --- .../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 de11771c8e6..214c44fc888 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 82b1da005a4..b5f693f1fae 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 c913a5195ef..a93bbad0b99 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 From f39f2b7636f52568a556987c8b7f7393299b0351 Mon Sep 17 00:00:00 2001 From: Christoph Grothaus Date: Sun, 24 Feb 2013 21:24:30 +0100 Subject: [PATCH 0006/2521] 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 --- .../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 214c44fc888..38216ce62f5 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 b5f693f1fae..e1482dafbe7 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 a93bbad0b99..8648c0380a3 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! From 4aa1205202f26663f59347f25a7d1f03c755545d Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 11 Mar 2013 12:37:29 -0600 Subject: [PATCH 0007/2521] 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 b0b6e21681b..44c8058e9d6 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", From c1c3682c984c83f75352fc22dcadd3e46058cfaf Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 11 Mar 2013 12:40:44 -0600 Subject: [PATCH 0008/2521] adding checkpoint dir to .gitignore --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index 155e785b01b..6c9ffa54265 100644 --- a/.gitignore +++ b/.gitignore @@ -36,3 +36,4 @@ streaming-tests.log dependency-reduced-pom.xml .ensime .ensime_lucene +checkpoint From 42822cf95de71039988e22d8690ba6a4bd639227 Mon Sep 17 00:00:00 2001 From: seanm Date: Wed, 13 Mar 2013 11:40:42 -0600 Subject: [PATCH 0009/2521] 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 44c8058e9d6..7e65979a5d6 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", From cfa8e769a86664722f47182fa572179e8beadcb7 Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 11 Mar 2013 17:16:15 -0600 Subject: [PATCH 0010/2521] KafkaInputDStream improvements. Allows more Kafka configurability --- .../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 25c67b279b7..4e1732adf56 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 dc7139cc273..f769fc1cc35 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 + } + } } From d06928321194b11e082986cd2bb2737d9bc3b698 Mon Sep 17 00:00:00 2001 From: seanm Date: Thu, 14 Mar 2013 23:25:35 -0600 Subject: [PATCH 0011/2521] fixing memory leak in kafka MessageHandler --- .../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 f769fc1cc35..d674b6ee87a 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 } } } From 33fa1e7e4aca4d9e0edf65d2b768b569305fd044 Mon Sep 17 00:00:00 2001 From: seanm Date: Thu, 14 Mar 2013 23:32:52 -0600 Subject: [PATCH 0012/2521] removing dependency on ZookeeperConsumerConnector + purging last relic of kafka reliability that never solidified (ie- setOffsets) --- .../spark/streaming/StreamingContext.scala | 9 ++---- .../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 4e1732adf56..bb7f216ca7d 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 f3b40b5b88e..2373f4824a8 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 d674b6ee87a..c6da1a7f70b 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() { From d61978d0abad30a148680c8a63df33e40e469525 Mon Sep 17 00:00:00 2001 From: seanm Date: Fri, 15 Mar 2013 23:36:52 -0600 Subject: [PATCH 0013/2521] keeping JavaStreamingContext in sync with StreamingContext + adding comments for better clarity --- .../spark/streaming/api/java/JavaStreamingContext.scala | 7 +++---- .../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 2373f4824a8..7a8864614c9 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 c6da1a7f70b..85693808d10 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 From 15530c2b23d1f3871a51fd24d14ce11b8ffaff4a Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sun, 17 Mar 2013 10:47:17 +0530 Subject: [PATCH 0014/2521] porting of repl to scala-2.10 --- project/SparkBuild.scala | 17 +- project/plugins.sbt | 4 +- repl/src/main/scala/spark/repl/Main.scala | 8 +- .../scala/spark/repl/SparkExprTyper.scala | 109 ++ .../main/scala/spark/repl/SparkILoop.scala | 908 +++++---- .../scala/spark/repl/SparkILoopInit.scala | 142 ++ .../main/scala/spark/repl/SparkIMain.scala | 1675 +++++++++-------- .../scala/spark/repl/SparkISettings.scala | 63 - .../main/scala/spark/repl/SparkImports.scala | 113 +- .../spark/repl/SparkJLineCompletion.scala | 207 +- .../scala/spark/repl/SparkJLineReader.scala | 65 +- .../spark/repl/SparkMemberHandlers.scala | 112 +- run | 2 +- 13 files changed, 1908 insertions(+), 1517 deletions(-) create mode 100644 repl/src/main/scala/spark/repl/SparkExprTyper.scala create mode 100644 repl/src/main/scala/spark/repl/SparkILoopInit.scala delete mode 100644 repl/src/main/scala/spark/repl/SparkISettings.scala diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d0b3c350f12..9fcdbb5b92e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -17,11 +17,11 @@ object SparkBuild extends Build { //val HADOOP_VERSION = "2.0.0-mr1-cdh4.1.1" //val HADOOP_MAJOR_VERSION = "2" - lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, /*repl,*/ examples, bagel) + lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel) lazy val core = Project("core", file("core"), settings = coreSettings) -// lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) + lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) @@ -35,7 +35,7 @@ object SparkBuild extends Build { organization := "org.spark-project", version := "0.7.0-SNAPSHOT", scalaVersion := "2.10.0", - scalacOptions := Seq(/*"-deprecation",*/ "-unchecked", "-optimize"), // -deprecation is too noisy due to usage of old Hadoop API, enable it once that's no longer an issue + scalacOptions := Seq("-unchecked", "-optimize"), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", @@ -136,7 +136,9 @@ object SparkBuild extends Build { "io.spray" %% "spray-json" % "1.2.3", "colt" % "colt" % "1.2.0", "org.apache.mesos" % "mesos" % "0.9.0-incubating", - "org.scala-lang" % "scala-actors" % "2.10.0" + "org.scala-lang" % "scala-actors" % "2.10.0", + "org.scala-lang" % "jline" % "2.10.0", + "org.scala-lang" % "scala-reflect" % "2.10.0" ) ++ (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") } @@ -146,10 +148,11 @@ object SparkBuild extends Build { publish := {} ) -/* def replSettings = sharedSettings ++ Seq( + def replSettings = sharedSettings ++ Seq( name := "spark-repl", - libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _) - )*/ + // libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _) + libraryDependencies ++= Seq("org.scala-lang" % "scala-compiler" % "2.10.0") + ) def examplesSettings = sharedSettings ++ Seq( name := "spark-examples" diff --git a/project/plugins.sbt b/project/plugins.sbt index 0e4eb7085c0..4bb4a06a505 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -6,9 +6,9 @@ resolvers += "Spray Repository" at "http://repo.spray.cc/" addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.3") -addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.1") +// addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.1") -addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.2.0") +// addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.2.0") addSbtPlugin("io.spray" %% "sbt-twirl" % "0.6.1") diff --git a/repl/src/main/scala/spark/repl/Main.scala b/repl/src/main/scala/spark/repl/Main.scala index 58809ab646a..b0a78e0bb81 100644 --- a/repl/src/main/scala/spark/repl/Main.scala +++ b/repl/src/main/scala/spark/repl/Main.scala @@ -3,12 +3,12 @@ package spark.repl import scala.collection.mutable.Set object Main { - private var _interp: SparkILoop = null - + private var _interp: SparkILoop = _ + def interp = _interp - + def interp_=(i: SparkILoop) { _interp = i } - + def main(args: Array[String]) { _interp = new SparkILoop _interp.process(args) diff --git a/repl/src/main/scala/spark/repl/SparkExprTyper.scala b/repl/src/main/scala/spark/repl/SparkExprTyper.scala new file mode 100644 index 00000000000..c55797eb54e --- /dev/null +++ b/repl/src/main/scala/spark/repl/SparkExprTyper.scala @@ -0,0 +1,109 @@ +/* NSC -- new Scala compiler + * Copyright 2005-2013 LAMP/EPFL + * @author Paul Phillips + */ + +package spark.repl + +import scala.tools.nsc._ +import scala.tools.nsc.interpreter._ + +import scala.reflect.internal.util.BatchSourceFile +import scala.tools.nsc.ast.parser.Tokens.EOF + +import spark.Logging + +trait SparkExprTyper extends Logging { + val repl: SparkIMain + + import repl._ + import global.{ reporter => _, Import => _, _ } + import definitions._ + import syntaxAnalyzer.{ UnitParser, UnitScanner, token2name } + import naming.freshInternalVarName + + object codeParser extends { val global: repl.global.type = repl.global } with CodeHandlers[Tree] { + def applyRule[T](code: String, rule: UnitParser => T): T = { + reporter.reset() + val scanner = newUnitParser(code) + val result = rule(scanner) + + if (!reporter.hasErrors) + scanner.accept(EOF) + + result + } + + def defns(code: String) = stmts(code) collect { case x: DefTree => x } + def expr(code: String) = applyRule(code, _.expr()) + def stmts(code: String) = applyRule(code, _.templateStats()) + def stmt(code: String) = stmts(code).last // guaranteed nonempty + } + + /** Parse a line into a sequence of trees. Returns None if the input is incomplete. */ + def parse(line: String): Option[List[Tree]] = debugging(s"""parse("$line")""") { + var isIncomplete = false + reporter.withIncompleteHandler((_, _) => isIncomplete = true) { + val trees = codeParser.stmts(line) + if (reporter.hasErrors) Some(Nil) + else if (isIncomplete) None + else Some(trees) + } + } + // def parsesAsExpr(line: String) = { + // import codeParser._ + // (opt expr line).isDefined + // } + + def symbolOfLine(code: String): Symbol = { + def asExpr(): Symbol = { + val name = freshInternalVarName() + // Typing it with a lazy val would give us the right type, but runs + // into compiler bugs with things like existentials, so we compile it + // behind a def and strip the NullaryMethodType which wraps the expr. + val line = "def " + name + " = {\n" + code + "\n}" + + interpretSynthetic(line) match { + case IR.Success => + val sym0 = symbolOfTerm(name) + // drop NullaryMethodType + val sym = sym0.cloneSymbol setInfo afterTyper(sym0.info.finalResultType) + if (sym.info.typeSymbol eq UnitClass) NoSymbol + else sym + case _ => NoSymbol + } + } + def asDefn(): Symbol = { + val old = repl.definedSymbolList.toSet + + interpretSynthetic(code) match { + case IR.Success => + repl.definedSymbolList filterNot old match { + case Nil => NoSymbol + case sym :: Nil => sym + case syms => NoSymbol.newOverloaded(NoPrefix, syms) + } + case _ => NoSymbol + } + } + beQuietDuring(asExpr()) orElse beQuietDuring(asDefn()) + } + + private var typeOfExpressionDepth = 0 + def typeOfExpression(expr: String, silent: Boolean = true): Type = { + if (typeOfExpressionDepth > 2) { + logDebug("Terminating typeOfExpression recursion for expression: " + expr) + return NoType + } + typeOfExpressionDepth += 1 + // Don't presently have a good way to suppress undesirable success output + // while letting errors through, so it is first trying it silently: if there + // is an error, and errors are desired, then it re-evaluates non-silently + // to induce the error message. + try beSilentDuring(symbolOfLine(expr).tpe) match { + case NoType if !silent => symbolOfLine(expr).tpe // generate error + case tpe => tpe + } + finally typeOfExpressionDepth -= 1 + } +} diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/spark/repl/SparkILoop.scala index 22bcb4be8a8..2f2b5b23726 100644 --- a/repl/src/main/scala/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/spark/repl/SparkILoop.scala @@ -1,26 +1,38 @@ /* NSC -- new Scala compiler - * Copyright 2005-2011 LAMP/EPFL + * Copyright 2005-2013 LAMP/EPFL * @author Alexander Spoon */ package spark.repl + import scala.tools.nsc._ import scala.tools.nsc.interpreter._ +import scala.tools.nsc.interpreter.{ Results => IR } import Predef.{ println => _, _ } -import java.io.{ BufferedReader, FileReader, PrintWriter } +import java.io.{ BufferedReader, FileReader } +import java.util.concurrent.locks.ReentrantLock import scala.sys.process.Process -import session._ -import scala.tools.nsc.interpreter.{ Results => IR } -import scala.tools.util.{ SignalManager, Signallable, Javap } +import scala.tools.nsc.interpreter.session._ +import scala.util.Properties.{ jdkHome, javaVersion } +import scala.tools.util.{ Javap } import scala.annotation.tailrec -import scala.util.control.Exception.{ ignoring } import scala.collection.mutable.ListBuffer import scala.concurrent.ops -import util.{ ClassPath, Exceptional, stringFromWriter, stringFromStream } -import interpreter._ -import io.{ File, Sources } +import scala.tools.nsc.util.{ ClassPath, Exceptional, stringFromWriter, stringFromStream } +import scala.tools.nsc.interpreter._ +import scala.tools.nsc.io.{ File, Directory } +import scala.reflect.NameTransformer._ +import scala.tools.nsc.util.ScalaClassLoader +import scala.tools.nsc.util.ScalaClassLoader._ +import scala.tools.util._ +import scala.language.{implicitConversions, existentials} +import scala.reflect.{ClassTag, classTag} +import scala.tools.reflect.StdRuntimeTags._ +import scala.reflect.{ClassTag, classTag} +import java.lang.{Class => jClass} +import scala.reflect.api.{Mirror, TypeCreator, Universe => ApiUniverse} import spark.Logging import spark.SparkContext @@ -37,45 +49,85 @@ import spark.SparkContext * @author Lex Spoon * @version 1.2 */ -class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: Option[String]) +class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, + val master: Option[String]) extends AnyRef with LoopCommands + with SparkILoopInit with Logging { - def this(in0: BufferedReader, out: PrintWriter, master: String) = this(Some(in0), out, Some(master)) - def this(in0: BufferedReader, out: PrintWriter) = this(Some(in0), out, None) - def this() = this(None, new PrintWriter(Console.out, true), None) - + def this(in0: BufferedReader, out: JPrintWriter, master: String) = this(Some(in0), out, Some(master)) + def this(in0: BufferedReader, out: JPrintWriter) = this(Some(in0), out, None) + def this() = this(None, new JPrintWriter(Console.out, true), None) + var in: InteractiveReader = _ // the input stream from which commands come var settings: Settings = _ var intp: SparkIMain = _ - /* - lazy val power = { - val g = intp.global - Power[g.type](this, g) - } - */ - - // TODO - // object opt extends AestheticSettings - // - @deprecated("Use `intp` instead.", "2.9.0") - def interpreter = intp - - @deprecated("Use `intp` instead.", "2.9.0") - def interpreter_= (i: SparkIMain): Unit = intp = i - + @deprecated("Use `intp` instead.", "2.9.0") def interpreter = intp + @deprecated("Use `intp` instead.", "2.9.0") def interpreter_= (i: SparkIMain): Unit = intp = i + + /** Having inherited the difficult "var-ness" of the repl instance, + * I'm trying to work around it by moving operations into a class from + * which it will appear a stable prefix. + */ + private def onIntp[T](f: SparkIMain => T): T = f(intp) + + class IMainOps[T <: SparkIMain](val intp: T) { + import intp._ + import global._ + + def printAfterTyper(msg: => String) = + intp.reporter printMessage afterTyper(msg) + + /** Strip NullaryMethodType artifacts. */ + private def replInfo(sym: Symbol) = { + sym.info match { + case NullaryMethodType(restpe) if sym.isAccessor => restpe + case info => info + } + } + def echoTypeStructure(sym: Symbol) = + printAfterTyper("" + deconstruct.show(replInfo(sym))) + + def echoTypeSignature(sym: Symbol, verbose: Boolean) = { + if (verbose) SparkILoop.this.echo("// Type signature") + printAfterTyper("" + replInfo(sym)) + + if (verbose) { + SparkILoop.this.echo("\n// Internal Type structure") + echoTypeStructure(sym) + } + } + } + implicit def stabilizeIMain(intp: SparkIMain) = new IMainOps[intp.type](intp) + + /** TODO - + * -n normalize + * -l label with case class parameter names + * -c complete - leave nothing out + */ + private def typeCommandInternal(expr: String, verbose: Boolean): Result = { + onIntp { intp => + val sym = intp.symbolOfLine(expr) + if (sym.exists) intp.echoTypeSignature(sym, verbose) + else "" + } + } + + var sparkContext: SparkContext = _ + + override def echoCommandMessage(msg: String) { + intp.reporter printMessage msg + } + + def isAsync = !settings.Yreplsync.value + // lazy val power = new Power(intp, new StdReplVals(this))(tagOfStdReplVals, classTag[StdReplVals]) def history = in.history /** The context class loader at the time this object was created */ protected val originalClassLoader = Thread.currentThread.getContextClassLoader - // Install a signal handler so we can be prodded. - private val signallable = - /*if (isReplDebug) Signallable("Dump repl state.")(dumpCommand()) - else*/ null - // classpath entries added via :cp var addedClasspath: String = "" @@ -87,74 +139,41 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: /** Record a command for replay should the user request a :replay */ def addReplay(cmd: String) = replayCommandStack ::= cmd - - /** Try to install sigint handler: ignore failure. Signal handler - * will interrupt current line execution if any is in progress. - * - * Attempting to protect the repl from accidental exit, we only honor - * a single ctrl-C if the current buffer is empty: otherwise we look - * for a second one within a short time. - */ - private def installSigIntHandler() { - def onExit() { - Console.println("") // avoiding "shell prompt in middle of line" syndrome - sys.exit(1) - } - ignoring(classOf[Exception]) { - SignalManager("INT") = { - if (intp == null) - onExit() - else if (intp.lineManager.running) - intp.lineManager.cancel() - else if (in.currentLine != "") { - // non-empty buffer, so make them hit ctrl-C a second time - SignalManager("INT") = onExit() - io.timer(5)(installSigIntHandler()) // and restore original handler if they don't - } - else onExit() - } - } + + def savingReplayStack[T](body: => T): T = { + val saved = replayCommandStack + try body + finally replayCommandStack = saved + } + def savingReader[T](body: => T): T = { + val saved = in + try body + finally in = saved } /** Close the interpreter and set the var to null. */ def closeInterpreter() { if (intp ne null) { - intp.close + intp.close() intp = null - Thread.currentThread.setContextClassLoader(originalClassLoader) } } - + class SparkILoopInterpreter extends SparkIMain(settings, out) { + outer => + override lazy val formatting = new Formatting { def prompt = SparkILoop.this.prompt } - override protected def createLineManager() = new Line.Manager { - override def onRunaway(line: Line[_]): Unit = { - val template = """ - |// She's gone rogue, captain! Have to take her out! - |// Calling Thread.stop on runaway %s with offending code: - |// scala> %s""".stripMargin - - echo(template.format(line.thread, line.code)) - // XXX no way to suppress the deprecation warning - line.thread.stop() - in.redrawLine() - } - } - override protected def parentClassLoader = { - SparkHelper.explicitParentLoader(settings).getOrElse( classOf[SparkILoop].getClassLoader ) - } + override protected def parentClassLoader = SparkHelper.explicitParentLoader(settings).getOrElse(classOf[SparkILoop].getClassLoader) } /** Create a new interpreter. */ def createInterpreter() { if (addedClasspath != "") settings.classpath append addedClasspath - + intp = new SparkILoopInterpreter - intp.setContextClassLoader() - installSigIntHandler() } /** print a friendly help message */ @@ -168,10 +187,10 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: private def helpSummary() = { val usageWidth = commands map (_.usageMsg.length) max val formatStr = "%-" + usageWidth + "s %s %s" - + echo("All commands can be abbreviated, e.g. :he instead of :help.") echo("Those marked with a * have more detailed help, e.g. :help imports.\n") - + commands foreach { cmd => val star = if (cmd.hasLongHelp) "*" else " " echo(formatStr.format(cmd.usageMsg, star, cmd.help)) @@ -182,7 +201,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: case Nil => echo(cmd + ": no such command. Type :help for help.") case xs => echo(cmd + " is ambiguous: did you mean " + xs.map(":" + _.name).mkString(" or ") + "?") } - Result(true, None) + Result(true, None) } private def matchingCommands(cmd: String) = commands filter (_.name startsWith cmd) private def uniqueCommand(cmd: String): Option[LoopCommand] = { @@ -193,31 +212,16 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: case xs => xs find (_.name == cmd) } } - - /** Print a welcome message */ - def printWelcome() { - echo("""Welcome to - ____ __ - / __/__ ___ _____/ /__ - _\ \/ _ \/ _ `/ __/ '_/ - /___/ .__/\_,_/_/ /_/\_\ version 0.7.0 - /_/ -""") - import Properties._ - val welcomeMsg = "Using Scala %s (%s, Java %s)".format( - versionString, javaVmName, javaVersion) - echo(welcomeMsg) - } - + /** Show the history */ lazy val historyCommand = new LoopCommand("history", "show the history (optional num is commands to show)") { override def usage = "[num]" def defaultLines = 20 - + def apply(line: String): Result = { if (history eq NoHistory) return "No history available." - + val xs = words(line) val current = history.index val count = try xs.head.toInt catch { case _: Exception => defaultLines } @@ -229,32 +233,38 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } } - private def echo(msg: String) = { + // When you know you are most likely breaking into the middle + // of a line being typed. This softens the blow. + protected def echoAndRefresh(msg: String) = { + echo("\n" + msg) + in.redrawLine() + } + protected def echo(msg: String) = { out println msg out.flush() } - private def echoNoNL(msg: String) = { + protected def echoNoNL(msg: String) = { out print msg out.flush() } - + /** Search the history */ def searchHistory(_cmdline: String) { val cmdline = _cmdline.toLowerCase val offset = history.index - history.size + 1 - + for ((line, index) <- history.asStrings.zipWithIndex ; if line.toLowerCase contains cmdline) echo("%d %s".format(index + offset, line)) } - + private var currentPrompt = Properties.shellPromptString def setPrompt(prompt: String) = currentPrompt = prompt /** Prompt to print when awaiting input */ def prompt = currentPrompt - + import LoopCommand.{ cmd, nullary } - /** Standard commands **/ + /** Standard commands */ lazy val standardCommands = List( cmd("cp", "", "add a jar or directory to the classpath", addClasspath), cmd("help", "[command]", "print this summary or command-specific help", helpCommand), @@ -263,53 +273,30 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: cmd("imports", "[name name ...]", "show import history, identifying sources of names", importsCommand), cmd("implicits", "[-v]", "show the implicits in scope", implicitsCommand), cmd("javap", "", "disassemble a file or class name", javapCommand), - nullary("keybindings", "show how ctrl-[A-Z] and other keys are bound", keybindingsCommand), cmd("load", "", "load and interpret a Scala file", loadCommand), nullary("paste", "enter paste mode: all input up to ctrl-D compiled together", pasteCommand), - //nullary("power", "enable power user mode", powerCmd), - nullary("quit", "exit the interpreter", () => Result(false, None)), +// nullary("power", "enable power user mode", powerCmd), + nullary("quit", "exit the repl", () => Result(false, None)), nullary("replay", "reset execution and replay all previous commands", replay), + nullary("reset", "reset the repl to its initial state, forgetting all session entries", resetCommand), shCommand, nullary("silent", "disable/enable automatic printing of results", verbosity), - cmd("type", "", "display the type of an expression without evaluating it", typeCommand) + cmd("type", "[-v] ", "display the type of an expression without evaluating it", typeCommand), + nullary("warnings", "show the suppressed warnings from the most recent line which had any", warningsCommand) ) - + /** Power user commands */ lazy val powerCommands: List[LoopCommand] = List( - //nullary("dump", "displays a view of the interpreter's internal state", dumpCommand), - //cmd("phase", "", "set the implicit phase for power commands", phaseCommand), - cmd("wrap", "", "name of method to wrap around each repl line", wrapCommand) withLongHelp (""" - |:wrap - |:wrap clear - |:wrap - | - |Installs a wrapper around each line entered into the repl. - |Currently it must be the simple name of an existing method - |with the specific signature shown in the following example. - | - |def timed[T](body: => T): T = { - | val start = System.nanoTime - | try body - | finally println((System.nanoTime - start) + " nanos elapsed.") - |} - |:wrap timed - | - |If given no argument, :wrap names the wrapper installed. - |An argument of clear will remove the wrapper if any is active. - |Note that wrappers do not compose (a new one replaces the old - |one) and also that the :phase command uses the same machinery, - |so setting :wrap will clear any :phase setting. - """.stripMargin.trim) + // cmd("phase", "", "set the implicit phase for power commands", phaseCommand) ) - - /* - private def dumpCommand(): Result = { - echo("" + power) - history.asStrings takeRight 30 foreach echo - in.redrawLine() - } - */ - + + // private def dumpCommand(): Result = { + // echo("" + power) + // history.asStrings takeRight 30 foreach echo + // in.redrawLine() + // } + // private def valsCommand(): Result = power.valsDescription + private val typeTransforms = List( "scala.collection.immutable." -> "immutable.", "scala.collection.mutable." -> "mutable.", @@ -317,7 +304,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: "java.lang." -> "jl.", "scala.runtime." -> "runtime." ) - + private def importsCommand(line: String): Result = { val tokens = words(line) val handlers = intp.languageWildcardHandlers ++ intp.importHandlers @@ -333,7 +320,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: val implicitMsg = if (imps.isEmpty) "" else imps.size + " are implicit" val foundMsg = if (found.isEmpty) "" else found.mkString(" // imports: ", ", ", "") val statsMsg = List(typeMsg, termMsg, implicitMsg) filterNot (_ == "") mkString ("(", ", ", ")") - + intp.reporter.printMessage("%2d) %-30s %s%s".format( idx + 1, handler.importString, @@ -342,12 +329,11 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: )) } } - - private def implicitsCommand(line: String): Result = { - val intp = SparkILoop.this.intp + + private def implicitsCommand(line: String): Result = onIntp { intp => import intp._ - import global.Symbol - + import global._ + def p(x: Any) = intp.reporter.printMessage("" + x) // If an argument is given, only show a source with that @@ -360,17 +346,17 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: else (args exists (source.name.toString contains _)) } } - + if (filtered.isEmpty) return "No implicits have been imported other than those in Predef." - + filtered foreach { case (source, syms) => p("/* " + syms.size + " implicit members imported from " + source.fullName + " */") - + // This groups the members by where the symbol is defined val byOwner = syms groupBy (_.owner) - val sortedOwners = byOwner.toList sortBy { case (owner, _) => intp.afterTyper(source.info.baseClasses indexOf owner) } + val sortedOwners = byOwner.toList sortBy { case (owner, _) => afterTyper(source.info.baseClasses indexOf owner) } sortedOwners foreach { case (owner, members) => @@ -388,10 +374,10 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: xss map (xs => xs sortBy (_.name.toString)) } - - val ownerMessage = if (owner == source) " defined in " else " inherited from " + + val ownerMessage = if (owner == source) " defined in " else " inherited from " p(" /* " + members.size + ownerMessage + owner.fullName + " */") - + memberGroups foreach { group => group foreach (s => p(" " + intp.symbolDefString(s))) p("") @@ -400,158 +386,182 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: p("") } } - - protected def newJavap() = new Javap(intp.classLoader, new SparkIMain.ReplStrippingWriter(intp)) { - override def tryClass(path: String): Array[Byte] = { - // Look for Foo first, then Foo$, but if Foo$ is given explicitly, - // we have to drop the $ to find object Foo, then tack it back onto - // the end of the flattened name. - def className = intp flatName path - def moduleName = (intp flatName path.stripSuffix("$")) + "$" - val bytes = super.tryClass(className) - if (bytes.nonEmpty) bytes - else super.tryClass(moduleName) + private def findToolsJar() = { + val jdkPath = Directory(jdkHome) + val jar = jdkPath / "lib" / "tools.jar" toFile; + + if (jar isFile) + Some(jar) + else if (jdkPath.isDirectory) + jdkPath.deepFiles find (_.name == "tools.jar") + else None + } + private def addToolsJarToLoader() = { + val cl = findToolsJar match { + case Some(tools) => ScalaClassLoader.fromURLs(Seq(tools.toURL), intp.classLoader) + case _ => intp.classLoader + } + if (Javap.isAvailable(cl)) { + logDebug(":javap available.") + cl + } + else { + logDebug(":javap unavailable: no tools.jar at " + jdkHome) + intp.classLoader + } + } + + protected def newJavap() = new JavapClass(addToolsJarToLoader(), new SparkIMain.ReplStrippingWriter(intp)) { + override def tryClass(path: String): Array[Byte] = { + val hd :: rest = path split '.' toList; + // If there are dots in the name, the first segment is the + // key to finding it. + if (rest.nonEmpty) { + intp optFlatName hd match { + case Some(flat) => + val clazz = flat :: rest mkString NAME_JOIN_STRING + val bytes = super.tryClass(clazz) + if (bytes.nonEmpty) bytes + else super.tryClass(clazz + MODULE_SUFFIX_STRING) + case _ => super.tryClass(path) + } + } + else { + // Look for Foo first, then Foo$, but if Foo$ is given explicitly, + // we have to drop the $ to find object Foo, then tack it back onto + // the end of the flattened name. + def className = intp flatName path + def moduleName = (intp flatName path.stripSuffix(MODULE_SUFFIX_STRING)) + MODULE_SUFFIX_STRING + + val bytes = super.tryClass(className) + if (bytes.nonEmpty) bytes + else super.tryClass(moduleName) + } } } + // private lazy val javap = substituteAndLog[Javap]("javap", NoJavap)(newJavap()) private lazy val javap = try newJavap() catch { case _: Exception => null } - - private def typeCommand(line: String): Result = { - intp.typeOfExpression(line) match { - case Some(tp) => tp.toString - case _ => "Failed to determine type." + + // Still todo: modules. + private def typeCommand(line0: String): Result = { + line0.trim match { + case "" => ":type [-v] " + case s if s startsWith "-v " => typeCommandInternal(s stripPrefix "-v " trim, true) + case s => typeCommandInternal(s, false) } } - + + private def warningsCommand(): Result = { + if (intp.lastWarnings.isEmpty) + "Can't find any cached warnings." + else + intp.lastWarnings foreach { case (pos, msg) => intp.reporter.warning(pos, msg) } + } + private def javapCommand(line: String): Result = { if (javap == null) - return ":javap unavailable on this platform." - if (line == "") - return ":javap [-lcsvp] [path1 path2 ...]" - - javap(words(line)) foreach { res => - if (res.isError) return "Failed: " + res.value - else res.show() - } - } - private def keybindingsCommand(): Result = { - if (in.keyBindings.isEmpty) "Key bindings unavailable." - else { - echo("Reading jline properties for default key bindings.") - echo("Accuracy not guaranteed: treat this as a guideline only.\n") - in.keyBindings foreach (x => echo ("" + x)) - } + ":javap unavailable, no tools.jar at %s. Set JDK_HOME.".format(jdkHome) + else if (javaVersion startsWith "1.7") + ":javap not yet working with java 1.7" + else if (line == "") + ":javap [-lcsvp] [path1 path2 ...]" + else + javap(words(line)) foreach { res => + if (res.isError) return "Failed: " + res.value + else res.show() + } } + private def wrapCommand(line: String): Result = { def failMsg = "Argument to :wrap must be the name of a method with signature [T](=> T): T" - val intp = SparkILoop.this.intp - val g: intp.global.type = intp.global - import g._ - - words(line) match { - case Nil => - intp.executionWrapper match { - case "" => "No execution wrapper is set." - case s => "Current execution wrapper: " + s - } - case "clear" :: Nil => - intp.executionWrapper match { - case "" => "No execution wrapper is set." - case s => intp.clearExecutionWrapper() ; "Cleared execution wrapper." - } - case wrapper :: Nil => - intp.typeOfExpression(wrapper) match { - case Some(PolyType(List(targ), MethodType(List(arg), restpe))) => - intp setExecutionWrapper intp.pathToTerm(wrapper) - "Set wrapper to '" + wrapper + "'" - case Some(x) => - failMsg + "\nFound: " + x - case _ => - failMsg + "\nFound: " - } - case _ => failMsg + onIntp { intp => + import intp._ + import global._ + + words(line) match { + case Nil => + intp.executionWrapper match { + case "" => "No execution wrapper is set." + case s => "Current execution wrapper: " + s + } + case "clear" :: Nil => + intp.executionWrapper match { + case "" => "No execution wrapper is set." + case s => intp.clearExecutionWrapper() ; "Cleared execution wrapper." + } + case wrapper :: Nil => + intp.typeOfExpression(wrapper) match { + case PolyType(List(targ), MethodType(List(arg), restpe)) => + intp setExecutionWrapper intp.pathToTerm(wrapper) + "Set wrapper to '" + wrapper + "'" + case tp => + failMsg + "\nFound: " + } + case _ => failMsg + } } } private def pathToPhaseWrapper = intp.pathToTerm("$r") + ".phased.atCurrent" - /* - private def phaseCommand(name: String): Result = { - // This line crashes us in TreeGen: - // - // if (intp.power.phased set name) "..." - // - // Exception in thread "main" java.lang.AssertionError: assertion failed: ._7.type - // at scala.Predef$.assert(Predef.scala:99) - // at scala.tools.nsc.ast.TreeGen.mkAttributedQualifier(TreeGen.scala:69) - // at scala.tools.nsc.ast.TreeGen.mkAttributedQualifier(TreeGen.scala:44) - // at scala.tools.nsc.ast.TreeGen.mkAttributedRef(TreeGen.scala:101) - // at scala.tools.nsc.ast.TreeGen.mkAttributedStableRef(TreeGen.scala:143) - // - // But it works like so, type annotated. - val phased: Phased = power.phased - import phased.NoPhaseName - - if (name == "clear") { - phased.set(NoPhaseName) - intp.clearExecutionWrapper() - "Cleared active phase." - } - else if (name == "") phased.get match { - case NoPhaseName => "Usage: :phase (e.g. typer, erasure.next, erasure+3)" - case ph => "Active phase is '%s'. (To clear, :phase clear)".format(phased.get) - } - else { - val what = phased.parse(name) - if (what.isEmpty || !phased.set(what)) - "'" + name + "' does not appear to represent a valid phase." - else { - intp.setExecutionWrapper(pathToPhaseWrapper) - val activeMessage = - if (what.toString.length == name.length) "" + what - else "%s (%s)".format(what, name) - - "Active phase is now: " + activeMessage - } - } - } - */ - + // private def phaseCommand(name: String): Result = { + // val phased: Phased = power.phased + // import phased.NoPhaseName + + // if (name == "clear") { + // phased.set(NoPhaseName) + // intp.clearExecutionWrapper() + // "Cleared active phase." + // } + // else if (name == "") phased.get match { + // case NoPhaseName => "Usage: :phase (e.g. typer, erasure.next, erasure+3)" + // case ph => "Active phase is '%s'. (To clear, :phase clear)".format(phased.get) + // } + // else { + // val what = phased.parse(name) + // if (what.isEmpty || !phased.set(what)) + // "'" + name + "' does not appear to represent a valid phase." + // else { + // intp.setExecutionWrapper(pathToPhaseWrapper) + // val activeMessage = + // if (what.toString.length == name.length) "" + what + // else "%s (%s)".format(what, name) + + // "Active phase is now: " + activeMessage + // } + // } + // } + /** Available commands */ - def commands: List[LoopCommand] = standardCommands /* ++ ( + def commands: List[LoopCommand] = standardCommands /*++ ( if (isReplPower) powerCommands else Nil )*/ - + val replayQuestionMessage = - """|The repl compiler has crashed spectacularly. Shall I replay your - |session? I can re-run all lines except the last one. + """|That entry seems to have slain the compiler. Shall I replay + |your session? I can re-run each line except the last one. |[y/n] """.trim.stripMargin - private val crashRecovery: PartialFunction[Throwable, Unit] = { + private val crashRecovery: PartialFunction[Throwable, Boolean] = { case ex: Throwable => - if (settings.YrichExes.value) { - val sources = implicitly[Sources] - echo("\n" + ex.getMessage) - echo( - if (isReplDebug) "[searching " + sources.path + " for exception contexts...]" - else "[searching for exception contexts...]" - ) - echo(Exceptional(ex).force().context()) - } - else { - echo(util.stackTraceString(ex)) - } + echo(intp.global.throwableAsString(ex)) + ex match { case _: NoSuchMethodError | _: NoClassDefFoundError => - echo("Unrecoverable error.") + echo("\nUnrecoverable error.") throw ex case _ => - def fn(): Boolean = in.readYesOrNo(replayQuestionMessage, { echo("\nYou must enter y or n.") ; fn() }) + def fn(): Boolean = + try in.readYesOrNo(replayQuestionMessage, { echo("\nYou must enter y or n.") ; fn() }) + catch { case _: RuntimeException => false } + if (fn()) replay() else echo("\nAbandoning crashed session.") } + true } /** The main read-eval-print loop for the repl. It calls @@ -564,66 +574,88 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: in readLine prompt } // return false if repl should exit - def processLine(line: String): Boolean = + def processLine(line: String): Boolean = { + if (isAsync) { + if (!awaitInitialized()) return false + runThunks() + } if (line eq null) false // assume null means EOF else command(line) match { case Result(false, _) => false case Result(_, Some(finalLine)) => addReplay(finalLine) ; true case _ => true } - - while (true) { - try if (!processLine(readOneLine)) return - catch crashRecovery } + def innerLoop() { + if ( try processLine(readOneLine()) catch crashRecovery ) + innerLoop() + } + innerLoop() } /** interpret all lines from a specified file */ - def interpretAllFrom(file: File) { - val oldIn = in - val oldReplay = replayCommandStack - - try file applyReader { reader => - in = SimpleReader(reader, out, false) - echo("Loading " + file + "...") - loop() - } - finally { - in = oldIn - replayCommandStack = oldReplay + def interpretAllFrom(file: File) { + savingReader { + savingReplayStack { + file applyReader { reader => + in = SimpleReader(reader, out, false) + echo("Loading " + file + "...") + loop() + } + } } } - /** create a new interpreter and replay all commands so far */ + /** create a new interpreter and replay the given commands */ def replay() { - closeInterpreter() - createInterpreter() - for (cmd <- replayCommands) { + reset() + if (replayCommandStack.isEmpty) + echo("Nothing to replay.") + else for (cmd <- replayCommands) { echo("Replaying: " + cmd) // flush because maybe cmd will have its own output command(cmd) echo("") } } - + def resetCommand() { + echo("Resetting repl state.") + if (replayCommandStack.nonEmpty) { + echo("Forgetting this session history:\n") + replayCommands foreach echo + echo("") + replayCommandStack = Nil + } + if (intp.namedDefinedTerms.nonEmpty) + echo("Forgetting all expression results and named terms: " + intp.namedDefinedTerms.mkString(", ")) + if (intp.definedTypes.nonEmpty) + echo("Forgetting defined types: " + intp.definedTypes.mkString(", ")) + + reset() + } + def reset() { + intp.reset() + // unleashAndSetPhase() + } + /** fork a shell and run a command */ lazy val shCommand = new LoopCommand("sh", "run a shell command (result is implicitly => List[String])") { override def usage = "" def apply(line: String): Result = line match { case "" => showUsage() - case _ => + case _ => val toRun = classOf[ProcessResult].getName + "(" + string2codeQuoted(line) + ")" intp interpret toRun () } } - + def withFile(filename: String)(action: File => Unit) { val f = File(filename) - + if (f.exists) action(f) else echo("That file does not exist") } - + def loadCommand(arg: String) = { var shouldReplay: Option[String] = None withFile(arg)(f => { @@ -643,23 +675,36 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } else echo("The path '" + f + "' doesn't seem to exist.") } - + def powerCmd(): Result = { if (isReplPower) "Already in power mode." - else enablePowerMode() + else enablePowerMode(false) + } + + def enablePowerMode(isDuringInit: Boolean) = { + // replProps.power setValue true + // unleashAndSetPhase() + // asyncEcho(isDuringInit, power.banner) } - def enablePowerMode() = { - //replProps.power setValue true - //power.unleash() - //echo(power.banner) + // private def unleashAndSetPhase() { +// if (isReplPower) { +// // power.unleash() +// // Set the phase to "typer" +// intp beSilentDuring phaseCommand("typer") +// } +// } + + def asyncEcho(async: Boolean, msg: => String) { + if (async) asyncMessage(msg) + else echo(msg) } - + def verbosity() = { - val old = intp.printResults - intp.printResults = !old - echo("Switched " + (if (old) "off" else "on") + " result printing.") + // val old = intp.printResults + // intp.printResults = !old + // echo("Switched " + (if (old) "off" else "on") + " result printing.") } - + /** Run one command submitted by the user. Two values are returned: * (1) whether to keep running, (2) the line to record for replay, * if any. */ @@ -674,11 +719,11 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: else if (intp.global == null) Result(false, None) // Notice failure to create compiler else Result(true, interpretStartingWith(line)) } - + private def readWhile(cond: String => Boolean) = { Iterator continually in.readLine("") takeWhile (x => x != null && cond(x)) } - + def pasteCommand(): Result = { echo("// Entering paste mode (ctrl-D to finish)\n") val code = readWhile(_ => true) mkString "\n" @@ -686,23 +731,19 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: intp interpret code () } - + private object paste extends Pasted { val ContinueString = " | " val PromptString = "scala> " - + def interpret(line: String): Unit = { echo(line.trim) intp interpret line echo("") } - + def transcript(start: String) = { - // Printing this message doesn't work very well because it's buried in the - // transcript they just pasted. Todo: a short timer goes off when - // lines stop coming which tells them to hit ctrl-D. - // - // echo("// Detected repl transcript paste: ctrl-D to finish.") + echo("\n// Detected repl transcript paste: ctrl-D to finish.\n") apply(Iterator(start) ++ readWhile(_.trim != PromptString.trim)) } } @@ -717,7 +758,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: def interpretStartingWith(code: String): Option[String] = { // signal completion non-completion input has been received in.completion.resetVerbosity() - + def reallyInterpret = { val reallyResult = intp.interpret(code) (reallyResult, reallyResult match { @@ -727,7 +768,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: if (in.interactive && code.endsWith("\n\n")) { echo("You typed two blank lines. Starting a new command.") None - } + } else in.readLine(ContinueString) match { case null => // we know compilation is going to fail since we're at EOF and the @@ -741,10 +782,10 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } }) } - + /** Here we place ourselves between the user and the interpreter and examine * the input they are ostensibly submitting. We intervene in several cases: - * + * * 1) If the line starts with "scala> " it is assumed to be an interpreter paste. * 2) If the line starts with "." (but not ".." or "./") it is treated as an invocation * on the previous result. @@ -759,28 +800,12 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: else if (Completion.looksLikeInvocation(code) && intp.mostRecentVar != "") { interpretStartingWith(intp.mostRecentVar + code) } - else { - def runCompletion = in.completion execute code map (intp bindValue _) - /** Due to my accidentally letting file completion execution sneak ahead - * of actual parsing this now operates in such a way that the scala - * interpretation always wins. However to avoid losing useful file - * completion I let it fail and then check the others. So if you - * type /tmp it will echo a failure and then give you a Directory object. - * It's not pretty: maybe I'll implement the silence bits I need to avoid - * echoing the failure. - */ - if (intp isParseable code) { - val (code, result) = reallyInterpret - //if (power != null && code == IR.Error) - // runCompletion - - result - } - else runCompletion match { - case Some(_) => None // completion hit: avoid the latent error - case _ => reallyInterpret._2 // trigger the latent error - } + else if (code.trim startsWith "//") { + // line comment, do nothing + None } + else + reallyInterpret._2 } // runs :load `file` on any files passed via -i @@ -794,7 +819,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } case _ => } - + /** Tries to create a JLineReader, falling back to SimpleReader: * unless settings or properties are such that it should start * with SimpleReader. @@ -802,7 +827,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: def chooseReader(settings: Settings): InteractiveReader = { if (settings.Xnojline.value || Properties.isEmacsShell) SimpleReader() - else try SparkJLineReader( + else try new SparkJLineReader( if (settings.noCompletion.value) NoCompletion else new SparkJLineCompletion(intp) ) @@ -813,96 +838,97 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: } } - def initializeSpark() { - intp.beQuietDuring { - command(""" - spark.repl.Main.interp.out.println("Creating SparkContext..."); - spark.repl.Main.interp.out.flush(); - @transient val sc = spark.repl.Main.interp.createSparkContext(); - spark.repl.Main.interp.out.println("Spark context available as sc."); - spark.repl.Main.interp.out.flush(); - """) - command("import spark.SparkContext._"); - } - echo("Type in expressions to have them evaluated.") - echo("Type :help for more information.") - } - - var sparkContext: SparkContext = null - - def createSparkContext(): SparkContext = { - val master = this.master match { - case Some(m) => m - case None => { - val prop = System.getenv("MASTER") - if (prop != null) prop else "local" - } - } - sparkContext = new SparkContext(master, "Spark shell") - sparkContext - } - - def process(settings: Settings): Boolean = { - // Ensure logging is initialized before any Spark threads try to use logs - // (because SLF4J initialization is not thread safe) - initLogging() - - printWelcome() - echo("Initializing interpreter...") + val u: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe + val m = u.runtimeMirror(getClass.getClassLoader) + private def tagOfStaticClass[T: ClassTag]: u.TypeTag[T] = + u.TypeTag[T]( + m, + new TypeCreator { + def apply[U <: ApiUniverse with Singleton](m: Mirror[U]): U # Type = + m.staticClass(classTag[T].runtimeClass.getName).toTypeConstructor.asInstanceOf[U # Type] + }) + def process(settings: Settings): Boolean = savingContextLoader { this.settings = settings createInterpreter() - + // sets in to some kind of reader depending on environmental cues in = in0 match { case Some(reader) => SimpleReader(reader, out, true) - case None => chooseReader(settings) + case None => + // some post-initialization + chooseReader(settings) match { + case x: SparkJLineReader => addThunk(x.consoleReader.postInit) ; x + case x => x + } } + lazy val tagOfSparkIMain = tagOfStaticClass[spark.repl.SparkIMain] + // Bind intp somewhere out of the regular namespace where + // we can get at it in generated code. + addThunk(intp.quietBind(NamedParam[SparkIMain]("$intp", intp)(tagOfSparkIMain, classTag[SparkIMain]))) + addThunk({ + import scala.tools.nsc.io._ + import Properties.userHome + import scala.compat.Platform.EOL + val autorun = replProps.replAutorunCode.option flatMap (f => io.File(f).safeSlurp()) + if (autorun.isDefined) intp.quietRun(autorun.get) + }) loadFiles(settings) // it is broken on startup; go ahead and exit if (intp.reporter.hasErrors) return false - - try { - // this is about the illusion of snappiness. We call initialize() - // which spins off a separate thread, then print the prompt and try - // our best to look ready. Ideally the user will spend a - // couple seconds saying "wow, it starts so fast!" and by the time - // they type a command the compiler is ready to roll. - intp.initialize() - initializeSpark() - if (isReplPower) { - echo("Starting in power mode, one moment...\n") - enablePowerMode() - } - loop() + + // This is about the illusion of snappiness. We call initialize() + // which spins off a separate thread, then print the prompt and try + // our best to look ready. The interlocking lazy vals tend to + // inter-deadlock, so we break the cycle with a single asynchronous + // message to an actor. + if (isAsync) { + intp initialize initializedCallback() + addThunk(initializeSpark()) + createAsyncListener() // listens for signal to run postInitialization + } + else { + intp.initializeSynchronous() + postInitialization() } + printWelcome() + + try loop() + catch AbstractOrMissingHandler() finally closeInterpreter() + true } + def createSparkContext(): SparkContext = { + val master = this.master match { + case Some(m) => m + case None => { + val prop = System.getenv("MASTER") + if (prop != null) prop else "local" + } + } + sparkContext = new SparkContext(master, "Spark shell") + echo("Created spark context..") + sparkContext + } + /** process command-line arguments and do as they request */ def process(args: Array[String]): Boolean = { - val command = new CommandLine(args.toList, msg => echo("scala: " + msg)) + val command = new CommandLine(args.toList, echo) def neededHelp(): String = (if (command.settings.help.value) command.usageMsg + "\n" else "") + (if (command.settings.Xhelp.value) command.xusageMsg + "\n" else "") - + // if they asked for no help and command is valid, we call the real main neededHelp() match { case "" => command.ok && process(command.settings) case help => echoNoNL(help) ; true } } - - @deprecated("Use `process` instead", "2.9.0") - def main(args: Array[String]): Unit = { - if (isReplDebug) - System.out.println(new java.util.Date) - - process(args) - } + @deprecated("Use `process` instead", "2.9.0") def main(settings: Settings): Unit = process(settings) } @@ -916,10 +942,10 @@ object SparkILoop { // like if you'd just typed it into the repl. def runForTranscript(code: String, settings: Settings): String = { import java.io.{ BufferedReader, StringReader, OutputStreamWriter } - + stringFromStream { ostream => Console.withOut(ostream) { - val output = new PrintWriter(new OutputStreamWriter(ostream), true) { + val output = new JPrintWriter(new OutputStreamWriter(ostream), true) { override def write(str: String) = { // completely skip continuation lines if (str forall (ch => ch.isWhitespace || ch == '|')) () @@ -945,19 +971,19 @@ object SparkILoop { } } } - + /** Creates an interpreter loop with default settings and feeds * the given code to it as input. */ def run(code: String, sets: Settings = new Settings): String = { import java.io.{ BufferedReader, StringReader, OutputStreamWriter } - + stringFromStream { ostream => Console.withOut(ostream) { val input = new BufferedReader(new StringReader(code)) - val output = new PrintWriter(new OutputStreamWriter(ostream), true) - val repl = new SparkILoop(input, output) - + val output = new JPrintWriter(new OutputStreamWriter(ostream), true) + val repl = new ILoop(input, output) + if (sets.classpath.isDefault) sets.classpath.value = sys.props("java.class.path") @@ -966,32 +992,4 @@ object SparkILoop { } } def run(lines: List[String]): String = run(lines map (_ + "\n") mkString) - - // provide the enclosing type T - // in order to set up the interpreter's classpath and parent class loader properly - def breakIf[T: Manifest](assertion: => Boolean, args: NamedParam*): Unit = - if (assertion) break[T](args.toList) - - // start a repl, binding supplied args - def break[T: Manifest](args: List[NamedParam]): Unit = { - val msg = if (args.isEmpty) "" else " Binding " + args.size + " value%s.".format( - if (args.size == 1) "" else "s" - ) - echo("Debug repl starting." + msg) - val repl = new SparkILoop { - override def prompt = "\ndebug> " - } - repl.settings = new Settings(echo) - repl.settings.embeddedDefaults[T] - repl.createInterpreter() - repl.in = SparkJLineReader(repl) - - // rebind exit so people don't accidentally call sys.exit by way of predef - repl.quietRun("""def exit = println("Type :quit to resume program execution.")""") - args foreach (p => repl.bind(p.name, p.tpe, p.value)) - repl.loop() - - echo("\nDebug repl exiting.") - repl.closeInterpreter() - } } diff --git a/repl/src/main/scala/spark/repl/SparkILoopInit.scala b/repl/src/main/scala/spark/repl/SparkILoopInit.scala new file mode 100644 index 00000000000..b52c477474a --- /dev/null +++ b/repl/src/main/scala/spark/repl/SparkILoopInit.scala @@ -0,0 +1,142 @@ +/* NSC -- new Scala compiler + * Copyright 2005-2013 LAMP/EPFL + * @author Paul Phillips + */ + +package spark.repl + +import scala.tools.nsc._ +import scala.tools.nsc.interpreter._ + +import scala.reflect.internal.util.Position +import scala.util.control.Exception.ignoring +import scala.tools.nsc.util.stackTraceString + +/** + * Machinery for the asynchronous initialization of the repl. + */ +trait SparkILoopInit { + self: SparkILoop => + + /** Print a welcome message */ + def printWelcome() { + echo("""Welcome to + ____ __ + / __/__ ___ _____/ /__ + _\ \/ _ \/ _ `/ __/ '_/ + /___/ .__/\_,_/_/ /_/\_\ version 0.7.1-SNAPSHOT + /_/ +""") + import Properties._ + val welcomeMsg = "Using Scala %s (%s, Java %s)".format( + versionString, javaVmName, javaVersion) + echo(welcomeMsg) + } + + protected def asyncMessage(msg: String) { + if (isReplInfo || isReplPower) + echoAndRefresh(msg) + } + + private val initLock = new java.util.concurrent.locks.ReentrantLock() + private val initCompilerCondition = initLock.newCondition() // signal the compiler is initialized + private val initLoopCondition = initLock.newCondition() // signal the whole repl is initialized + private val initStart = System.nanoTime + + private def withLock[T](body: => T): T = { + initLock.lock() + try body + finally initLock.unlock() + } + // a condition used to ensure serial access to the compiler. + @volatile private var initIsComplete = false + @volatile private var initError: String = null + private def elapsed() = "%.3f".format((System.nanoTime - initStart).toDouble / 1000000000L) + + // the method to be called when the interpreter is initialized. + // Very important this method does nothing synchronous (i.e. do + // not try to use the interpreter) because until it returns, the + // repl's lazy val `global` is still locked. + protected def initializedCallback() = withLock(initCompilerCondition.signal()) + + // Spins off a thread which awaits a single message once the interpreter + // has been initialized. + protected def createAsyncListener() = { + io.spawn { + withLock(initCompilerCondition.await()) + asyncMessage("[info] compiler init time: " + elapsed() + " s.") + postInitialization() + } + } + + // called from main repl loop + protected def awaitInitialized(): Boolean = { + if (!initIsComplete) + withLock { while (!initIsComplete) initLoopCondition.await() } + if (initError != null) { + println(""" + |Failed to initialize the REPL due to an unexpected error. + |This is a bug, please, report it along with the error diagnostics printed below. + |%s.""".stripMargin.format(initError) + ) + false + } else true + } + // private def warningsThunks = List( + // () => intp.bind("lastWarnings", "" + typeTag[List[(Position, String)]], intp.lastWarnings _), + // ) + + protected def postInitThunks = List[Option[() => Unit]]( + Some(intp.setContextClassLoader _), + if (isReplPower) Some(() => enablePowerMode(true)) else None + ).flatten + // ++ ( + // warningsThunks + // ) + // called once after init condition is signalled + protected def postInitialization() { + try { + postInitThunks foreach (f => addThunk(f())) + runThunks() + } catch { + case ex: Throwable => + initError = stackTraceString(ex) + throw ex + } finally { + initIsComplete = true + + if (isAsync) { + asyncMessage("[info] total init time: " + elapsed() + " s.") + withLock(initLoopCondition.signal()) + } + } + } + + def initializeSpark() { + intp.beQuietDuring { + command(""" + @transient lazy val sc = spark.repl.Main.interp.createSparkContext(); + """) + command("import spark.SparkContext._"); + } + // echo("Type in expressions to have them evaluated.") + // echo("Type :help for more information.") + } + + // code to be executed only after the interpreter is initialized + // and the lazy val `global` can be accessed without risk of deadlock. + private var pendingThunks: List[() => Unit] = Nil + protected def addThunk(body: => Unit) = synchronized { + pendingThunks :+= (() => body) + } + protected def runThunks(): Unit = synchronized { + if (pendingThunks.nonEmpty) + println("Clearing " + pendingThunks.size + " thunks.") + + while (pendingThunks.nonEmpty) { + val thunk = pendingThunks.head + pendingThunks = pendingThunks.tail + thunk() + } + } +} diff --git a/repl/src/main/scala/spark/repl/SparkIMain.scala b/repl/src/main/scala/spark/repl/SparkIMain.scala index 43b6a6c9500..84536282333 100644 --- a/repl/src/main/scala/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/spark/repl/SparkIMain.scala @@ -1,5 +1,5 @@ /* NSC -- new Scala compiler - * Copyright 2005-2011 LAMP/EPFL + * Copyright 2005-2013 LAMP/EPFL * @author Martin Odersky */ @@ -9,304 +9,334 @@ import scala.tools.nsc._ import scala.tools.nsc.interpreter._ import Predef.{ println => _, _ } -import java.io.{ PrintWriter } -import java.lang.reflect +import util.stringFromWriter +import scala.reflect.internal.util._ import java.net.URL -import util.{ Set => _, _ } -import io.{ AbstractFile, PlainFile, VirtualDirectory } -import reporters.{ ConsoleReporter, Reporter } -import symtab.{ Flags, Names } -import scala.tools.nsc.interpreter.{ Results => IR } +import scala.sys.BooleanProp +import io.{AbstractFile, PlainFile, VirtualDirectory} + +import reporters._ +import symtab.Flags +import scala.reflect.internal.Names import scala.tools.util.PathResolver -import scala.tools.nsc.util.{ ScalaClassLoader, Exceptional } +import scala.tools.nsc.util.ScalaClassLoader import ScalaClassLoader.URLClassLoader -import Exceptional.unwrap +import scala.tools.nsc.util.Exceptional.unwrap import scala.collection.{ mutable, immutable } -import scala.PartialFunction.{ cond, condOpt } import scala.util.control.Exception.{ ultimately } -import scala.reflect.NameTransformer import SparkIMain._ +import java.util.concurrent.Future +import typechecker.Analyzer +import scala.language.implicitConversions +import scala.reflect.runtime.{ universe => ru } +import scala.reflect.{ ClassTag, classTag } +import scala.tools.reflect.StdRuntimeTags._ +import scala.util.control.ControlThrowable +import util.stackTraceString import spark.HttpServer import spark.Utils import spark.SparkEnv +import spark.Logging + +// /** directory to save .class files to */ +// private class ReplVirtualDirectory(out: JPrintWriter) extends VirtualDirectory("((memory))", None) { +// private def pp(root: AbstractFile, indentLevel: Int) { +// val spaces = " " * indentLevel +// out.println(spaces + root.name) +// if (root.isDirectory) +// root.toList sortBy (_.name) foreach (x => pp(x, indentLevel + 1)) +// } +// // print the contents hierarchically +// def show() = pp(this, 0) +// } + + /** An interpreter for Scala code. + * + * The main public entry points are compile(), interpret(), and bind(). + * The compile() method loads a complete Scala file. The interpret() method + * executes one line of Scala code at the request of the user. The bind() + * method binds an object to a variable that can then be used by later + * interpreted code. + * + * The overall approach is based on compiling the requested code and then + * using a Java classloader and Java reflection to run the code + * and access its results. + * + * In more detail, a single compiler instance is used + * to accumulate all successfully compiled or interpreted Scala code. To + * "interpret" a line of code, the compiler generates a fresh object that + * includes the line of code and which has public member(s) to export + * all variables defined by that code. To extract the result of an + * interpreted line to show the user, a second "result object" is created + * which imports the variables exported by the above object and then + * exports members called "$eval" and "$print". To accomodate user expressions + * that read from variables or methods defined in previous statements, "import" + * statements are used. + * + * This interpreter shares the strengths and weaknesses of using the + * full compiler-to-Java. The main strength is that interpreted code + * behaves exactly as does compiled code, including running at full speed. + * The main weakness is that redefining classes and methods is not handled + * properly, because rebinding at the Java level is technically difficult. + * + * @author Moez A. Abdel-Gawad + * @author Lex Spoon + */ + class SparkIMain(initialSettings: Settings, val out: JPrintWriter) extends SparkImports with Logging { + imain => -/** An interpreter for Scala code. - * - * The main public entry points are compile(), interpret(), and bind(). - * The compile() method loads a complete Scala file. The interpret() method - * executes one line of Scala code at the request of the user. The bind() - * method binds an object to a variable that can then be used by later - * interpreted code. - * - * The overall approach is based on compiling the requested code and then - * using a Java classloader and Java reflection to run the code - * and access its results. - * - * In more detail, a single compiler instance is used - * to accumulate all successfully compiled or interpreted Scala code. To - * "interpret" a line of code, the compiler generates a fresh object that - * includes the line of code and which has public member(s) to export - * all variables defined by that code. To extract the result of an - * interpreted line to show the user, a second "result object" is created - * which imports the variables exported by the above object and then - * exports a single member named "$export". To accomodate user expressions - * that read from variables or methods defined in previous statements, "import" - * statements are used. - * - * This interpreter shares the strengths and weaknesses of using the - * full compiler-to-Java. The main strength is that interpreted code - * behaves exactly as does compiled code, including running at full speed. - * The main weakness is that redefining classes and methods is not handled - * properly, because rebinding at the Java level is technically difficult. - * - * @author Moez A. Abdel-Gawad - * @author Lex Spoon - */ -class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends SparkImports { - imain => - - /** construct an interpreter that reports to Console */ - def this(settings: Settings) = this(settings, new NewLinePrintWriter(new ConsoleWriter, true)) - def this() = this(new Settings()) - - /** whether to print out result lines */ - var printResults: Boolean = true - - /** whether to print errors */ - var totalSilence: Boolean = false - - private val RESULT_OBJECT_PREFIX = "RequestResult$" - - lazy val formatting: Formatting = new Formatting { - val prompt = Properties.shellPromptString - } - import formatting._ - - val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1") - - /** Local directory to save .class files too */ - val outputDir = { - val tmp = System.getProperty("java.io.tmpdir") - val rootDir = System.getProperty("spark.repl.classdir", tmp) - Utils.createTempDir(rootDir) - } - if (SPARK_DEBUG_REPL) { - echo("Output directory: " + outputDir) - } - /** Scala compiler virtual directory for outputDir */ - val virtualDirectory = new PlainFile(outputDir) + val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1") - /** Jetty server that will serve our classes to worker nodes */ - val classServer = new HttpServer(outputDir) + /** Local directory to save .class files too */ + val outputDir = { + val tmp = System.getProperty("java.io.tmpdir") + val rootDir = System.getProperty("spark.repl.classdir", tmp) + Utils.createTempDir(rootDir) + } + if (SPARK_DEBUG_REPL) { + echo("Output directory: " + outputDir) + } - // Start the classServer and store its URI in a spark system property - // (which will be passed to executors so that they can connect to it) - classServer.start() - System.setProperty("spark.repl.class.uri", classServer.uri) - if (SPARK_DEBUG_REPL) { - echo("Class server started, URI = " + classServer.uri) - } + val virtualDirectory = new PlainFile(outputDir) // "directory" for classfiles + val classServer = new HttpServer(outputDir) /** Jetty server that will serve our classes to worker nodes */ + private var currentSettings: Settings = initialSettings + var printResults = true // whether to print result lines + var totalSilence = false // whether to print anything + private var _initializeComplete = false // compiler is initialized + private var _isInitialized: Future[Boolean] = null // set up initialization future + private var bindExceptions = true // whether to bind the lastException variable + private var _executionWrapper = "" // code to be wrapped around all lines + + + // Start the classServer and store its URI in a spark system property + // (which will be passed to executors so that they can connect to it) + classServer.start() + System.setProperty("spark.repl.class.uri", classServer.uri) + if (SPARK_DEBUG_REPL) { + echo("Class server started, URI = " + classServer.uri) + } + + /** We're going to go to some trouble to initialize the compiler asynchronously. + * It's critical that nothing call into it until it's been initialized or we will + * run into unrecoverable issues, but the perceived repl startup time goes + * through the roof if we wait for it. So we initialize it with a future and + * use a lazy val to ensure that any attempt to use the compiler object waits + * on the future. + */ + private var _classLoader: AbstractFileClassLoader = null // active classloader + private val _compiler: Global = newCompiler(settings, reporter) // our private compiler - /* - // directory to save .class files to - val virtualDirectory = new VirtualDirectory("(memory)", None) { - private def pp(root: io.AbstractFile, indentLevel: Int) { - val spaces = " " * indentLevel - out.println(spaces + root.name) - if (root.isDirectory) - root.toList sortBy (_.name) foreach (x => pp(x, indentLevel + 1)) + private val nextReqId = { + var counter = 0 + () => { counter += 1 ; counter } } - // print the contents hierarchically - def show() = pp(this, 0) - } - */ - - /** reporter */ - lazy val reporter: ConsoleReporter = new SparkIMain.ReplReporter(this) - import reporter.{ printMessage, withoutTruncating } - - // not sure if we have some motivation to print directly to console - private def echo(msg: String) { Console println msg } - - // protected def defaultImports: List[String] = List("_root_.scala.sys.exit") - - /** We're going to go to some trouble to initialize the compiler asynchronously. - * It's critical that nothing call into it until it's been initialized or we will - * run into unrecoverable issues, but the perceived repl startup time goes - * through the roof if we wait for it. So we initialize it with a future and - * use a lazy val to ensure that any attempt to use the compiler object waits - * on the future. - */ - private val _compiler: Global = newCompiler(settings, reporter) - private var _initializeComplete = false - def isInitializeComplete = _initializeComplete - - private def _initialize(): Boolean = { - val source = """ - |class $repl_$init { - | List(1) map (_ + 1) - |} - |""".stripMargin - - val result = try { - new _compiler.Run() compileSources List(new BatchSourceFile("", source)) - if (isReplDebug || settings.debug.value) { - // Can't use printMessage here, it deadlocks - Console.println("Repl compiler initialized.") - } - // addImports(defaultImports: _*) - true - } - catch { - case x: AbstractMethodError => - printMessage(""" - |Failed to initialize compiler: abstract method error. - |This is most often remedied by a full clean and recompile. - |""".stripMargin - ) - x.printStackTrace() - false - case x: MissingRequirementError => printMessage(""" - |Failed to initialize compiler: %s not found. - |** Note that as of 2.8 scala does not assume use of the java classpath. - |** For the old behavior pass -usejavacp to scala, or if using a Settings - |** object programatically, settings.usejavacp.value = true.""".stripMargin.format(x.req) + + def compilerClasspath: Seq[URL] = ( + if (isInitializeComplete) global.classPath.asURLs + else new PathResolver(settings).result.asURLs // the compiler's classpath ) - false + def settings = currentSettings + def mostRecentLine = prevRequestList match { + case Nil => "" + case req :: _ => req.originalLine + } + // Run the code body with the given boolean settings flipped to true. + def withoutWarnings[T](body: => T): T = beQuietDuring { + val saved = settings.nowarn.value + if (!saved) + settings.nowarn.value = true + + try body + finally if (!saved) settings.nowarn.value = false } - - try result - finally _initializeComplete = result - } - - // set up initialization future - private var _isInitialized: () => Boolean = null - def initialize() = synchronized { - if (_isInitialized == null) - _isInitialized = scala.concurrent.ops future _initialize() - } - /** the public, go through the future compiler */ - lazy val global: Global = { - initialize() + /** construct an interpreter that reports to Console */ + def this(settings: Settings) = this(settings, new NewLinePrintWriter(new ConsoleWriter, true)) + def this() = this(new Settings()) - // blocks until it is ; false means catastrophic failure - if (_isInitialized()) _compiler - else null - } - @deprecated("Use `global` for access to the compiler instance.", "2.9.0") - lazy val compiler: global.type = global - - import global._ - - object naming extends { - val global: imain.global.type = imain.global - } with Naming { - // make sure we don't overwrite their unwisely named res3 etc. - override def freshUserVarName(): String = { - val name = super.freshUserVarName() - if (definedNameMap contains name) freshUserVarName() - else name + lazy val repllog: Logger = new Logger { + val out: JPrintWriter = imain.out + val isInfo: Boolean = BooleanProp keyExists "scala.repl.info" + val isDebug: Boolean = BooleanProp keyExists "scala.repl.debug" + val isTrace: Boolean = BooleanProp keyExists "scala.repl.trace" } - } - import naming._ - - // object dossiers extends { - // val intp: imain.type = imain - // } with Dossiers { } - // import dossiers._ - - lazy val memberHandlers = new { - val intp: imain.type = imain - } with SparkMemberHandlers - import memberHandlers._ - - def atPickler[T](op: => T): T = atPhase(currentRun.picklerPhase)(op) - def afterTyper[T](op: => T): T = atPhase(currentRun.typerPhase.next)(op) - - /** Temporarily be quiet */ - def beQuietDuring[T](operation: => T): T = { - val wasPrinting = printResults - ultimately(printResults = wasPrinting) { - if (isReplDebug) echo(">> beQuietDuring") - else printResults = false - - operation + lazy val formatting: Formatting = new Formatting { + val prompt = Properties.shellPromptString } - } - def beSilentDuring[T](operation: => T): T = { - val saved = totalSilence - totalSilence = true - try operation - finally totalSilence = saved - } - - def quietRun[T](code: String) = beQuietDuring(interpret(code)) - - /** whether to bind the lastException variable */ - private var bindLastException = true - - /** A string representing code to be wrapped around all lines. */ - private var _executionWrapper: String = "" - def executionWrapper = _executionWrapper - def setExecutionWrapper(code: String) = _executionWrapper = code - def clearExecutionWrapper() = _executionWrapper = "" - - /** Temporarily stop binding lastException */ - def withoutBindingLastException[T](operation: => T): T = { - val wasBinding = bindLastException - ultimately(bindLastException = wasBinding) { - bindLastException = false - operation + lazy val reporter: ConsoleReporter = new SparkIMain.ReplReporter(this) + + import formatting._ + import reporter.{ printMessage, withoutTruncating } + + // This exists mostly because using the reporter too early leads to deadlock. + private def echo(msg: String) { Console println msg } + private def _initSources = List(new BatchSourceFile("", "class $repl_$init { }")) + private def _initialize() = { + try { + // todo. if this crashes, REPL will hang + new _compiler.Run() compileSources _initSources + _initializeComplete = true + true + } + catch AbstractOrMissingHandler() + } + private def tquoted(s: String) = "\"\"\"" + s + "\"\"\"" + + // argument is a thunk to execute after init is done + def initialize(postInitSignal: => Unit) { + synchronized { + if (_isInitialized == null) { + _isInitialized = io.spawn { + try _initialize() + finally postInitSignal + } + } + } + } + def initializeSynchronous(): Unit = { + if (!isInitializeComplete) { + _initialize() + assert(global != null, global) + } + } + def isInitializeComplete = _initializeComplete + + /** the public, go through the future compiler */ + lazy val global: Global = { + if (isInitializeComplete) _compiler + else { + // If init hasn't been called yet you're on your own. + if (_isInitialized == null) { + logWarning("Warning: compiler accessed before init set up. Assuming no postInit code.") + initialize(()) + } + // // blocks until it is ; false means catastrophic failure + if (_isInitialized.get()) _compiler + else null + } + } + @deprecated("Use `global` for access to the compiler instance.", "2.9.0") + lazy val compiler: global.type = global + + import global._ + import definitions.{ScalaPackage, JavaLangPackage, termMember, typeMember} + import rootMirror.{RootClass, getClassIfDefined, getModuleIfDefined, getRequiredModule, getRequiredClass} + + implicit class ReplTypeOps(tp: Type) { + def orElse(other: => Type): Type = if (tp ne NoType) tp else other + def andAlso(fn: Type => Type): Type = if (tp eq NoType) tp else fn(tp) + } + + // TODO: If we try to make naming a lazy val, we run into big time + // scalac unhappiness with what look like cycles. It has not been easy to + // reduce, but name resolution clearly takes different paths. + object naming extends { + val global: imain.global.type = imain.global + } with Naming { + // make sure we don't overwrite their unwisely named res3 etc. + def freshUserTermName(): TermName = { + val name = newTermName(freshUserVarName()) + if (definedNameMap contains name) freshUserTermName() + else name + } + def isUserTermName(name: Name) = isUserVarName("" + name) + def isInternalTermName(name: Name) = isInternalVarName("" + name) + } + import naming._ + + object deconstruct extends { + val global: imain.global.type = imain.global + } with StructuredTypeStrings + + lazy val memberHandlers = new { + val intp: imain.type = imain + } with SparkMemberHandlers + import memberHandlers._ + + /** Temporarily be quiet */ + def beQuietDuring[T](body: => T): T = { + val saved = printResults + printResults = false + try body + finally printResults = saved + } + def beSilentDuring[T](operation: => T): T = { + val saved = totalSilence + totalSilence = true + try operation + finally totalSilence = saved + } + + def quietRun[T](code: String) = beQuietDuring(interpret(code)) + + + private def logAndDiscard[T](label: String, alt: => T): PartialFunction[Throwable, T] = { + case t: ControlThrowable => throw t + case t: Throwable => + logDebug(label + ": " + unwrap(t)) + logDebug(stackTraceString(unwrap(t))) + alt + } + /** takes AnyRef because it may be binding a Throwable or an Exceptional */ + + private def withLastExceptionLock[T](body: => T, alt: => T): T = { + assert(bindExceptions, "withLastExceptionLock called incorrectly.") + bindExceptions = false + + try beQuietDuring(body) + catch logAndDiscard("withLastExceptionLock", alt) + finally bindExceptions = true } - } - - protected def createLineManager(): Line.Manager = new Line.Manager - lazy val lineManager = createLineManager() - - /** interpreter settings */ - lazy val isettings = new SparkISettings(this) - - /** Instantiate a compiler. Subclasses can override this to - * change the compiler class used by this interpreter. */ - protected def newCompiler(settings: Settings, reporter: Reporter) = { - settings.outputDirs setSingleOutput virtualDirectory - settings.exposeEmptyPackage.value = true - new Global(settings, reporter) - } - - /** the compiler's classpath, as URL's */ - lazy val compilerClasspath: List[URL] = new PathResolver(settings) asURLs - /* A single class loader is used for all commands interpreted by this Interpreter. + def executionWrapper = _executionWrapper + def setExecutionWrapper(code: String) = _executionWrapper = code + def clearExecutionWrapper() = _executionWrapper = "" + + /** interpreter settings */ + lazy val isettings = new SparkISettings(this) + + /** Instantiate a compiler. Overridable. */ + protected def newCompiler(settings: Settings, reporter: Reporter): ReplGlobal = { + settings.outputDirs setSingleOutput virtualDirectory + settings.exposeEmptyPackage.value = true + new Global(settings, reporter) with ReplGlobal { + override def toString: String = "" + } + } + + /** Parent classloader. Overridable. */ + protected def parentClassLoader: ClassLoader = + SparkHelper.explicitParentLoader(settings).getOrElse( this.getClass.getClassLoader() ) + + /* A single class loader is used for all commands interpreted by this Interpreter. It would also be possible to create a new class loader for each command to interpret. The advantages of the current approach are: - - Expressions are only evaluated one time. This is especially - significant for I/O, e.g. "val x = Console.readLine" - - The main disadvantage is: - - - Objects, classes, and methods cannot be rebound. Instead, definitions - shadow the old ones, and old code objects refer to the old - definitions. - */ - private var _classLoader: AbstractFileClassLoader = null - def resetClassLoader() = _classLoader = makeClassLoader() - def classLoader: AbstractFileClassLoader = { - if (_classLoader == null) - resetClassLoader() - - _classLoader - } - private def makeClassLoader(): AbstractFileClassLoader = { - val parent = - if (parentClassLoader == null) ScalaClassLoader fromURLs compilerClasspath - else new URLClassLoader(compilerClasspath, parentClassLoader) + - Expressions are only evaluated one time. This is especially + significant for I/O, e.g. "val x = Console.readLine" + + The main disadvantage is: - new AbstractFileClassLoader(virtualDirectory, parent) { + - Objects, classes, and methods cannot be rebound. Instead, definitions + shadow the old ones, and old code objects refer to the old + definitions. + */ + def resetClassLoader() = { + logDebug("Setting new classloader: was " + _classLoader) + _classLoader = null + ensureClassLoader() + } + final def ensureClassLoader() { + if (_classLoader == null) + _classLoader = makeClassLoader() + } + def classLoader: AbstractFileClassLoader = { + ensureClassLoader() + _classLoader + } + private class TranslatingClassLoader(parent: ClassLoader) extends AbstractFileClassLoader(virtualDirectory, parent) { /** Overridden here to try translating a simple name to the generated * class name if the original attempt fails. This method is used by * getResourceAsStream as well as findClass. @@ -314,223 +344,300 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends override protected def findAbstractFile(name: String): AbstractFile = { super.findAbstractFile(name) match { // deadlocks on startup if we try to translate names too early - case null if isInitializeComplete => generatedName(name) map (x => super.findAbstractFile(x)) orNull - case file => file + case null if isInitializeComplete => + generatedName(name) map (x => super.findAbstractFile(x)) orNull + case file => + file } } } - } - private def loadByName(s: String): JClass = - (classLoader tryToInitializeClass s) getOrElse sys.error("Failed to load expected class: '" + s + "'") - - protected def parentClassLoader: ClassLoader = - SparkHelper.explicitParentLoader(settings).getOrElse( this.getClass.getClassLoader() ) - - def getInterpreterClassLoader() = classLoader - - // Set the current Java "context" class loader to this interpreter's class loader - def setContextClassLoader() = classLoader.setAsContext() - - /** Given a simple repl-defined name, returns the real name of - * the class representing it, e.g. for "Bippy" it may return - * - * $line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$Bippy - */ - def generatedName(simpleName: String): Option[String] = { - if (simpleName endsWith "$") optFlatName(simpleName.init) map (_ + "$") - else optFlatName(simpleName) - } - def flatName(id: String) = optFlatName(id) getOrElse id - def optFlatName(id: String) = requestForIdent(id) map (_ fullFlatName id) - - def allDefinedNames = definedNameMap.keys.toList sortBy (_.toString) - def pathToType(id: String): String = pathToName(newTypeName(id)) - def pathToTerm(id: String): String = pathToName(newTermName(id)) - def pathToName(name: Name): String = { - if (definedNameMap contains name) - definedNameMap(name) fullPath name - else name.toString - } + private def makeClassLoader(): AbstractFileClassLoader = + new TranslatingClassLoader(parentClassLoader match { + case null => ScalaClassLoader fromURLs compilerClasspath + case p => new URLClassLoader(compilerClasspath, p) + }) + + def getInterpreterClassLoader() = classLoader + + // Set the current Java "context" class loader to this interpreter's class loader + def setContextClassLoader() = classLoader.setAsContext() + + /** Given a simple repl-defined name, returns the real name of + * the class representing it, e.g. for "Bippy" it may return + * {{{ + * $line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$Bippy + * }}} + */ + def generatedName(simpleName: String): Option[String] = { + if (simpleName endsWith nme.MODULE_SUFFIX_STRING) optFlatName(simpleName.init) map (_ + nme.MODULE_SUFFIX_STRING) + else optFlatName(simpleName) + } + def flatName(id: String) = optFlatName(id) getOrElse id + def optFlatName(id: String) = requestForIdent(id) map (_ fullFlatName id) + + def allDefinedNames = definedNameMap.keys.toList.sorted + def pathToType(id: String): String = pathToName(newTypeName(id)) + def pathToTerm(id: String): String = pathToName(newTermName(id)) + def pathToName(name: Name): String = { + if (definedNameMap contains name) + definedNameMap(name) fullPath name + else name.toString + } - /** Most recent tree handled which wasn't wholly synthetic. */ - private def mostRecentlyHandledTree: Option[Tree] = { - prevRequests.reverse foreach { req => - req.handlers.reverse foreach { - case x: MemberDefHandler if x.definesValue && !isInternalVarName(x.name.toString) => return Some(x.member) - case _ => () + /** Most recent tree handled which wasn't wholly synthetic. */ + private def mostRecentlyHandledTree: Option[Tree] = { + prevRequests.reverse foreach { req => + req.handlers.reverse foreach { + case x: MemberDefHandler if x.definesValue && !isInternalTermName(x.name) => return Some(x.member) + case _ => () + } } + None } - None - } - - /** Stubs for work in progress. */ - def handleTypeRedefinition(name: TypeName, old: Request, req: Request) = { - for (t1 <- old.simpleNameOfType(name) ; t2 <- req.simpleNameOfType(name)) { - DBG("Redefining type '%s'\n %s -> %s".format(name, t1, t2)) + + /** Stubs for work in progress. */ + def handleTypeRedefinition(name: TypeName, old: Request, req: Request) = { + for (t1 <- old.simpleNameOfType(name) ; t2 <- req.simpleNameOfType(name)) { + logDebug("Redefining type '%s'\n %s -> %s".format(name, t1, t2)) + } } - } - def handleTermRedefinition(name: TermName, old: Request, req: Request) = { - for (t1 <- old.compilerTypeOf get name ; t2 <- req.compilerTypeOf get name) { - // Printing the types here has a tendency to cause assertion errors, like - // assertion failed: fatal: has owner value x, but a class owner is required - // so DBG is by-name now to keep it in the family. (It also traps the assertion error, - // but we don't want to unnecessarily risk hosing the compiler's internal state.) - DBG("Redefining term '%s'\n %s -> %s".format(name, t1, t2)) + def handleTermRedefinition(name: TermName, old: Request, req: Request) = { + for (t1 <- old.compilerTypeOf get name ; t2 <- req.compilerTypeOf get name) { + // Printing the types here has a tendency to cause assertion errors, like + // assertion failed: fatal: has owner value x, but a class owner is required + // so DBG is by-name now to keep it in the family. (It also traps the assertion error, + // but we don't want to unnecessarily risk hosing the compiler's internal state.) + logDebug("Redefining term '%s'\n %s -> %s".format(name, t1, t2)) + } } - } - def recordRequest(req: Request) { - if (req == null || referencedNameMap == null) - return - - prevRequests += req - req.referencedNames foreach (x => referencedNameMap(x) = req) - - // warning about serially defining companions. It'd be easy - // enough to just redefine them together but that may not always - // be what people want so I'm waiting until I can do it better. - if (!settings.nowarnings.value) { + + def recordRequest(req: Request) { + if (req == null || referencedNameMap == null) + return + + prevRequests += req + req.referencedNames foreach (x => referencedNameMap(x) = req) + + // warning about serially defining companions. It'd be easy + // enough to just redefine them together but that may not always + // be what people want so I'm waiting until I can do it better. for { name <- req.definedNames filterNot (x => req.definedNames contains x.companionName) oldReq <- definedNameMap get name.companionName newSym <- req.definedSymbols get name oldSym <- oldReq.definedSymbols get name.companionName + if Seq(oldSym, newSym).permutations exists { case Seq(s1, s2) => s1.isClass && s2.isModule } } { - printMessage("warning: previously defined %s is not a companion to %s.".format(oldSym, newSym)) - printMessage("Companions must be defined together; you may wish to use :paste mode for this.") + afterTyper(replwarn(s"warning: previously defined $oldSym is not a companion to $newSym.")) + replwarn("Companions must be defined together; you may wish to use :paste mode for this.") } - } - - // Updating the defined name map - req.definedNames foreach { name => - if (definedNameMap contains name) { - if (name.isTypeName) handleTypeRedefinition(name.toTypeName, definedNameMap(name), req) - else handleTermRedefinition(name.toTermName, definedNameMap(name), req) + + // Updating the defined name map + req.definedNames foreach { name => + if (definedNameMap contains name) { + if (name.isTypeName) handleTypeRedefinition(name.toTypeName, definedNameMap(name), req) + else handleTermRedefinition(name.toTermName, definedNameMap(name), req) + } + definedNameMap(name) = req } - definedNameMap(name) = req } - } - /** Parse a line into a sequence of trees. Returns None if the input is incomplete. */ - def parse(line: String): Option[List[Tree]] = { - var justNeedsMore = false - reporter.withIncompleteHandler((pos,msg) => {justNeedsMore = true}) { - // simple parse: just parse it, nothing else - def simpleParse(code: String): List[Tree] = { - reporter.reset() - val unit = new CompilationUnit(new BatchSourceFile("", code)) - val scanner = new syntaxAnalyzer.UnitParser(unit) - - scanner.templateStatSeq(false)._2 - } - val trees = simpleParse(line) - - if (reporter.hasErrors) Some(Nil) // the result did not parse, so stop - else if (justNeedsMore) None - else Some(trees) + def replwarn(msg: => String) { + if (!settings.nowarnings.value) + printMessage(msg) } - } - - def isParseable(line: String): Boolean = { - beSilentDuring { - parse(line) match { - case Some(xs) => xs.nonEmpty // parses as-is - case None => true // incomplete + + def isParseable(line: String): Boolean = { + beSilentDuring { + try parse(line) match { + case Some(xs) => xs.nonEmpty // parses as-is + case None => true // incomplete + } + catch { case x: Exception => // crashed the compiler + replwarn("Exception in isParseable(\"" + line + "\"): " + x) + false + } } } + + def compileSourcesKeepingRun(sources: SourceFile*) = { + val run = new Run() + reporter.reset() + run compileSources sources.toList + (!reporter.hasErrors, run) + } + + /** Compile an nsc SourceFile. Returns true if there are + * no compilation errors, or false otherwise. + */ + def compileSources(sources: SourceFile*): Boolean = + compileSourcesKeepingRun(sources: _*)._1 + + /** Compile a string. Returns true if there are no + * compilation errors, or false otherwise. + */ + def compileString(code: String): Boolean = + compileSources(new BatchSourceFile(" + {title} + + +
+
+
+ +

+ {title} +

+
+
+ {content} +
+ + + } +} From 9161db5478c3e8c1c124070bacdf571b744cee22 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 09:50:24 -0700 Subject: [PATCH 0131/2521] Cleaning up master web UI --- .../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 c76b5b1e0ee..0b7d1639809 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 { From ee73c09ac965076c9239587721ffa76741b99483 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 10:10:48 -0700 Subject: [PATCH 0132/2521] 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 0b7d1639809..34f50fd5e49 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) } From fc94576ece99f2b224a951b32f0f6360701b7cd3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 13:11:57 -0700 Subject: [PATCH 0133/2521] Adding worker version of UI --- .../spark/deploy/master/MasterWebUI.scala | 16 +-- .../scala/spark/deploy/worker/Worker.scala | 12 +- .../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 34f50fd5e49..a2e9dfd762e 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 f8fdab927e5..3878fe3f7b7 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 3235c50d1bd..b8b4b897387 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 eaacd956915..34b776f1d81 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) => From 91ec5a1a04339983d57a72d8df8f1d769d8d855a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 15:13:12 -0700 Subject: [PATCH 0134/2521] Changing JSON protocol and removing spray code --- .../scala/spark/deploy/JsonProtocol.scala | 114 ++++++++---------- .../spark/deploy/master/MasterWebUI.scala | 6 +- .../spark/deploy/worker/WorkerWebUI.scala | 6 +- .../scala/spark/storage/BlockManagerUI.scala | 11 +- .../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 ea832101d21..b4365d31e92 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 a2e9dfd762e..6623142d699 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 b8b4b897387..0af9eb8efaa 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 6ac4398de73..e9c362fce75 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 bd2d637ae74..134c912c467 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 34b776f1d81..e6b39b15ebe 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 faf6e2ae8e8..ec26b2a2293 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 f806e664813..1b0f879b948 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) From d6fde4ffe402b406ac9d9ea6d5c3ee0fb682f628 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 15:34:54 -0700 Subject: [PATCH 0135/2521] Some JSON cleanup --- .../scala/spark/deploy/master/MasterWebUI.scala | 13 +++++++++++++ .../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 6623142d699..49300ea7429 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 0af9eb8efaa..3257f0f513f 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 e6b39b15ebe..4d9df55a4fa 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)) } } } From 4e1f202481844d950ef2a80679bdc9cee85f0b75 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 15:37:27 -0700 Subject: [PATCH 0136/2521] 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 4d9df55a4fa..ae69c159e77 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 == "*") { From 17f145f3bcdee12d2d965af3bc2ae3f9aa2d8ff3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 16:50:23 -0700 Subject: [PATCH 0137/2521] 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 88f0ed70f3c..5edafb3706f 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 3bcb2a3f345..81043e3b6c8 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 From 32a45d01b104f79d9e5f10c333c44cbf910949ef Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 18:45:14 -0700 Subject: [PATCH 0138/2521] Removing twirl files --- core/src/main/scala/spark/util/WebUI.scala | 12 ++- .../main/twirl/spark/common/layout.scala.html | 35 -------- .../deploy/master/app_details.scala.html | 38 --------- .../spark/deploy/master/app_row.scala.html | 20 ----- .../spark/deploy/master/app_table.scala.html | 21 ----- .../deploy/master/executor_row.scala.html | 15 ---- .../deploy/master/executors_table.scala.html | 19 ----- .../spark/deploy/master/index.scala.html | 53 ------------ .../spark/deploy/master/worker_row.scala.html | 14 ---- .../deploy/master/worker_table.scala.html | 18 ----- .../deploy/worker/executor_row.scala.html | 20 ----- .../deploy/worker/executors_table.scala.html | 18 ----- .../spark/deploy/worker/index.scala.html | 44 ---------- .../main/twirl/spark/storage/index.scala.html | 40 --------- .../main/twirl/spark/storage/rdd.scala.html | 81 ------------------- .../twirl/spark/storage/rdd_table.scala.html | 32 -------- .../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 ae69c159e77..b9ddfdb40f0 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 b9192060aa5..00000000000 --- 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 5e5e5de5513..00000000000 --- 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 feb306f35cc..00000000000 --- 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 f789cee0f16..00000000000 --- 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 21e72c7aabf..00000000000 --- 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 cafc42c80e8..00000000000 --- 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 b9b9f088109..00000000000 --- 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 46277ca421b..00000000000 --- 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 b249411a625..00000000000 --- 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 dad0a89080f..00000000000 --- 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 327a2399c73..00000000000 --- 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 0e66af92847..00000000000 --- 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 2b337f61339..00000000000 --- 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 d85addeb175..00000000000 --- 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 a51e64aed00..00000000000 --- 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 cd72a688c1f..00000000000 --- 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 From 8b5c7e71c4cbf8b8ad9ab9c81f3b21ee3da49e24 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 19:02:20 -0700 Subject: [PATCH 0139/2521] Import cleanup --- core/src/main/scala/spark/deploy/WebUI.scala | 2 +- .../spark/deploy/master/MasterWebUI.scala | 25 +++++++++---------- .../spark/deploy/worker/WorkerWebUI.scala | 13 +++++----- .../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 ad1a1092b2e..844c4142c7a 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 49300ea7429..171e74a3a6d 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 3257f0f513f..80b285b3f6f 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 e9c362fce75..ecffd1edcf1 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 b9ddfdb40f0..bde1acdbcac 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 From 77c53f7868a036dc7712865e3a43d9fdea47cc98 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Jun 2013 22:18:04 -0700 Subject: [PATCH 0140/2521] Refactoring UI packages --- .../static/bootstrap-responsive.min.css | 0 .../static/bootstrap.min.css | 0 .../{deploy => webui}/static/sorttable.js | 0 .../{deploy => webui}/static/spark_logo.png | Bin core/src/main/scala/spark/SparkContext.scala | 8 +-- .../spark/deploy/master/MasterWebUI.scala | 13 +++-- .../spark/deploy/worker/WorkerWebUI.scala | 42 ++-------------- .../{storage => ui}/BlockManagerUI.scala | 47 +++++------------- core/src/main/scala/spark/ui/SparkUI.scala | 41 +++++++++++++++ .../main/scala/spark/{util => ui}/WebUI.scala | 30 ++++++++--- 10 files changed, 93 insertions(+), 88 deletions(-) rename core/src/main/resources/spark/{deploy => webui}/static/bootstrap-responsive.min.css (100%) rename core/src/main/resources/spark/{deploy => webui}/static/bootstrap.min.css (100%) rename core/src/main/resources/spark/{deploy => webui}/static/sorttable.js (100%) rename core/src/main/resources/spark/{deploy => webui}/static/spark_logo.png (100%) rename core/src/main/scala/spark/{storage => ui}/BlockManagerUI.scala (79%) create mode 100644 core/src/main/scala/spark/ui/SparkUI.scala rename core/src/main/scala/spark/{util => ui}/WebUI.scala (79%) diff --git a/core/src/main/resources/spark/deploy/static/bootstrap-responsive.min.css b/core/src/main/resources/spark/webui/static/bootstrap-responsive.min.css similarity index 100% rename from core/src/main/resources/spark/deploy/static/bootstrap-responsive.min.css rename to core/src/main/resources/spark/webui/static/bootstrap-responsive.min.css diff --git a/core/src/main/resources/spark/deploy/static/bootstrap.min.css b/core/src/main/resources/spark/webui/static/bootstrap.min.css similarity index 100% rename from core/src/main/resources/spark/deploy/static/bootstrap.min.css rename to core/src/main/resources/spark/webui/static/bootstrap.min.css diff --git a/core/src/main/resources/spark/deploy/static/sorttable.js b/core/src/main/resources/spark/webui/static/sorttable.js similarity index 100% rename from core/src/main/resources/spark/deploy/static/sorttable.js rename to core/src/main/resources/spark/webui/static/sorttable.js diff --git a/core/src/main/resources/spark/deploy/static/spark_logo.png b/core/src/main/resources/spark/webui/static/spark_logo.png similarity index 100% rename from core/src/main/resources/spark/deploy/static/spark_logo.png rename to core/src/main/resources/spark/webui/static/spark_logo.png diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 70a9d7698c4..cc634b3ec42 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -46,8 +46,9 @@ import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler} import spark.scheduler.local.LocalScheduler import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import spark.storage.{BlockManagerUI, StorageStatus, StorageUtils, RDDInfo} +import spark.storage.{StorageStatus, StorageUtils, RDDInfo} import spark.util.{MetadataCleaner, TimeStampedHashMap} +import ui.{SparkUI, BlockManagerUI} /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -93,9 +94,8 @@ class SparkContext( isLocal) SparkEnv.set(env) - // Start the BlockManager UI - private[spark] val ui = new BlockManagerUI( - env.actorSystem, env.blockManager.master.driverActor, this) + // 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 diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 171e74a3a6d..328a7cb2979 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -10,10 +10,11 @@ import net.liftweb.json.JsonAST.JValue import org.eclipse.jetty.server.Handler import scala.xml.Node import spark.{Logging, Utils} -import spark.util.WebUI -import spark.util.WebUI._ +import spark.ui.WebUI +import WebUI._ import spark.deploy._ import spark.deploy.MasterState +import spark.ui.WebUI /** * Web UI server for the standalone master. @@ -64,6 +65,7 @@ class MasterWebUI(master: ActorRef) extends Logging { state.completedApps.find(_.id == appId).getOrElse(null) }) val content = +
      @@ -97,7 +99,7 @@ class MasterWebUI(master: ActorRef) extends Logging { {executorTable(app.executors.values.toList)}
    ; - WebUI.makePage(content, "Application Info: " + app.desc.name) + WebUI.sparkPage(content, "Application Info: " + app.desc.name) } def executorTable(executors: Seq[ExecutorInfo]): Seq[Node] = { @@ -142,6 +144,7 @@ class MasterWebUI(master: ActorRef) extends Logging { val state = Await.result(stateFuture, 3 seconds) val content = +
      @@ -186,7 +189,7 @@ class MasterWebUI(master: ActorRef) extends Logging { {appTable(state.completedApps.sortBy(_.endTime).reverse)}
    ; - WebUI.makePage(content, "Spark Master: " + state.uri) + WebUI.sparkPage(content, "Spark Master: " + state.uri) } def workerTable(workers: Seq[spark.deploy.master.WorkerInfo]) = { @@ -258,6 +261,6 @@ class MasterWebUI(master: ActorRef) extends Logging { } object MasterWebUI { - val STATIC_RESOURCE_DIR = "spark/deploy/static" + 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/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index 80b285b3f6f..f661d99815d 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.util.{WebUI => UtilsWebUI} -import spark.util.WebUI._ +import spark.ui.{WebUI => UtilsWebUI} +import spark.ui.WebUI._ import xml.Node /** @@ -55,6 +55,7 @@ class WorkerWebUI(worker: ActorRef, workDir: File) extends Logging { val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] val workerState = Await.result(stateFuture, 3 seconds) val content = +
      @@ -88,7 +89,7 @@ class WorkerWebUI(worker: ActorRef, workDir: File) extends Logging {
    ; - UtilsWebUI.makePage(content, "Spark Worker on %s:%s".format(workerState.host, workerState.port)) + UtilsWebUI.sparkPage(content, "Spark Worker on %s:%s".format(workerState.host, workerState.port)) } def executorTable(executors: Seq[ExecutorRunner]): Seq[Node] = { @@ -139,42 +140,9 @@ class WorkerWebUI(worker: ActorRef, workDir: File) extends Logging { source.close() lines } - - /* - val handler = { - get { - (path("") & parameters('format ?)) { - case Some(js) if js.equalsIgnoreCase("json") => { - val future = worker ? RequestWorkerState - respondWithMediaType(MediaTypes.`application/json`) { ctx => - ctx.complete(future.mapTo[WorkerState]) - } - } - case _ => - completeWith{ - val future = worker ? RequestWorkerState - future.map { workerState => - spark.deploy.worker.html.index(workerState.asInstanceOf[WorkerState]) - } - } - } ~ - path("log") { - parameters("appId", "executorId", "logType") { (appId, executorId, logType) => - respondWithMediaType(cc.spray.http.MediaTypes.`text/plain`) { - getFromFileName(workDir.getPath() + "/" + appId + "/" + executorId + "/" + logType) - } - } - } ~ - pathPrefix("static") { - getFromResourceDirectory(STATIC_RESOURCE_DIR) - } ~ - getFromResourceDirectory(RESOURCE_DIR) - } - } - */ } object WorkerWebUI { - val STATIC_RESOURCE_DIR = "spark/deploy/static" + val STATIC_RESOURCE_DIR = "spark/webui/static" val DEFAULT_PORT="8081" } diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/ui/BlockManagerUI.scala similarity index 79% rename from core/src/main/scala/spark/storage/BlockManagerUI.scala rename to core/src/main/scala/spark/ui/BlockManagerUI.scala index ecffd1edcf1..f3197515905 100644 --- a/core/src/main/scala/spark/storage/BlockManagerUI.scala +++ b/core/src/main/scala/spark/ui/BlockManagerUI.scala @@ -1,4 +1,4 @@ -package spark.storage +package spark.ui import akka.actor.{ActorRef, ActorSystem} import akka.util.Duration @@ -6,38 +6,23 @@ 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 WebUI._ import xml.Node +import spark.storage.StorageUtils /** * Web UI server for the BlockManager inside each SparkContext. */ private[spark] -class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, sc: SparkContext) - extends Logging { +class BlockManagerUI(sc: SparkContext) + extends UIComponent with 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")) - .getOrElse(BlockManagerUI.DEFAULT_PORT).toInt - /** Start a HTTP server to run the Web interface */ - def start() { - try { - 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) - System.exit(1) - } - } - - val handlers = Array[(String, Handler)]( - ("/static", createStaticHandler(BlockManagerUI.STATIC_RESOURCE_DIR)), - ("/rdd", (request: HttpServletRequest) => rddPage(request)), - ("*", (request: HttpServletRequest) => indexPage) + def getHandlers = Seq[(String, Handler)]( + ("/storage/rdd", (request: HttpServletRequest) => rddPage(request)), + ("/storage", (request: HttpServletRequest) => indexPage) ) def rddPage(request: HttpServletRequest): Seq[Node] = { @@ -137,7 +122,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, ; - WebUI.makePage(content, "RDD Info: " + id) + WebUI.headerSparkPage(content, "RDD Info: " + id) } def indexPage: Seq[Node] = { @@ -176,7 +161,7 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, {for (rdd <- rdds) yield - + {rdd.name} @@ -191,12 +176,6 @@ class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, ; - WebUI.makePage(content, "Spark Storage") + WebUI.headerSparkPage(content, "Spark Storage ") } - private[spark] def appUIAddress = "http://" + host + ":" + port -} - -object BlockManagerUI { - val STATIC_RESOURCE_DIR = "spark/deploy/static" - val DEFAULT_PORT = "33000" } diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala new file mode 100644 index 00000000000..6c9affc1f7b --- /dev/null +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -0,0 +1,41 @@ +package spark.ui + +import spark.{Logging, SparkContext, Utils} +import javax.servlet.http.HttpServletRequest +import org.eclipse.jetty.server.Handler +import WebUI._ + +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 + + + val handlers = Seq[(String, Handler)]( + ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), + ("*", (request: HttpServletRequest) => WebUI.headerSparkPage(

    Test

    , "Test page")) + ) + val components = Seq(new BlockManagerUI(sc)) + + 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)) + } catch { + case e: Exception => + logError("Failed to create Spark WebUI", e) + System.exit(1) + } + } + + private[spark] def appUIAddress = "http://" + host + ":" + port +} + +object SparkUI { + val DEFAULT_PORT = "33000" + val STATIC_RESOURCE_DIR = "spark/webui/static" +} + + + diff --git a/core/src/main/scala/spark/util/WebUI.scala b/core/src/main/scala/spark/ui/WebUI.scala similarity index 79% rename from core/src/main/scala/spark/util/WebUI.scala rename to core/src/main/scala/spark/ui/WebUI.scala index bde1acdbcac..16251e727ac 100644 --- a/core/src/main/scala/spark/util/WebUI.scala +++ b/core/src/main/scala/spark/ui/WebUI.scala @@ -1,4 +1,4 @@ -package spark.util +package spark.ui import annotation.tailrec import javax.servlet.http.{HttpServletResponse, HttpServletRequest} @@ -9,6 +9,10 @@ import spark.Logging import util.{Try, Success, Failure} import xml.Node +abstract class UIComponent { + def getHandlers(): Seq[(String, Handler)] +} + object WebUI extends Logging { type Responder[T] = HttpServletRequest => T @@ -41,24 +45,27 @@ object WebUI extends Logging { val staticHandler = new ResourceHandler Option(getClass.getClassLoader.getResource(resourceBase)) match { case Some(res) => - staticHandler.setResourceBase (res.toString) - staticHandler + staticHandler.setResourceBase(res.toString) + case None => + logError("Could not find resource path for Web UI: " + resourceBase) } + staticHandler } - def startJettyServer(ip: String, port: Int, handlers: Array[(String, Handler)]): (Server, Int) = { + 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) + println("Adding handler for path: " + path) contextHandler.setHandler(handler) contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] } } val handlerList = new HandlerList - handlerList.setHandlers(handlersToRegister) + handlerList.setHandlers(handlersToRegister.toArray) @tailrec def connect(currentPort: Int): (Server, Int) = { @@ -71,11 +78,19 @@ object WebUI extends Logging { connect((currentPort + 1) % 65536) } } - connect(port) } - def makePage(content: => Seq[Node], title: String): Seq[Node] = { + /** 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] = { @@ -98,7 +113,6 @@ object WebUI extends Logging { -
    {content} From bc4a811c57ee126559721a3a4284510baef184d1 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 19 Jun 2013 12:56:01 -0700 Subject: [PATCH 0141/2521] 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 00000000000..eb782d85c82 --- /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 { + +} + From 9fd5dc3ea9016b59667fd903992843605087e43b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 20 Jun 2013 09:02:56 -0700 Subject: [PATCH 0142/2521] Initial steps towards job progress UI --- core/src/main/scala/spark/SparkContext.scala | 8 +- .../scala/spark/scheduler/DAGScheduler.scala | 1 + .../main/scala/spark/scheduler/Stage.scala | 1 + .../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 cc634b3ec42..155a5ee7213 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 f7d60be5dbf..bdd8792ce96 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 7fc9e13fd9d..539cf8233bf 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 eb782d85c82..1f12df10b85 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 6c9affc1f7b..63b75df36f4 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 16251e727ac..e80d48dd4a6 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) } From ce81c320acfd265b8de7c30a46c6a3adc4f39a74 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 20 Jun 2013 10:45:52 -0700 Subject: [PATCH 0143/2521] Adding helper function to make listing tables --- .../main/scala/spark/ui/BlockManagerUI.scala | 155 ++++++++---------- .../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 f3197515905..3be50648372 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 1f12df10b85..b51e62bb1a9 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 e80d48dd4a6..265624a726a 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
    + } } From dcf6a68177912ffa69ee75345c8172fd2db9e0ea Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 20 Jun 2013 12:05:49 -0700 Subject: [PATCH 0144/2521] Refactoring into different modules --- core/src/main/scala/spark/SparkContext.scala | 2 +- .../main/scala/spark/ui/BlockManagerUI.scala | 156 ------------------ .../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 +- .../main/scala/spark/ui/jobs/IndexPage.scala | 55 ++++++ .../scala/spark/ui/jobs/JobProgressUI.scala | 82 +++++++++ .../main/scala/spark/ui/jobs/StagePage.scala | 63 +++++++ .../spark/ui/storage/BlockManagerUI.scala | 26 +++ .../scala/spark/ui/storage/IndexPage.scala | 61 +++++++ .../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 155a5ee7213..901cda4174f 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 3be50648372..00000000000 --- 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 b51e62bb1a9..00000000000 --- 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 63b75df36f4..a1f6cc60ec4 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 265624a726a..6dedd28fc10 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 00000000000..1740524f49f --- /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 00000000000..9ee962bf2e5 --- /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 00000000000..49f5b1c73c3 --- /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 00000000000..21555384dd0 --- /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 00000000000..2f4857d48af --- /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 00000000000..957c4000800 --- /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))} + + } +} From 17776323a6ea21170a10169af66e86c4a6cc65fb Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 20 Jun 2013 17:46:33 -0700 Subject: [PATCH 0145/2521] More work on percentile data: --- .../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 49f5b1c73c3..64b8c8418c7 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("") } } From 5c872e9ef55a467a8a5e71f0bd12537eac9d447f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 20 Jun 2013 21:06:18 -0700 Subject: [PATCH 0146/2521] Documentation and some refactoring --- .../spark/deploy/master/MasterWebUI.scala | 14 +++--- .../spark/deploy/worker/WorkerWebUI.scala | 6 +-- .../spark/ui/{WebUI.scala => JettyUI.scala} | 44 ++++++++++------- core/src/main/scala/spark/ui/SparkUI.scala | 33 +++++++------ .../main/scala/spark/ui/jobs/IndexPage.scala | 22 +++++---- .../scala/spark/ui/jobs/JobProgressUI.scala | 47 +++++++------------ .../main/scala/spark/ui/jobs/StagePage.scala | 19 ++++---- .../spark/ui/storage/BlockManagerUI.scala | 9 ++-- .../scala/spark/ui/storage/IndexPage.scala | 8 ++-- .../main/scala/spark/ui/storage/RDDPage.scala | 12 ++--- 10 files changed, 108 insertions(+), 106 deletions(-) rename core/src/main/scala/spark/ui/{WebUI.scala => JettyUI.scala} (78%) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 328a7cb2979..1ec5f77bd79 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 f661d99815d..0febb9364cd 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/WebUI.scala b/core/src/main/scala/spark/ui/JettyUI.scala similarity index 78% rename from core/src/main/scala/spark/ui/WebUI.scala rename to core/src/main/scala/spark/ui/JettyUI.scala index 6dedd28fc10..c3f01073d52 100644 --- a/core/src/main/scala/spark/ui/WebUI.scala +++ b/core/src/main/scala/spark/ui/JettyUI.scala @@ -1,26 +1,26 @@ package spark.ui import annotation.tailrec + import javax.servlet.http.{HttpServletResponse, HttpServletRequest} -import net.liftweb.json._ + +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 spark.Logging -import util.{Try, Success, Failure} -import xml.Node -abstract class UIComponent { - def getHandlers(): Seq[(String, Handler)] -} +import scala.util.{Try, Success, Failure} -abstract class View[T] { - def render(request: HttpServletRequest): T -} +import spark.Logging + +import xml.Node -object WebUI extends Logging { - // CORE WEB UI COMPONENTS +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))) @@ -30,7 +30,7 @@ object WebUI extends Logging { implicit def textResponderToHandler(responder: Responder[String]): Handler = createHandler(responder, "text/plain") - def createHandler[T <% AnyRef](responder: Responder[T], contentType: String, + private def createHandler[T <% AnyRef](responder: Responder[T], contentType: String, extractFn: T => String = (in: Any) => in.toString): Handler = { new AbstractHandler { def handle(target: String, @@ -46,6 +46,7 @@ object WebUI extends Logging { } } + /** 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 { @@ -57,6 +58,12 @@ object WebUI extends Logging { 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 == "*") { @@ -84,18 +91,19 @@ object WebUI extends Logging { } connect(port) } +} - // HELPER FUNCTIONS AND SHORTCUTS +/** Utility functions for generating XML pages with spark content. */ +object UIUtils { - /** Page with Spark logo, title, and Spark UI headers */ + /** 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) } - /** Page with Spark logo and title */ + /** Returns a page containing the supplied content and the spark css, js, and logo. */ def sparkPage(content: => Seq[Node], title: String): Seq[Node] = { @@ -125,7 +133,7 @@ object WebUI extends Logging { } - /** Shortcut for making a table derived from a sequence of objects. */ + /** 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 => )} diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index a1f6cc60ec4..dd7d33e0fad 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/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 1740524f49f..811baae811b 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]] { - + {data.map(d => )} diff --git a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala index 21555384dd0..d83c8260339 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 2f4857d48af..5ead772bc07 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 957c4000800..7628fde4aa4 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] = { From dd696f3a3d04f5e3b453173c0d4e86690f6bcb1b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 20 Jun 2013 23:20:56 -0700 Subject: [PATCH 0147/2521] Some renaming and comments --- .../scala/spark/deploy/master/MasterWebUI.scala | 14 +++++++------- .../scala/spark/deploy/worker/WorkerWebUI.scala | 6 +++--- .../spark/ui/{JettyUI.scala => JettyUtils.scala} | 4 +++- 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 ++++-- 7 files changed, 25 insertions(+), 19 deletions(-) rename core/src/main/scala/spark/ui/{JettyUI.scala => JettyUtils.scala} (97%) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 1ec5f77bd79..32386611270 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 0febb9364cd..805e7b52dbf 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/JettyUtils.scala similarity index 97% rename from core/src/main/scala/spark/ui/JettyUI.scala rename to core/src/main/scala/spark/ui/JettyUtils.scala index c3f01073d52..8bb343163ad 100644 --- a/core/src/main/scala/spark/ui/JettyUI.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -15,7 +15,9 @@ import spark.Logging import xml.Node -private[spark] object JettyUI extends Logging { + +/** 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 diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index dd7d33e0fad..7428e7a3431 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 31aa0f9f0d6..97fda84e06b 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 d83c8260339..4f1928fac98 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 5ead772bc07..1f9953fb6b7 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) { From 3485e73376fcef524e62547ce3d69bb28a4381ad Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 21 Jun 2013 13:47:27 -0700 Subject: [PATCH 0148/2521] Style cleanup --- .../scala/spark/deploy/master/Master.scala | 1 + .../spark/deploy/master/MasterWebUI.scala | 266 ------------------ .../deploy/master/ui/ApplicationPage.scala | 100 +++++++ .../spark/deploy/master/ui/IndexPage.scala | 115 ++++++++ .../spark/deploy/master/ui/MasterWebUI.scala | 50 ++++ .../scala/spark/deploy/worker/Worker.scala | 1 + .../spark/deploy/worker/WorkerWebUI.scala | 148 ---------- .../spark/deploy/worker/ui/IndexPage.scala | 97 +++++++ .../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 6a7bbdfcbf3..3eca522c7ef 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 32386611270..00000000000 --- 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] = { -
    {h}
    {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 9ee962bf2e5..31aa0f9f0d6 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 64b8c8418c7..74ac811cef3 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] =
    {d}
    - - - - - - - - - - - - {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 00000000000..2a810b71d21 --- /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 00000000000..f833c59de8d --- /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 00000000000..5fd17f10c6c --- /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 3878fe3f7b7..690bdfe128d 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 805e7b52dbf..00000000000 --- 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 00000000000..a6aba59e9fa --- /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 00000000000..abfc847527b --- /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 8bb343163ad..93e7129fc01 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 From a86bb459e220cfb1375a6f31a94680ab24004b69 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 21 Jun 2013 14:56:57 -0700 Subject: [PATCH 0149/2521] Showing shuffle status and purging old stages --- core/src/main/scala/spark/ui/SparkUI.scala | 2 +- .../main/scala/spark/ui/jobs/IndexPage.scala | 12 +++++- .../scala/spark/ui/jobs/JobProgressUI.scala | 37 ++++++++++++++++++- .../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 7428e7a3431..deb6284a666 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 811baae811b..0174e1ddd15 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 97fda84e06b..d2605235904 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 74ac811cef3..3c23a86d12c 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("") } } - - } From f91e1c48224366c9cf7a32e99d67e966e540b86b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 21 Jun 2013 15:11:42 -0700 Subject: [PATCH 0150/2521] 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 0174e1ddd15..8ebde7a3796 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 7628fde4aa4..466f4643bc5 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] = { From 9a24d1a2d0deb5cbd96b806a1ef3afb9353872dd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 21 Jun 2013 15:17:50 -0700 Subject: [PATCH 0151/2521] 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 a6aba59e9fa..9e7b2bfd929 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 93e7129fc01..726a95594b7 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 8ebde7a3796..87f3abc6b04 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 3c23a86d12c..d76c07e458d 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 1f9953fb6b7..52b90870992 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 466f4643bc5..1031b2e17f0 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 From be6107ce446a25f6fccaf34085beb23e81828495 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 21 Jun 2013 21:30:56 -0700 Subject: [PATCH 0152/2521] 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 +- .../main/scala/spark/ui/jobs/JobProgressUI.scala | 2 +- core/src/main/scala/spark/ui/jobs/StagePage.scala | 2 +- .../main/scala/spark/ui/storage/IndexPage.scala | 2 +- .../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 726a95594b7..225daf57787 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 deb6284a666..cfa805fcf81 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 87f3abc6b04..5a4658499ab 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 d2605235904..47a6bcc2c31 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 d76c07e458d..42eb1f9eef1 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 52b90870992..cd7f6bd9f25 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 1031b2e17f0..059affedbd2 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] = { From 3b7ebdeeb86868ff1ebd269c2ed832012f22e0d1 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 22 Jun 2013 09:00:42 -0700 Subject: [PATCH 0153/2521] Handling entirely failed stages --- .../main/scala/spark/ui/jobs/IndexPage.scala | 18 +++++--- .../scala/spark/ui/jobs/JobProgressUI.scala | 43 ++++++++++++------- .../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 5a4658499ab..2e2dcdfbc6e 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 47a6bcc2c31..dea4f0e3b0c 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 059affedbd2..73f151959e5 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) } From 7e9f1ed0decbbb261432dcbcd2179b829dbcbc82 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 22 Jun 2013 10:17:29 -0700 Subject: [PATCH 0154/2521] 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 225daf57787..96d773979fe 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 cfa805fcf81..66dcf073843 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) From b5df1cd668e45fd0cc22c1666136d05548cae3e9 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 17:12:39 -0700 Subject: [PATCH 0155/2521] 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 b0da130fcb3..e9cf9ef36fc 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 23556dbc8f3..86eed090d0e 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() From 0e0f9d3069039f03bbf5eefe3b0637c89fddf0f1 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 17:44:04 -0700 Subject: [PATCH 0156/2521] Fix search path for REPL class loader to really find added JARs --- .../main/scala/spark/executor/Executor.scala | 38 +++++++++++-------- .../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 8bebfafce44..2bf55ea9a90 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 86eed090d0e..59f9d05683d 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 } From 78ffe164b33c6b11a2e511442605acd2f795a1b5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 23 Jun 2013 10:07:16 -0700 Subject: [PATCH 0157/2521] 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 --- .../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 fa4bbfc76f3..7630fe78035 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 1916885a738..0c1ec29f96c 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 { From 8955787a596216a35ad4ec52b57331aa40444bef Mon Sep 17 00:00:00 2001 From: James Phillpotts Date: Mon, 24 Jun 2013 09:15:17 +0100 Subject: [PATCH 0158/2521] Twitter API v1 is retired - username/password auth no longer possible --- .../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 f97e47ada0d..05be6bd58ab 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 From f6e64b5cd6e6ac5ae3bee05af2832e1f71992310 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 22 Jun 2013 10:40:41 -0700 Subject: [PATCH 0159/2521] Updating based on changes to JobLogger (and one small change to JobLogger) --- .../scala/spark/scheduler/DAGScheduler.scala | 6 +++--- .../scala/spark/scheduler/JobLogger.scala | 2 +- .../scala/spark/scheduler/JobResult.scala | 2 +- .../scala/spark/scheduler/JobWaiter.scala | 2 +- .../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 bdd8792ce96..82d419453b5 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 178bfaba3d9..6a9d52f3561 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 654131ee841..a0fdf391e68 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 3cc6a86345f..6ff2e294345 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 dea4f0e3b0c..70f8e431f2b 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 _ => } } From 93e8ed85aa1fbcb6428934b30d01f2b4090538b9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Jun 2013 13:11:18 -0700 Subject: [PATCH 0160/2521] 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 901cda4174f..1f420d73f79 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 66dcf073843..2d5a3280159 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 2e2dcdfbc6e..134c93091d5 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 70f8e431f2b..99f9f2d9f6c 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 42eb1f9eef1..8a488498d98 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] = { From b5e6e8bcc8ca05cd97f35a502a89c686aa4a5a12 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Jun 2013 14:13:24 -0700 Subject: [PATCH 0161/2521] Cleaning up some code for Job Progress --- .../main/scala/spark/ui/jobs/IndexPage.scala | 4 +- .../scala/spark/ui/jobs/JobProgressUI.scala | 9 +++-- .../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 134c93091d5..c7ee9dc1c13 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 99f9f2d9f6c..027eadde3ab 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 8a488498d98..65dbd389b1b 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("") } } } From a4248138b4c4b374e61f474886463c58f5bb3915 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Jun 2013 14:22:28 -0700 Subject: [PATCH 0162/2521] 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 96d773979fe..b5270e60621 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}

    From 4cda8f865a003ab354890c4915ea1b5a7674f5b0 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 24 Jun 2013 15:14:48 -0700 Subject: [PATCH 0163/2521] 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 26b5b9d4620..1082c09eb10 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` From 0bcaf036050c3d2b4389339927239e0e35bf02ff Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 24 Jun 2013 15:39:52 -0700 Subject: [PATCH 0164/2521] 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 c0065c53f17..30a2885a4dc 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 00000000000..4d09bd4416e --- /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 From 42157027f247035e9ae41efe899e27c0942f5cd8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Jun 2013 16:25:05 -0700 Subject: [PATCH 0165/2521] A few bug fixes and a unit test --- core/src/main/scala/spark/SparkContext.scala | 1 + .../main/scala/spark/deploy/master/Master.scala | 9 +++++---- .../spark/deploy/master/ui/MasterWebUI.scala | 10 ++++++++-- .../main/scala/spark/deploy/worker/Worker.scala | 9 +++------ .../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 1f420d73f79..500d25efdd6 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 3eca522c7ef..2a49dfb4867 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 5fd17f10c6c..065b62a4b20 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 690bdfe128d..28553b6c02d 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 abfc847527b..ee3889192d7 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 b5270e60621..85d6a7e8674 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 2d5a3280159..487f0056157 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 6ec89c01840..d05cf3dec1e 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 0866fb47b39..0024ede828a 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 00000000000..6766b158f6a --- /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) + } + +} From f7389330c35a6372c4c7b455b96b8498b9a9da27 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Jun 2013 16:51:52 -0700 Subject: [PATCH 0166/2521] Allowing for requested port on construction --- .../main/scala/spark/deploy/master/Master.scala | 2 +- .../scala/spark/deploy/master/ui/MasterWebUI.scala | 11 +++++++---- .../main/scala/spark/deploy/worker/Worker.scala | 6 +++--- .../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 2a49dfb4867..127be9e7d49 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 065b62a4b20..50e93cd00f9 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 28553b6c02d..b6a26245fc9 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 ee3889192d7..2615458deeb 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 85d6a7e8674..1f665cbb42a 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)) From 81df20e5b44407c8c321471be2faee4bf6853fd6 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 24 Jun 2013 17:05:37 -0700 Subject: [PATCH 0167/2521] 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 00000000000..34f6d4a36b6 --- /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 From d66bd6f8851e87c8f35cca86ee45e26a8f547040 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Jun 2013 17:08:58 -0700 Subject: [PATCH 0168/2521] Adding another unit test to Web UI suite --- .../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 50e93cd00f9..dfefd6571b7 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 1f665cbb42a..45c0805b22c 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 6766b158f6a..7df7356a6e7 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) => + } + } } From 48c7e373c62b2e8cf48157ceb0d92c38c3a40652 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 24 Jun 2013 23:11:04 -0700 Subject: [PATCH 0169/2521] Minor formatting fixes --- .../main/scala/spark/streaming/DStream.scala | 9 ++++-- .../spark/streaming/StreamingContext.scala | 29 +++++++++++-------- .../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 e125310861e..9be7926a4a8 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 2c6326943dc..03d2907323c 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 b35d9032f1d..fd5e06b50f9 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, From cee05a174897af294f52cbda551da09cb869557e Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Mon, 24 Jun 2013 23:55:09 -0700 Subject: [PATCH 0170/2521] 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 574ae2104d8..afbb7a9a8ea 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 From c3d11d0d57739ec3f08783f71bf4d0efdec3d627 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 25 Jun 2013 00:02:00 -0700 Subject: [PATCH 0171/2521] Get rid of debugging statements --- run | 3 --- 1 file changed, 3 deletions(-) diff --git a/run b/run index 30a2885a4dc..646d12c1ebb 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 From 243d71cb066d888f3d1a9c613859522438b69ba8 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 25 Jun 2013 00:15:58 -0700 Subject: [PATCH 0172/2521] Add deploy/testing procedure --- make-distribution.sh | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/make-distribution.sh b/make-distribution.sh index 34f6d4a36b6..855475864d0 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)" From 982a686ff858399067de960fa62cc80f60c6fa32 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 25 Jun 2013 11:14:21 -0700 Subject: [PATCH 0173/2521] 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 afbb7a9a8ea..ea67a3e6b8c 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. From 7680ce0bd65fc44716c5bc03d5909a3ddbd43501 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 16:11:44 -0400 Subject: [PATCH 0174/2521] Fixed deprecated use of expect in SizeEstimatorSuite --- .../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 e235ef2f67a..b5c8525f914 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) From 15b00914c53f1f4f00a3313968f68a8f032e7cb7 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 17:17:27 -0400 Subject: [PATCH 0175/2521] 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 ++++++++++++++++++- .../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 f3621c6beee..bdc1494cc9d 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 4d31657d9e3..db580e39abd 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 ed4701574fd..4a113e16bf5 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("")) } } From 366572edcab87701fd795ca0142ac9829b312d36 Mon Sep 17 00:00:00 2001 From: James Phillpotts Date: Tue, 25 Jun 2013 22:59:34 +0100 Subject: [PATCH 0176/2521] Include a default OAuth implementation, and update examples and JavaStreamingContext --- .../examples/TwitterAlgebirdCMS.scala | 2 +- .../examples/TwitterAlgebirdHLL.scala | 2 +- .../examples/TwitterPopularTags.scala | 2 +- .../spark/streaming/StreamingContext.scala | 2 +- .../api/java/JavaStreamingContext.scala | 69 +++++++++++++------ .../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 a9642100e3d..548190309e9 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 f3288bfb854..5a86c6318d6 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 9d4494c6f27..076c3878c81 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 05be6bd58ab..0f36504c0d8 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 3d149a742cb..85390ef57e1 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 0b01091a520..e0c654d385f 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) } } From 8dd78f80b578751df164772a01772aad26540ac9 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 25 Jun 2013 15:01:06 -0700 Subject: [PATCH 0177/2521] 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 4d09bd4416e..d031c56baff 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 From 176193b1e8acdbe2f1cfaed16b8f42f89e226f79 Mon Sep 17 00:00:00 2001 From: James Phillpotts Date: Tue, 25 Jun 2013 23:06:15 +0100 Subject: [PATCH 0178/2521] Fix usage and parameter extraction --- .../spark/streaming/examples/TwitterAlgebirdCMS.scala | 7 +++---- .../spark/streaming/examples/TwitterAlgebirdHLL.scala | 7 +++---- .../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 548190309e9..528778ed722 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 5a86c6318d6..896e9fd8af1 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 076c3878c81..65f0b6d3525 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"))) From 6c8d1b2ca618a1a17566ede46821c0807a1b11f5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 18:21:00 -0400 Subject: [PATCH 0179/2521] 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 +++++++ .../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 00000000000..1dff8fea220 --- /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 00000000000..3a788802904 --- /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 bdc1494cc9d..f41efa9d299 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 db580e39abd..4f8e1dcb260 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 0fb15f8b24c..7c06a550622 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 bf76844d113..25e4f3b57c1 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: From f2263350eda780aba45f383b722e20702c310e6a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 18:35:35 -0400 Subject: [PATCH 0180/2521] Added a local-cluster mode test to ReplSuite --- .../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 1c64f9b98d0..72ed8aca5b6 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", """ From 2bd04c3513ffa6deabc290a3931be946b1c18713 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 18:37:14 -0400 Subject: [PATCH 0181/2521] 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 72ed8aca5b6..f46e6d8be47 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) } From 9f0d91329516c829c86fc8e95d02071ca7d1f186 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 19:18:30 -0400 Subject: [PATCH 0182/2521] 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. --- .../test/scala/spark/CheckpointSuite.scala | 10 + .../scala/spark/PairRDDFunctionsSuite.scala | 287 +++++++++++++++++ .../test/scala/spark/PartitioningSuite.scala | 19 +- core/src/test/scala/spark/PipedRDDSuite.scala | 16 +- core/src/test/scala/spark/RDDSuite.scala | 87 +---- .../test/scala/spark/SharedSparkContext.scala | 25 ++ core/src/test/scala/spark/ShuffleSuite.scala | 298 +----------------- core/src/test/scala/spark/SortingSuite.scala | 23 +- .../src/test/scala/spark/UnpersistSuite.scala | 30 ++ .../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 ca385972fb2..28a7b21b92b 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 00000000000..682d2745bff --- /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 16f93e71a37..99e433e3bd6 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 ed075f93ec5..1c9ca508112 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 67f3332d444..d8db69b1c91 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 00000000000..1da79f9824d --- /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 0c1ec29f96c..950218fa280 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 495f957e53f..f7bf207c68c 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 00000000000..94776e75720 --- /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 5f60aa75d7f..96cb295f45b 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) From 32370da4e40062b88c921417cd7418d804e16f23 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 25 Jun 2013 22:08:19 -0400 Subject: [PATCH 0183/2521] 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 484f97d9922..07572201ded 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 } From d11025dc6aedd9763cfd2390e8daf24747d17258 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Wed, 26 Jun 2013 09:53:35 -0500 Subject: [PATCH 0184/2521] Be cute with Option and getenv. --- .../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 4f8e1dcb260..a9b12421e6c 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 From 2cbaa0734b1a23e5d6ce1543a22358f436b5b1ef Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 26 Jun 2013 08:44:55 -0700 Subject: [PATCH 0185/2521] Making all new classes package private --- core/src/main/scala/spark/deploy/JsonProtocol.scala | 2 +- .../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 b4365d31e92..335aaeb769b 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 2a810b71d21..9cb9c94e9f0 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 f833c59de8d..3c3679a4b04 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 dfefd6571b7..0cfe6000434 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 9e7b2bfd929..c1ca09fc977 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 2615458deeb..65bdba4735b 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 45c0805b22c..1f311bb75ff 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 487f0056157..86b394c09fb 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 c7ee9dc1c13..152e1a79b04 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 65dbd389b1b..03200861e0d 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 4f1928fac98..3d67888e2cb 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 cd7f6bd9f25..772c669a1ad 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 7df7356a6e7..6519bf77c6b 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) From b14ad509ba4c00d57bd4a52a7b70a18325064a7b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 26 Jun 2013 08:46:51 -0700 Subject: [PATCH 0186/2521] Moving static ui package --- .../static/bootstrap-responsive.min.css | 0 .../spark/{webui => ui}/static/bootstrap.min.css | 0 .../spark/{webui => ui}/static/sorttable.js | 0 .../spark/{webui => ui}/static/spark_logo.png | Bin .../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 +- 7 files changed, 3 insertions(+), 3 deletions(-) rename core/src/main/resources/spark/{webui => ui}/static/bootstrap-responsive.min.css (100%) rename core/src/main/resources/spark/{webui => ui}/static/bootstrap.min.css (100%) rename core/src/main/resources/spark/{webui => ui}/static/sorttable.js (100%) rename core/src/main/resources/spark/{webui => ui}/static/spark_logo.png (100%) diff --git a/core/src/main/resources/spark/webui/static/bootstrap-responsive.min.css b/core/src/main/resources/spark/ui/static/bootstrap-responsive.min.css similarity index 100% rename from core/src/main/resources/spark/webui/static/bootstrap-responsive.min.css rename to core/src/main/resources/spark/ui/static/bootstrap-responsive.min.css diff --git a/core/src/main/resources/spark/webui/static/bootstrap.min.css b/core/src/main/resources/spark/ui/static/bootstrap.min.css similarity index 100% rename from core/src/main/resources/spark/webui/static/bootstrap.min.css rename to core/src/main/resources/spark/ui/static/bootstrap.min.css diff --git a/core/src/main/resources/spark/webui/static/sorttable.js b/core/src/main/resources/spark/ui/static/sorttable.js similarity index 100% rename from core/src/main/resources/spark/webui/static/sorttable.js rename to core/src/main/resources/spark/ui/static/sorttable.js diff --git a/core/src/main/resources/spark/webui/static/spark_logo.png b/core/src/main/resources/spark/ui/static/spark_logo.png similarity index 100% rename from core/src/main/resources/spark/webui/static/spark_logo.png rename to core/src/main/resources/spark/ui/static/spark_logo.png 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 0cfe6000434..82bc6961e26 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -54,6 +54,6 @@ class MasterWebUI(val master: ActorRef, requestedPort: Option[Int] = None) exten } private[spark] object MasterWebUI { - val STATIC_RESOURCE_DIR = "spark/webui/static" + 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/ui/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala index 65bdba4735b..2e6279566c5 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -69,6 +69,6 @@ class WorkerWebUI(val worker: ActorRef, val workDir: File, requestedPort: Option } private[spark] object WorkerWebUI { - val STATIC_RESOURCE_DIR = "spark/webui/static" + val STATIC_RESOURCE_DIR = "spark/ui/static" val DEFAULT_PORT="8081" } diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 86b394c09fb..b3bdc2c4902 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -59,5 +59,5 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { private[spark] object SparkUI { val DEFAULT_PORT = "33000" - val STATIC_RESOURCE_DIR = "spark/webui/static" + val STATIC_RESOURCE_DIR = "spark/ui/static" } From 274193664ae13adbe5215f5dcc2be11e672557b6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 26 Jun 2013 08:49:58 -0700 Subject: [PATCH 0187/2521] Bumping timeouts --- .../main/scala/spark/deploy/master/ui/ApplicationPage.scala | 4 ++-- core/src/main/scala/spark/deploy/master/ui/IndexPage.scala | 2 +- core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala | 4 ++-- 3 files changed, 5 insertions(+), 5 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 9cb9c94e9f0..fb2c803a912 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -22,7 +22,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { 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 state = Await.result(stateFuture, 30 seconds) val app = state.activeApps.find(_.id == appId).getOrElse({ state.completedApps.find(_.id == appId).getOrElse(null) }) @@ -33,7 +33,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { 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 state = Await.result(stateFuture, 30 seconds) val app = state.activeApps.find(_.id == appId).getOrElse({ state.completedApps.find(_.id == appId).getOrElse(null) }) 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 3c3679a4b04..4892f258e1a 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -20,7 +20,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { /** 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 state = Await.result(stateFuture, 30 seconds) val workerHeaders = Seq("Id", "Address", "State", "Cores", "Memory") val workers = state.workers.sortBy(_.id) 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 c1ca09fc977..e95a74a37f8 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -21,13 +21,13 @@ private[spark] class IndexPage(parent: WorkerWebUI) { def renderJson(request: HttpServletRequest): JValue = { val stateFuture = (worker ? RequestWorkerState)(timeout).mapTo[WorkerState] - val workerState = Await.result(stateFuture, 3 seconds) + val workerState = Await.result(stateFuture, 30 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 workerState = Await.result(stateFuture, 30 seconds) val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") val runningExecutorTable = From a59c15a37eba0b585ba49a1b1b2b309f71e56d02 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 26 Jun 2013 08:54:57 -0700 Subject: [PATCH 0188/2521] Adding config option for retained stages --- core/src/main/scala/spark/ui/jobs/JobProgressUI.scala | 2 +- docs/configuration.md | 11 +++++++++-- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 027eadde3ab..46ba1d31af0 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -42,7 +42,7 @@ private[spark] class JobProgressUI(val sc: SparkContext) { private[spark] class JobProgressListener extends SparkListener { // How many stages to remember - val RETAINED_STAGES = 1000 + val RETAINED_STAGES = System.getProperty("spark.ui.retained_stages", "1000").toInt val activeStages = HashSet[Stage]() val completedStages = ListBuffer[Stage]() diff --git a/docs/configuration.md b/docs/configuration.md index 17fdbf04d1c..00a2dd73694 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -135,9 +135,16 @@ Apart from these, the following properties are also available, and may be useful spark.ui.port - (random) + 33000 + + Port for your application's dashboard, which shows memory and workload data + + + + spark.ui.retained_stages + 1000 - Port for your application's dashboard, which shows memory usage of each RDD. + How many stages the Spark UI remembers before garbage collecting. From ee692482a609b5d42f7270b020d6f38fb5e466fd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 26 Jun 2013 09:07:32 -0700 Subject: [PATCH 0189/2521] One more private class --- core/src/main/scala/spark/ui/storage/RDDPage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 73f151959e5..dd9fd2e4c54 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -10,7 +10,7 @@ import spark.Utils import spark.storage.BlockManagerMasterActor.BlockStatus /** Page showing storage details for a given RDD */ -class RDDPage(parent: BlockManagerUI) { +private[spark] class RDDPage(parent: BlockManagerUI) { val sc = parent.sc def render(request: HttpServletRequest): Seq[Node] = { From d7011632d15b6de2129d360277e304fddb8f2aac Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Wed, 26 Jun 2013 12:35:57 -0500 Subject: [PATCH 0190/2521] Wrap lines. --- core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index a9b12421e6c..d7f58b2cb1b 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -90,7 +90,9 @@ private[spark] class ExecutorRunner( * the way the JAVA_OPTS are assembled there. */ def buildJavaOpts(): Seq[String] = { - val libraryOpts = Option(getenv("SPARK_LIBRARY_PATH")).map(p => List("-Djava.library.path=" + p)).getOrElse(Nil) + 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") From 13e3dd98d86a89f9b2d2e6c9a8f03bd0b79fba70 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 26 Jun 2013 11:15:48 -0700 Subject: [PATCH 0191/2521] Removing mesos support --- .../root/mesos-ec2/ec2-variables.sh | 9 -- ec2/spark_ec2.py | 130 ++++-------------- 2 files changed, 30 insertions(+), 109 deletions(-) delete mode 100644 ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh diff --git a/ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh b/ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh deleted file mode 100644 index 50ecf834045..00000000000 --- a/ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh +++ /dev/null @@ -1,9 +0,0 @@ -#!/bin/bash - -# These variables are automatically filled in by the mesos-ec2 script. -export MESOS_MASTERS="{{master_list}}" -export MESOS_SLAVES="{{slave_list}}" -export MESOS_ZOO_LIST="{{zoo_list}}" -export MESOS_HDFS_DATA_DIRS="{{hdfs_data_dirs}}" -export MESOS_MAPRED_LOCAL_DIRS="{{mapred_local_dirs}}" -export MESOS_SPARK_LOCAL_DIRS="{{spark_local_dirs}}" diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 0b4dc2077ea..d8890f26f4e 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -94,17 +94,11 @@ def parse_args(): parser.add_option("--spot-price", metavar="PRICE", type="float", help="If specified, launch slaves as spot instances with the given " + "maximum price (in dollars)") - parser.add_option("--cluster-type", type="choice", metavar="TYPE", - choices=["mesos", "standalone"], default="standalone", - help="'mesos' for a Mesos cluster, 'standalone' for a standalone " + - "Spark cluster (default: standalone)") parser.add_option("--ganglia", action="store_true", default=True, help="Setup Ganglia monitoring on cluster (default: on). NOTE: " + "the Ganglia page will be publicly accessible") parser.add_option("--no-ganglia", action="store_false", dest="ganglia", help="Disable Ganglia monitoring for the cluster") - parser.add_option("--old-scripts", action="store_true", default=False, - help="Use old mesos-ec2 scripts, for Spark <= 0.6 AMIs") parser.add_option("-u", "--user", default="root", help="The SSH user you want to connect as (default: root)") parser.add_option("--delete-groups", action="store_true", default=False, @@ -119,9 +113,6 @@ def parse_args(): print >> stderr, ("ERROR: The -i or --identity-file argument is " + "required for " + action) sys.exit(1) - if opts.cluster_type not in ["mesos", "standalone"] and action == "launch": - print >> stderr, ("ERROR: Invalid cluster type: " + opts.cluster_type) - sys.exit(1) # Boto config check # http://boto.cloudhackers.com/en/latest/boto_config_tut.html @@ -219,52 +210,38 @@ def get_spark_ami(opts): # Launch a cluster of the given name, by setting up its security groups, # and then starting new instances in them. -# Returns a tuple of EC2 reservation objects for the master, slave -# and zookeeper instances (in that order). +# Returns a tuple of EC2 reservation objects for the master and slaves # Fails if there already instances running in the cluster's groups. def launch_cluster(conn, opts, cluster_name): print "Setting up security groups..." master_group = get_or_make_group(conn, cluster_name + "-master") slave_group = get_or_make_group(conn, cluster_name + "-slaves") - zoo_group = get_or_make_group(conn, cluster_name + "-zoo") if master_group.rules == []: # Group was just now created master_group.authorize(src_group=master_group) master_group.authorize(src_group=slave_group) - master_group.authorize(src_group=zoo_group) master_group.authorize('tcp', 22, 22, '0.0.0.0/0') master_group.authorize('tcp', 8080, 8081, '0.0.0.0/0') master_group.authorize('tcp', 50030, 50030, '0.0.0.0/0') master_group.authorize('tcp', 50070, 50070, '0.0.0.0/0') master_group.authorize('tcp', 60070, 60070, '0.0.0.0/0') - if opts.cluster_type == "mesos": - master_group.authorize('tcp', 38090, 38090, '0.0.0.0/0') if opts.ganglia: master_group.authorize('tcp', 5080, 5080, '0.0.0.0/0') if slave_group.rules == []: # Group was just now created slave_group.authorize(src_group=master_group) slave_group.authorize(src_group=slave_group) - slave_group.authorize(src_group=zoo_group) slave_group.authorize('tcp', 22, 22, '0.0.0.0/0') slave_group.authorize('tcp', 8080, 8081, '0.0.0.0/0') slave_group.authorize('tcp', 50060, 50060, '0.0.0.0/0') slave_group.authorize('tcp', 50075, 50075, '0.0.0.0/0') slave_group.authorize('tcp', 60060, 60060, '0.0.0.0/0') slave_group.authorize('tcp', 60075, 60075, '0.0.0.0/0') - if zoo_group.rules == []: # Group was just now created - zoo_group.authorize(src_group=master_group) - zoo_group.authorize(src_group=slave_group) - zoo_group.authorize(src_group=zoo_group) - zoo_group.authorize('tcp', 22, 22, '0.0.0.0/0') - zoo_group.authorize('tcp', 2181, 2181, '0.0.0.0/0') - zoo_group.authorize('tcp', 2888, 2888, '0.0.0.0/0') - zoo_group.authorize('tcp', 3888, 3888, '0.0.0.0/0') # Check if instances are already running in our groups active_nodes = get_existing_cluster(conn, opts, cluster_name, die_on_error=False) if any(active_nodes): print >> stderr, ("ERROR: There are already instances running in " + - "group %s, %s or %s" % (master_group.name, slave_group.name, zoo_group.name)) + "group %s, %s or %s" % (master_group.name, slave_group.name)) sys.exit(1) # Figure out Spark AMI @@ -336,9 +313,9 @@ def launch_cluster(conn, opts, cluster_name): print "Canceling spot instance requests" conn.cancel_spot_instance_requests(my_req_ids) # Log a warning if any of these requests actually launched instances: - (master_nodes, slave_nodes, zoo_nodes) = get_existing_cluster( + (master_nodes, slave_nodes) = get_existing_cluster( conn, opts, cluster_name, die_on_error=False) - running = len(master_nodes) + len(slave_nodes) + len(zoo_nodes) + running = len(master_nodes) + len(slave_nodes) if running: print >> stderr, ("WARNING: %d instances are still running" % running) sys.exit(0) @@ -379,21 +356,17 @@ def launch_cluster(conn, opts, cluster_name): master_nodes = master_res.instances print "Launched master in %s, regid = %s" % (zone, master_res.id) - zoo_nodes = [] - # Return all the instances - return (master_nodes, slave_nodes, zoo_nodes) + return (master_nodes, slave_nodes) # Get the EC2 instances in an existing cluster if available. -# Returns a tuple of lists of EC2 instance objects for the masters, -# slaves and zookeeper nodes (in that order). +# Returns a tuple of lists of EC2 instance objects for the masters and slaves def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): print "Searching for existing cluster " + cluster_name + "..." reservations = conn.get_all_instances() master_nodes = [] slave_nodes = [] - zoo_nodes = [] for res in reservations: active = [i for i in res.instances if is_active(i)] if len(active) > 0: @@ -402,13 +375,11 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): master_nodes += res.instances elif group_names == [cluster_name + "-slaves"]: slave_nodes += res.instances - elif group_names == [cluster_name + "-zoo"]: - zoo_nodes += res.instances - if any((master_nodes, slave_nodes, zoo_nodes)): - print ("Found %d master(s), %d slaves, %d ZooKeeper nodes" % - (len(master_nodes), len(slave_nodes), len(zoo_nodes))) + if any((master_nodes, slave_nodes)): + print ("Found %d master(s), %d slaves" % + (len(master_nodes), len(slave_nodes))) if (master_nodes != [] and slave_nodes != []) or not die_on_error: - return (master_nodes, slave_nodes, zoo_nodes) + return (master_nodes, slave_nodes) else: if master_nodes == [] and slave_nodes != []: print "ERROR: Could not find master in group " + cluster_name + "-master" @@ -421,7 +392,7 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): # Deploy configuration files and run setup scripts on a newly launched # or started EC2 cluster. -def setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, deploy_ssh_key): +def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): master = master_nodes[0].public_dns_name if deploy_ssh_key: print "Copying SSH key %s to master..." % opts.identity_file @@ -429,12 +400,8 @@ def setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, deploy_ssh_k scp(master, opts, opts.identity_file, '~/.ssh/id_rsa') ssh(master, opts, 'chmod 600 ~/.ssh/id_rsa') - if opts.cluster_type == "mesos": - modules = ['spark', 'shark', 'ephemeral-hdfs', 'persistent-hdfs', - 'mapreduce', 'mesos'] - elif opts.cluster_type == "standalone": - modules = ['spark', 'shark', 'ephemeral-hdfs', 'persistent-hdfs', - 'mapreduce', 'spark-standalone'] + modules = ['spark', 'shark', 'ephemeral-hdfs', 'persistent-hdfs', + 'mapreduce', 'spark-standalone'] if opts.ganglia: modules.append('ganglia') @@ -446,18 +413,12 @@ def setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, deploy_ssh_k ssh(master, opts, "rm -rf spark-ec2 && git clone https://github.com/pwendell/spark-ec2.git -b ec2-updates") print "Deploying files to master..." - deploy_files(conn, "deploy.generic", opts, master_nodes, slave_nodes, - zoo_nodes, modules) + deploy_files(conn, "deploy.generic", opts, master_nodes, slave_nodes, modules) print "Running setup on master..." setup_spark_cluster(master, opts) print "Done!" -def setup_mesos_cluster(master, opts): - ssh(master, opts, "chmod u+x mesos-ec2/setup") - ssh(master, opts, "mesos-ec2/setup %s %s %s %s" % - ("generic", "none", "master", opts.swap)) - def setup_standalone_cluster(master, slave_nodes, opts): slave_ips = '\n'.join([i.public_dns_name for i in slave_nodes]) ssh(master, opts, "echo \"%s\" > spark/conf/slaves" % (slave_ips)) @@ -466,23 +427,18 @@ def setup_standalone_cluster(master, slave_nodes, opts): def setup_spark_cluster(master, opts): ssh(master, opts, "chmod u+x spark-ec2/setup.sh") ssh(master, opts, "spark-ec2/setup.sh") - if opts.cluster_type == "mesos": - print "Mesos cluster started at http://%s:8080" % master - elif opts.cluster_type == "standalone": - print "Spark standalone cluster started at http://%s:8080" % master + print "Spark standalone cluster started at http://%s:8080" % master if opts.ganglia: print "Ganglia started at http://%s:5080/ganglia" % master # Wait for a whole cluster (masters, slaves and ZooKeeper) to start up -def wait_for_cluster(conn, wait_secs, master_nodes, slave_nodes, zoo_nodes): +def wait_for_cluster(conn, wait_secs, master_nodes, slave_nodes): print "Waiting for instances to start up..." time.sleep(5) wait_for_instances(conn, master_nodes) wait_for_instances(conn, slave_nodes) - if zoo_nodes != []: - wait_for_instances(conn, zoo_nodes) print "Waiting %d more seconds..." % wait_secs time.sleep(wait_secs) @@ -523,8 +479,7 @@ def get_num_disks(instance_type): # cluster (e.g. lists of masters and slaves). Files are only deployed to # the first master instance in the cluster, and we expect the setup # script to be run on that instance to copy them to other nodes. -def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, zoo_nodes, - modules): +def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): active_master = master_nodes[0].public_dns_name num_disks = get_num_disks(opts.instance_type) @@ -537,16 +492,7 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, zoo_nodes, mapred_local_dirs += ",/mnt%d/hadoop/mrlocal" % i spark_local_dirs += ",/mnt%d/spark" % i - if zoo_nodes != []: - zoo_list = '\n'.join([i.public_dns_name for i in zoo_nodes]) - cluster_url = "zoo://" + ",".join( - ["%s:2181/mesos" % i.public_dns_name for i in zoo_nodes]) - elif opts.cluster_type == "mesos": - zoo_list = "NONE" - cluster_url = "%s:5050" % active_master - elif opts.cluster_type == "standalone": - zoo_list = "NONE" - cluster_url = "%s:7077" % active_master + cluster_url = "%s:7077" % active_master if "." in opts.spark_version: (spark_v, shark_v) = get_spark_shark_version(opts) @@ -558,7 +504,6 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, zoo_nodes, "master_list": '\n'.join([i.public_dns_name for i in master_nodes]), "active_master": active_master, "slave_list": '\n'.join([i.public_dns_name for i in slave_nodes]), - "zoo_list": zoo_list, "cluster_url": cluster_url, "hdfs_data_dirs": hdfs_data_dirs, "mapred_local_dirs": mapred_local_dirs, @@ -656,20 +601,20 @@ def main(): if action == "launch": if opts.resume: - (master_nodes, slave_nodes, zoo_nodes) = get_existing_cluster( + (master_nodes, slave_nodes) = get_existing_cluster( conn, opts, cluster_name) else: - (master_nodes, slave_nodes, zoo_nodes) = launch_cluster( + (master_nodes, slave_nodes) = launch_cluster( conn, opts, cluster_name) - wait_for_cluster(conn, opts.wait, master_nodes, slave_nodes, zoo_nodes) - setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, True) + wait_for_cluster(conn, opts.wait, master_nodes, slave_nodes) + setup_cluster(conn, master_nodes, slave_nodes, opts, True) elif action == "destroy": response = raw_input("Are you sure you want to destroy the cluster " + cluster_name + "?\nALL DATA ON ALL NODES WILL BE LOST!!\n" + "Destroy cluster " + cluster_name + " (y/N): ") if response == "y": - (master_nodes, slave_nodes, zoo_nodes) = get_existing_cluster( + (master_nodes, slave_nodes) = get_existing_cluster( conn, opts, cluster_name, die_on_error=False) print "Terminating master..." for inst in master_nodes: @@ -677,15 +622,11 @@ def main(): print "Terminating slaves..." for inst in slave_nodes: inst.terminate() - if zoo_nodes != []: - print "Terminating zoo..." - for inst in zoo_nodes: - inst.terminate() # Delete security groups as well if opts.delete_groups: print "Deleting security groups (this will take some time)..." - group_names = [cluster_name + "-master", cluster_name + "-slaves", cluster_name + "-zoo"] + group_names = [cluster_name + "-master", cluster_name + "-slaves"] attempt = 1; while attempt <= 3: @@ -725,7 +666,7 @@ def main(): print "Try re-running in a few minutes." elif action == "login": - (master_nodes, slave_nodes, zoo_nodes) = get_existing_cluster( + (master_nodes, slave_nodes) = get_existing_cluster( conn, opts, cluster_name) master = master_nodes[0].public_dns_name print "Logging into master " + master + "..." @@ -736,7 +677,7 @@ def main(): (opts.identity_file, proxy_opt, opts.user, master), shell=True) elif action == "get-master": - (master_nodes, slave_nodes, zoo_nodes) = get_existing_cluster(conn, opts, cluster_name) + (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) print master_nodes[0].public_dns_name elif action == "stop": @@ -746,7 +687,7 @@ def main(): "AMAZON EBS IF IT IS EBS-BACKED!!\n" + "Stop cluster " + cluster_name + " (y/N): ") if response == "y": - (master_nodes, slave_nodes, zoo_nodes) = get_existing_cluster( + (master_nodes, slave_nodes) = get_existing_cluster( conn, opts, cluster_name, die_on_error=False) print "Stopping master..." for inst in master_nodes: @@ -756,15 +697,9 @@ def main(): for inst in slave_nodes: if inst.state not in ["shutting-down", "terminated"]: inst.stop() - if zoo_nodes != []: - print "Stopping zoo..." - for inst in zoo_nodes: - if inst.state not in ["shutting-down", "terminated"]: - inst.stop() elif action == "start": - (master_nodes, slave_nodes, zoo_nodes) = get_existing_cluster( - conn, opts, cluster_name) + (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) print "Starting slaves..." for inst in slave_nodes: if inst.state not in ["shutting-down", "terminated"]: @@ -773,13 +708,8 @@ def main(): for inst in master_nodes: if inst.state not in ["shutting-down", "terminated"]: inst.start() - if zoo_nodes != []: - print "Starting zoo..." - for inst in zoo_nodes: - if inst.state not in ["shutting-down", "terminated"]: - inst.start() - wait_for_cluster(conn, opts.wait, master_nodes, slave_nodes, zoo_nodes) - setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, False) + wait_for_cluster(conn, opts.wait, master_nodes, slave_nodes) + setup_cluster(conn, master_nodes, slave_nodes, opts, False) else: print >> stderr, "Invalid action: %s" % action From 03906f7f0a8c93b09e3b47ccaad5b5f72c29302b Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 26 Jun 2013 17:40:22 -0700 Subject: [PATCH 0192/2521] Fixes to compute-classpath on Windows --- bin/compute-classpath.cmd | 4 ++-- run2.cmd | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 1dff8fea220..6e7efbd3349 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -6,7 +6,7 @@ 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\.. +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" @@ -45,7 +45,7 @@ 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 +if "%DONT_PRINT_CLASSPATH%"=="1" goto exit echo %CLASSPATH% diff --git a/run2.cmd b/run2.cmd index 25e4f3b57c1..a9c4df180ff 100644 --- a/run2.cmd +++ b/run2.cmd @@ -48,6 +48,7 @@ set REPL_DIR=%FWDIR%repl rem Compute classpath using external script set DONT_PRINT_CLASSPATH=1 call "%FWDIR%bin\compute-classpath.cmd" +set DONT_PRINT_CLASSPATH=0 rem Figure out the JAR file that our examples were packaged into. rem First search in the build path from SBT: From aea727f68d5fe5e81fc04ece97ad94c6f12c7270 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 26 Jun 2013 21:14:38 -0700 Subject: [PATCH 0193/2521] Simplify Python docs a little to do substring search --- docs/python-programming-guide.md | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 3a7a8db4a6e..7f1e7cf93dc 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -27,14 +27,14 @@ Short functions can be passed to RDD methods using Python's [`lambda`](http://ww {% highlight python %} logData = sc.textFile(logFile).cache() -errors = logData.filter(lambda s: 'ERROR' in s.split()) +errors = logData.filter(lambda line: "ERROR" in line) {% endhighlight %} You can also pass functions that are defined using the `def` keyword; this is useful for more complicated functions that cannot be expressed using `lambda`: {% highlight python %} def is_error(line): - return 'ERROR' in line.split() + return "ERROR" in line errors = logData.filter(is_error) {% endhighlight %} @@ -43,8 +43,7 @@ Functions can access objects in enclosing scopes, although modifications to thos {% highlight python %} error_keywords = ["Exception", "Error"] def is_error(line): - words = line.split() - return any(keyword in words for keyword in error_keywords) + return any(keyword in line for keyword in error_keywords) errors = logData.filter(is_error) {% endhighlight %} From 92a4c2a5f6946bfae2136c52a22899db196f5799 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 27 Jun 2013 12:33:06 -0700 Subject: [PATCH 0194/2521] Fixing bug in local scheduler time recording --- .../main/scala/spark/scheduler/local/LocalScheduler.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 93d4318b299..9d375e1db84 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -153,10 +153,10 @@ 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 deserStart = System.currentTimeMillis() + val taskStart = System.currentTimeMillis() val deserializedTask = ser.deserialize[Task[_]]( taskBytes, Thread.currentThread.getContextClassLoader) - val deserTime = System.currentTimeMillis() - deserStart + val deserTime = System.currentTimeMillis() - taskStart // Run it val result: Any = deserializedTask.run(taskId) @@ -170,8 +170,9 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: val resultToReturn = ser.deserialize[Any](serResult) val accumUpdates = ser.deserialize[collection.mutable.Map[Long, Any]]( ser.serialize(Accumulators.values)) + val serviceTime = System.currentTimeMillis() - taskStart logInfo("Finished " + taskId) - deserializedTask.metrics.get.executorRunTime = deserTime.toInt//info.duration.toInt //close enough + deserializedTask.metrics.get.executorRunTime = serviceTime.toInt deserializedTask.metrics.get.executorDeserializeTime = deserTime.toInt val taskResult = new TaskResult(result, accumUpdates, deserializedTask.metrics.getOrElse(null)) From 362d996c81e2b3275a61df2266372ed470ea4d88 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 27 Jun 2013 13:02:47 -0700 Subject: [PATCH 0195/2521] Handful of changes based on matei's review - Avoid exception when no tasks have finished for a stage - Adding DOCTYPE so css renders properly - Adding progress slider --- core/src/main/scala/spark/Utils.scala | 20 ++++++++++++++++ core/src/main/scala/spark/ui/JettyUtils.scala | 2 +- .../main/scala/spark/ui/jobs/IndexPage.scala | 23 ++++++++++++++++--- .../scala/spark/ui/jobs/JobProgressUI.scala | 3 ++- .../main/scala/spark/ui/jobs/StagePage.scala | 14 +++++++++-- core/src/test/scala/spark/ui/UISuite.scala | 23 ++++++++++++++++--- .../main/scala/spark/streaming/Duration.scala | 6 ++++- 7 files changed, 80 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index f3621c6beee..6966ee9ee94 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -485,6 +485,26 @@ private object Utils extends Logging { "%.1f %s".formatLocal(Locale.US, value, unit) } + /** + * Returns a human-readable string representing a duration such as "35ms" + */ + def msDurationToString(ms: Long): String = { + val second = 1000 + val minute = 60 * second + val hour = 60 * minute + + ms match { + case t if t < second => + "%dms".format(t) + case t if t < minute => + "%d.%03ds".format(t / second, t % second) + case t if t < hour => + "%d:%02d".format(t / minute, (t % minute) / second) + case t => + "%d:%02d:%02d".format(t / hour, t % hour / minute, (t % hour) % minute / second) + } + } + /** * Convert a memory quantity in megabytes to a human-readable string such as "4.0 MB". */ diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 1f311bb75ff..fde36067407 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -26,7 +26,7 @@ private[spark] object JettyUtils extends Logging { createHandler(responder, "text/json", (in: JValue) => pretty(render(in))) implicit def htmlResponderToHandler(responder: Responder[Seq[Node]]): Handler = - createHandler(responder, "text/html") + createHandler(responder, "text/html", (in: Seq[Node]) => "" + in.toString) implicit def textResponderToHandler(responder: Responder[String]): Handler = createHandler(responder, "text/plain") diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 152e1a79b04..9b34cdd27fe 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -19,8 +19,8 @@ private[spark] 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", - "Shuffle Activity", "Stored RDD") + 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 @@ -43,6 +43,20 @@ private[spark] class IndexPage(parent: JobProgressUI) { } } + def makeSlider(completed: Int, total: Int): Seq[Node] = { + val width=130 + val height=15 + val completeWidth = (completed.toDouble / total) * width + + + + + + } + + def stageRow(showLink: Boolean = true)(s: Stage): Seq[Node] = { val submissionTime = s.submissionTime match { case Some(t) => dateFmt.format(new Date(t)) @@ -55,6 +69,8 @@ private[spark] class IndexPage(parent: JobProgressUI) { case (false, true) => "Write" case _ => "" } + val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) + val totalTasks = s.numPartitions {if (showLink) {{s.id}} @@ -63,7 +79,8 @@ private[spark] class IndexPage(parent: JobProgressUI) { {submissionTime} {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))} - {listener.stageToTasksComplete.getOrElse(s.id, 0)} / {s.numPartitions} + {makeSlider(completedTasks, totalTasks)} + {completedTasks} / {totalTasks} {listener.stageToTasksFailed.getOrElse(s.id, 0) match { case f if f > 0 => "(%s failed)".format(f) case _ => diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 46ba1d31af0..a18bf0f81e2 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -17,6 +17,7 @@ import spark.scheduler._ import spark.scheduler.cluster.TaskInfo import spark.executor.TaskMetrics import spark.Success +import spark.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ private[spark] class JobProgressUI(val sc: SparkContext) { @@ -32,7 +33,7 @@ private[spark] class JobProgressUI(val sc: SparkContext) { sc.addSparkListener(listener) } - def formatDuration(ms: Long) = Duration(ms, "milliseconds").printHMS + def formatDuration(ms: Long) = Utils.msDurationToString(ms) 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 03200861e0d..518f48cb818 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -19,20 +19,30 @@ private[spark] class StagePage(parent: JobProgressUI) { def render(request: HttpServletRequest): Seq[Node] = { val stageId = request.getParameter("id").toInt + + if (!listener.stageToTaskInfos.contains(stageId)) { + val content = +
    +

    Summary Metrics

    No tasks have finished yet +

    Tasks

    No tasks have finished yet +
    + return headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId)) + } + val tasks = listener.stageToTaskInfos(stageId) val shuffleRead = listener.hasShuffleRead(stageId) val shuffleWrite = listener.hasShuffleWrite(stageId) val taskHeaders: Seq[String] = - Seq("Task ID", "Service Time", "Locality Level", "Worker", "Launch Time") ++ + Seq("Task ID", "Duration", "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) val serviceTimes = tasks.map{case (info, metrics) => metrics.executorRunTime.toDouble} - val serviceQuantiles = "Service Time" +: Distribution(serviceTimes).get.getQuantiles().map( + val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles().map( ms => parent.formatDuration(ms.toLong)) def getQuantileCols(data: Seq[Double]) = diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index 6519bf77c6b..127ab5ebc21 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -2,10 +2,11 @@ package spark.ui import org.scalatest.FunSuite import org.eclipse.jetty.server.Server -import util.{Try, Success, Failure} import java.net.ServerSocket +import scala.util.{Failure, Success, Try} +import spark.Utils -private[spark] class UISuite extends FunSuite { +class UISuite extends FunSuite { test("jetty port increases under contention") { val startPort = 33333 val server = new Server(startPort) @@ -23,7 +24,23 @@ private[spark] class UISuite extends FunSuite { 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) => + 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) === "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") + } } diff --git a/streaming/src/main/scala/spark/streaming/Duration.scala b/streaming/src/main/scala/spark/streaming/Duration.scala index ee26206e249..c2135195d89 100644 --- a/streaming/src/main/scala/spark/streaming/Duration.scala +++ b/streaming/src/main/scala/spark/streaming/Duration.scala @@ -1,5 +1,7 @@ package spark.streaming +import spark.Utils + case class Duration (private val millis: Long) { def < (that: Duration): Boolean = (this.millis < that.millis) @@ -32,8 +34,10 @@ case class Duration (private val millis: Long) { def toFormattedString: String = millis.toString def milliseconds: Long = millis -} + def prettyPrint = Utils.msDurationToString(millis) + +} /** * Helper object that creates instance of [[spark.streaming.Duration]] representing From a55190d3143dae9a29650819621396175ef8a0dd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 27 Jun 2013 16:36:10 -0700 Subject: [PATCH 0196/2521] Adding better tabs for UI headers. --- .../ui/static/bootstrap-responsive.min.css | 4 +- .../spark/ui/static/bootstrap.min.css | 4 +- .../main/resources/spark/ui/static/webui.css | 41 +++++++++ .../deploy/master/ui/ApplicationPage.scala | 2 +- .../spark/deploy/master/ui/IndexPage.scala | 2 +- .../spark/deploy/worker/ui/IndexPage.scala | 2 +- core/src/main/scala/spark/ui/JettyUtils.scala | 85 ++++++++++++++----- .../main/scala/spark/ui/jobs/IndexPage.scala | 3 +- .../main/scala/spark/ui/jobs/StagePage.scala | 5 +- .../scala/spark/ui/storage/IndexPage.scala | 3 +- .../main/scala/spark/ui/storage/RDDPage.scala | 3 +- 11 files changed, 120 insertions(+), 34 deletions(-) create mode 100644 core/src/main/resources/spark/ui/static/webui.css 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 index 1f55036a074..f4ede63f32e 100644 --- a/core/src/main/resources/spark/ui/static/bootstrap-responsive.min.css +++ b/core/src/main/resources/spark/ui/static/bootstrap-responsive.min.css @@ -1,9 +1,9 @@ /*! - * Bootstrap Responsive v2.0.4 + * Bootstrap Responsive v2.3.2 * * 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}} + */.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;line-height:0;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:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}@-ms-viewport{width:device-width}.hidden{display:none;visibility:hidden}.visible-phone{display:none!important}.visible-tablet{display:none!important}.hidden-desktop{display:none!important}.visible-desktop{display:inherit!important}@media(min-width:768px) and (max-width:979px){.hidden-desktop{display:inherit!important}.visible-desktop{display:none!important}.visible-tablet{display:inherit!important}.hidden-tablet{display:none!important}}@media(max-width:767px){.hidden-desktop{display:inherit!important}.visible-desktop{display:none!important}.visible-phone{display:inherit!important}.hidden-phone{display:none!important}}.visible-print{display:none!important}@media print{.visible-print{display:inherit!important}.hidden-print{display:none!important}}@media(min-width:1200px){.row{margin-left:-30px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;min-height:1px;margin-left:30px}.container,.navbar-static-top .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;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.564102564102564%;*margin-left:2.5109110747408616%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .controls-row [class*="span"]+[class*="span"]{margin-left:2.564102564102564%}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.45299145299145%;*width:91.39979996362975%}.row-fluid .span10{width:82.90598290598291%;*width:82.8527914166212%}.row-fluid .span9{width:74.35897435897436%;*width:74.30578286961266%}.row-fluid .span8{width:65.81196581196582%;*width:65.75877432260411%}.row-fluid .span7{width:57.26495726495726%;*width:57.21176577559556%}.row-fluid .span6{width:48.717948717948715%;*width:48.664757228587014%}.row-fluid .span5{width:40.17094017094017%;*width:40.11774868157847%}.row-fluid .span4{width:31.623931623931625%;*width:31.570740134569924%}.row-fluid .span3{width:23.076923076923077%;*width:23.023731587561375%}.row-fluid .span2{width:14.52991452991453%;*width:14.476723040552828%}.row-fluid .span1{width:5.982905982905983%;*width:5.929714493544281%}.row-fluid .offset12{margin-left:105.12820512820512%;*margin-left:105.02182214948171%}.row-fluid .offset12:first-child{margin-left:102.56410256410257%;*margin-left:102.45771958537915%}.row-fluid .offset11{margin-left:96.58119658119658%;*margin-left:96.47481360247316%}.row-fluid .offset11:first-child{margin-left:94.01709401709402%;*margin-left:93.91071103837061%}.row-fluid .offset10{margin-left:88.03418803418803%;*margin-left:87.92780505546462%}.row-fluid .offset10:first-child{margin-left:85.47008547008548%;*margin-left:85.36370249136206%}.row-fluid .offset9{margin-left:79.48717948717949%;*margin-left:79.38079650845607%}.row-fluid .offset9:first-child{margin-left:76.92307692307693%;*margin-left:76.81669394435352%}.row-fluid .offset8{margin-left:70.94017094017094%;*margin-left:70.83378796144753%}.row-fluid .offset8:first-child{margin-left:68.37606837606839%;*margin-left:68.26968539734497%}.row-fluid .offset7{margin-left:62.393162393162385%;*margin-left:62.28677941443899%}.row-fluid .offset7:first-child{margin-left:59.82905982905982%;*margin-left:59.72267685033642%}.row-fluid .offset6{margin-left:53.84615384615384%;*margin-left:53.739770867430444%}.row-fluid .offset6:first-child{margin-left:51.28205128205128%;*margin-left:51.175668303327875%}.row-fluid .offset5{margin-left:45.299145299145295%;*margin-left:45.1927623204219%}.row-fluid .offset5:first-child{margin-left:42.73504273504273%;*margin-left:42.62865975631933%}.row-fluid .offset4{margin-left:36.75213675213675%;*margin-left:36.645753773413354%}.row-fluid .offset4:first-child{margin-left:34.18803418803419%;*margin-left:34.081651209310785%}.row-fluid .offset3{margin-left:28.205128205128204%;*margin-left:28.0987452264048%}.row-fluid .offset3:first-child{margin-left:25.641025641025642%;*margin-left:25.53464266230224%}.row-fluid .offset2{margin-left:19.65811965811966%;*margin-left:19.551736679396257%}.row-fluid .offset2:first-child{margin-left:17.094017094017094%;*margin-left:16.98763411529369%}.row-fluid .offset1{margin-left:11.11111111111111%;*margin-left:11.004728132387708%}.row-fluid .offset1:first-child{margin-left:8.547008547008547%;*margin-left:8.440625568285142%}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:30px}input.span12,textarea.span12,.uneditable-input.span12{width:1156px}input.span11,textarea.span11,.uneditable-input.span11{width:1056px}input.span10,textarea.span10,.uneditable-input.span10{width:956px}input.span9,textarea.span9,.uneditable-input.span9{width:856px}input.span8,textarea.span8,.uneditable-input.span8{width:756px}input.span7,textarea.span7,.uneditable-input.span7{width:656px}input.span6,textarea.span6,.uneditable-input.span6{width:556px}input.span5,textarea.span5,.uneditable-input.span5{width:456px}input.span4,textarea.span4,.uneditable-input.span4{width:356px}input.span3,textarea.span3,.uneditable-input.span3{width:256px}input.span2,textarea.span2,.uneditable-input.span2{width:156px}input.span1,textarea.span1,.uneditable-input.span1{width:56px}.thumbnails{margin-left:-30px}.thumbnails>li{margin-left:30px}.row-fluid .thumbnails{margin-left:0}}@media(min-width:768px) and (max-width:979px){.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;min-height:1px;margin-left:20px}.container,.navbar-static-top .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;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.7624309392265194%;*margin-left:2.709239449864817%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .controls-row [class*="span"]+[class*="span"]{margin-left:2.7624309392265194%}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.43646408839778%;*width:91.38327259903608%}.row-fluid .span10{width:82.87292817679558%;*width:82.81973668743387%}.row-fluid .span9{width:74.30939226519337%;*width:74.25620077583166%}.row-fluid .span8{width:65.74585635359117%;*width:65.69266486422946%}.row-fluid .span7{width:57.18232044198895%;*width:57.12912895262725%}.row-fluid .span6{width:48.61878453038674%;*width:48.56559304102504%}.row-fluid .span5{width:40.05524861878453%;*width:40.00205712942283%}.row-fluid .span4{width:31.491712707182323%;*width:31.43852121782062%}.row-fluid .span3{width:22.92817679558011%;*width:22.87498530621841%}.row-fluid .span2{width:14.3646408839779%;*width:14.311449394616199%}.row-fluid .span1{width:5.801104972375691%;*width:5.747913483013988%}.row-fluid .offset12{margin-left:105.52486187845304%;*margin-left:105.41847889972962%}.row-fluid .offset12:first-child{margin-left:102.76243093922652%;*margin-left:102.6560479605031%}.row-fluid .offset11{margin-left:96.96132596685082%;*margin-left:96.8549429881274%}.row-fluid .offset11:first-child{margin-left:94.1988950276243%;*margin-left:94.09251204890089%}.row-fluid .offset10{margin-left:88.39779005524862%;*margin-left:88.2914070765252%}.row-fluid .offset10:first-child{margin-left:85.6353591160221%;*margin-left:85.52897613729868%}.row-fluid .offset9{margin-left:79.8342541436464%;*margin-left:79.72787116492299%}.row-fluid .offset9:first-child{margin-left:77.07182320441989%;*margin-left:76.96544022569647%}.row-fluid .offset8{margin-left:71.2707182320442%;*margin-left:71.16433525332079%}.row-fluid .offset8:first-child{margin-left:68.50828729281768%;*margin-left:68.40190431409427%}.row-fluid .offset7{margin-left:62.70718232044199%;*margin-left:62.600799341718584%}.row-fluid .offset7:first-child{margin-left:59.94475138121547%;*margin-left:59.838368402492065%}.row-fluid .offset6{margin-left:54.14364640883978%;*margin-left:54.037263430116376%}.row-fluid .offset6:first-child{margin-left:51.38121546961326%;*margin-left:51.27483249088986%}.row-fluid .offset5{margin-left:45.58011049723757%;*margin-left:45.47372751851417%}.row-fluid .offset5:first-child{margin-left:42.81767955801105%;*margin-left:42.71129657928765%}.row-fluid .offset4{margin-left:37.01657458563536%;*margin-left:36.91019160691196%}.row-fluid .offset4:first-child{margin-left:34.25414364640884%;*margin-left:34.14776066768544%}.row-fluid .offset3{margin-left:28.45303867403315%;*margin-left:28.346655695309746%}.row-fluid .offset3:first-child{margin-left:25.69060773480663%;*margin-left:25.584224756083227%}.row-fluid .offset2{margin-left:19.88950276243094%;*margin-left:19.783119783707537%}.row-fluid .offset2:first-child{margin-left:17.12707182320442%;*margin-left:17.02068884448102%}.row-fluid .offset1{margin-left:11.32596685082873%;*margin-left:11.219583872105325%}.row-fluid .offset1:first-child{margin-left:8.56353591160221%;*margin-left:8.457152932878806%}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:710px}input.span11,textarea.span11,.uneditable-input.span11{width:648px}input.span10,textarea.span10,.uneditable-input.span10{width:586px}input.span9,textarea.span9,.uneditable-input.span9{width:524px}input.span8,textarea.span8,.uneditable-input.span8{width:462px}input.span7,textarea.span7,.uneditable-input.span7{width:400px}input.span6,textarea.span6,.uneditable-input.span6{width:338px}input.span5,textarea.span5,.uneditable-input.span5{width:276px}input.span4,textarea.span4,.uneditable-input.span4{width:214px}input.span3,textarea.span3,.uneditable-input.span3{width:152px}input.span2,textarea.span2,.uneditable-input.span2{width:90px}input.span1,textarea.span1,.uneditable-input.span1{width:28px}}@media(max-width:767px){body{padding-right:20px;padding-left:20px}.navbar-fixed-top,.navbar-fixed-bottom,.navbar-static-top{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}.thumbnails>li{float:none;margin-left:0}[class*="span"],.uneditable-input[class*="span"],.row-fluid [class*="span"]{display:block;float:none;width:100%;margin-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.span12,.row-fluid .span12{width:100%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="offset"]:first-child{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:30px;-webkit-box-sizing:border-box;-moz-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}.controls-row [class*="span"]+[class*="span"]{margin-left:0}.modal{position:fixed;top:20px;right:20px;left:20px;width:auto;margin:0}.modal.fade{top:-100px}.modal.fade.in{top:20px}}@media(max-width:480px){.nav-collapse{-webkit-transform:translate3d(0,0,0)}.page-header h1 small{display:block;line-height:20px}input[type="checkbox"],input[type="radio"]{border:1px solid #ccc}.form-horizontal .control-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}.media .pull-left,.media .pull-right{display:block;float:none;margin-bottom:10px}.media-object{margin-right:0;margin-left:0}.modal{top:10px;right:10px;left:10px}.modal-header .close{padding:10px;margin:-10px}.carousel-caption{position:static}}@media(max-width:979px){body{padding-top:0}.navbar-fixed-top,.navbar-fixed-bottom{position:static}.navbar-fixed-top{margin-bottom:20px}.navbar-fixed-bottom{margin-top:20px}.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 10px}.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:#777;text-shadow:none}.nav-collapse .nav>li>a,.nav-collapse .dropdown-menu a{padding:9px 15px;font-weight:bold;color:#777;-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 .nav>li>a:focus,.nav-collapse .dropdown-menu a:hover,.nav-collapse .dropdown-menu a:focus{background-color:#f2f2f2}.navbar-inverse .nav-collapse .nav>li>a,.navbar-inverse .nav-collapse .dropdown-menu a{color:#999}.navbar-inverse .nav-collapse .nav>li>a:hover,.navbar-inverse .nav-collapse .nav>li>a:focus,.navbar-inverse .nav-collapse .dropdown-menu a:hover,.navbar-inverse .nav-collapse .dropdown-menu a:focus{background-color:#111}.nav-collapse.in .btn-group{padding:0;margin-top:5px}.nav-collapse .dropdown-menu{position:static;top:auto;left:auto;display:none;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 .open>.dropdown-menu{display:block}.nav-collapse .dropdown-menu:before,.nav-collapse .dropdown-menu:after{display:none}.nav-collapse .dropdown-menu .divider{display:none}.nav-collapse .nav>li>.dropdown-menu:before,.nav-collapse .nav>li>.dropdown-menu:after{display:none}.nav-collapse .navbar-form,.nav-collapse .navbar-search{float:none;padding:10px 15px;margin:10px 0;border-top:1px solid #f2f2f2;border-bottom:1px solid #f2f2f2;-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-inverse .nav-collapse .navbar-form,.navbar-inverse .nav-collapse .navbar-search{border-top-color:#111;border-bottom-color:#111}.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 index b74b4546a62..b6428e69586 100644 --- a/core/src/main/resources/spark/ui/static/bootstrap.min.css +++ b/core/src/main/resources/spark/ui/static/bootstrap.min.css @@ -1,9 +1,9 @@ /*! - * Bootstrap v2.0.4 + * Bootstrap v2.3.2 * * 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} + */.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;line-height:0;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:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}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{width:auto\9;height:auto;max-width:100%;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic}#map_canvas img,.google-maps 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,html input[type="button"],input[type="reset"],input[type="submit"]{cursor:pointer;-webkit-appearance:button}label,select,button,input[type="button"],input[type="reset"],input[type="submit"],input[type="radio"],input[type="checkbox"]{cursor:pointer}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}@media print{*{color:#000!important;text-shadow:none!important;background:transparent!important;box-shadow:none!important}a,a:visited{text-decoration:underline}a[href]:after{content:" (" attr(href) ")"}abbr[title]:after{content:" (" attr(title) ")"}.ir a:after,a[href^="javascript:"]:after,a[href^="#"]:after{content:""}pre,blockquote{border:1px solid #999;page-break-inside:avoid}thead{display:table-header-group}tr,img{page-break-inside:avoid}img{max-width:100%!important}@page{margin:.5cm}p,h2,h3{orphans:3;widows:3}h2,h3{page-break-after:avoid}}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover,a:focus{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;min-height:1px;margin-left:20px}.container,.navbar-static-top .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;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .controls-row [class*="span"]+[class*="span"]{margin-left:2.127659574468085%}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:21px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}a.muted:hover,a.muted:focus{color:#808080}.text-warning{color:#c09853}a.text-warning:hover,a.text-warning:focus{color:#a47e3c}.text-error{color:#b94a48}a.text-error:hover,a.text-error:focus{color:#953b39}.text-info{color:#3a87ad}a.text-info:hover,a.text-info:focus{color:#2d6987}.text-success{color:#468847}a.text-success:hover,a.text-success:focus{color:#356635}.text-left{text-align:left}.text-right{text-align:right}.text-center{text-align:center}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:20px;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1,h2,h3{line-height:40px}h1{font-size:38.5px}h2{font-size:31.5px}h3{font-size:24.5px}h4{font-size:17.5px}h5{font-size:14px}h6{font-size:11.9px}h1 small{font-size:24.5px}h2 small{font-size:17.5px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}ul.inline,ol.inline{margin-left:0;list-style:none}ul.inline>li,ol.inline>li{display:inline-block;*display:inline;padding-right:5px;padding-left:5px;*zoom:1}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.dl-horizontal{*zoom:1}.dl-horizontal:before,.dl-horizontal:after{display:table;line-height:0;content:""}.dl-horizontal:after{clear:both}.dl-horizontal dt{float:left;width:160px;overflow:hidden;clear:left;text-align:right;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:180px}hr{margin:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}abbr[title],abbr[data-original-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 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:17.5px;font-weight:300;line-height:1.25}blockquote small{display:block;line-height:20px;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}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,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;white-space:nowrap;background-color:#f7f7f9;border:1px solid #e1e1e8}pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;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:20px}pre code{padding:0;color:inherit;white-space:pre;white-space:pre-wrap;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}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:20px;padding:4px 6px;margin-bottom:10px;font-size:14px;line-height:20px;color:#555;vertical-align:middle;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}input,textarea,.uneditable-input{width:206px}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-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;-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:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;border:1px solid #ccc}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}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-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)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.radio,.checkbox{min-height:20px;padding-left:20px}.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-20px}.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[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .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}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"],.row-fluid .controls-row [class*="span"]{float:left}.controls-row .checkbox[class*="span"],.controls-row .radio[class*="span"]{padding-top:5px}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent}.control-group.warning .control-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}.control-group.warning input,.control-group.warning select,.control-group.warning textarea{border-color:#c09853;-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)}.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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 .control-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}.control-group.error input,.control-group.error select,.control-group.error textarea{border-color:#b94a48;-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)}.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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 .control-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}.control-group.success input,.control-group.success select,.control-group.success textarea{border-color:#468847;-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)}.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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}.control-group.info .control-label,.control-group.info .help-block,.control-group.info .help-inline{color:#3a87ad}.control-group.info .checkbox,.control-group.info .radio,.control-group.info input,.control-group.info select,.control-group.info textarea{color:#3a87ad}.control-group.info input,.control-group.info select,.control-group.info textarea{border-color:#3a87ad;-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)}.control-group.info input:focus,.control-group.info select:focus,.control-group.info textarea:focus{border-color:#2d6987;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7ab5d3;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7ab5d3;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7ab5d3}.control-group.info .input-prepend .add-on,.control-group.info .input-append .add-on{color:#3a87ad;background-color:#d9edf7;border-color:#3a87ad}input:focus:invalid,textarea:focus:invalid,select:focus:invalid{color:#b94a48;border-color:#ee5f5b}input:focus:invalid:focus,textarea:focus:invalid:focus,select:focus: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:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{display:inline-block;margin-bottom:10px;font-size:0;white-space:nowrap;vertical-align:middle}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input,.input-append .dropdown-menu,.input-prepend .dropdown-menu,.input-append .popover,.input-prepend .popover{font-size:14px}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;vertical-align:top;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn,.input-append .btn-group>.dropdown-toggle,.input-prepend .btn-group>.dropdown-toggle{vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .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:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.input-append input+.btn-group .btn:last-child,.input-append select+.btn-group .btn:last-child,.input-append .uneditable-input+.btn-group .btn:last-child{-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.input-append .add-on,.input-append .btn,.input-append .btn-group{margin-left:-1px}.input-append .add-on:last-child,.input-append .btn:last-child,.input-append .btn-group:last-child>.dropdown-toggle{-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 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 input+.btn-group .btn,.input-prepend.input-append select+.btn-group .btn,.input-prepend.input-append .uneditable-input+.btn-group .btn{-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.input-prepend.input-append .btn-group:first-child{margin-left:0}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 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;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{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:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;content:""}.form-horizontal .control-group:after{clear:both}.form-horizontal .control-label{float:left;width:160px;padding-top:5px;text-align:right}.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:180px;*margin-left:0}.form-horizontal .controls:first-child{*padding-left:180px}.form-horizontal .help-block{margin-bottom:0}.form-horizontal input+.help-block,.form-horizontal select+.help-block,.form-horizontal textarea+.help-block,.form-horizontal .uneditable-input+.help-block,.form-horizontal .input-prepend+.help-block,.form-horizontal .input-append+.help-block{margin-top:10px}.form-horizontal .form-actions{padding-left:180px}table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0}.table{width:100%;margin-bottom:20px}.table th,.table td{padding:8px;line-height:20px;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 .table{background-color:#fff}.table-condensed th,.table-condensed td{padding:4px 5px}.table-bordered{border:1px solid #ddd;border-collapse:separate;*border-collapse:collapse;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,.table-bordered tbody:first-child tr:first-child>th: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,.table-bordered tbody:first-child tr:first-child>th: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,.table-bordered tbody:last-child tr:last-child>th:first-child,.table-bordered tfoot:last-child tr:last-child>td:first-child,.table-bordered tfoot:last-child tr:last-child>th:first-child{-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,.table-bordered tbody:last-child tr:last-child>th:last-child,.table-bordered tfoot:last-child tr:last-child>td:last-child,.table-bordered tfoot:last-child tr:last-child>th:last-child{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px}.table-bordered tfoot+tbody:last-child tr:last-child td:first-child{-webkit-border-bottom-left-radius:0;border-bottom-left-radius:0;-moz-border-radius-bottomleft:0}.table-bordered tfoot+tbody:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:0;border-bottom-right-radius:0;-moz-border-radius-bottomright:0}.table-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-striped tbody>tr:nth-child(odd)>td,.table-striped tbody>tr:nth-child(odd)>th{background-color:#f9f9f9}.table-hover tbody tr:hover>td,.table-hover tbody tr:hover>th{background-color:#f5f5f5}table td[class*="span"],table th[class*="span"],.row-fluid table td[class*="span"],.row-fluid table th[class*="span"]{display:table-cell;float:none;margin-left:0}.table td.span1,.table th.span1{float:none;width:44px;margin-left:0}.table td.span2,.table th.span2{float:none;width:124px;margin-left:0}.table td.span3,.table th.span3{float:none;width:204px;margin-left:0}.table td.span4,.table th.span4{float:none;width:284px;margin-left:0}.table td.span5,.table th.span5{float:none;width:364px;margin-left:0}.table td.span6,.table th.span6{float:none;width:444px;margin-left:0}.table td.span7,.table th.span7{float:none;width:524px;margin-left:0}.table td.span8,.table th.span8{float:none;width:604px;margin-left:0}.table td.span9,.table th.span9{float:none;width:684px;margin-left:0}.table td.span10,.table th.span10{float:none;width:764px;margin-left:0}.table td.span11,.table th.span11{float:none;width:844px;margin-left:0}.table td.span12,.table th.span12{float:none;width:924px;margin-left:0}.table tbody tr.success>td{background-color:#dff0d8}.table tbody tr.error>td{background-color:#f2dede}.table tbody tr.warning>td{background-color:#fcf8e3}.table tbody tr.info>td{background-color:#d9edf7}.table-hover tbody tr.success:hover>td{background-color:#d0e9c6}.table-hover tbody tr.error:hover>td{background-color:#ebcccc}.table-hover tbody tr.warning:hover>td{background-color:#faf2cc}.table-hover tbody tr.info:hover>td{background-color:#c4e3f3}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*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}.icon-white,.nav-pills>.active>a>[class^="icon-"],.nav-pills>.active>a>[class*=" icon-"],.nav-list>.active>a>[class^="icon-"],.nav-list>.active>a>[class*=" icon-"],.navbar-inverse .nav>.active>a>[class^="icon-"],.navbar-inverse .nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:focus>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>li>a:focus>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"],.dropdown-submenu:hover>a>[class^="icon-"],.dropdown-submenu:focus>a>[class^="icon-"],.dropdown-submenu:hover>a>[class*=" icon-"],.dropdown-submenu:focus>a>[class*=" icon-"]{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{width:16px;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{width:16px;background-position:-384px -120px}.icon-folder-open{width:16px;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:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 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:6px;-moz-border-radius:6px;border-radius:6px;-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:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu>li>a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu>li>a:hover,.dropdown-menu>li>a:focus,.dropdown-submenu:hover>a,.dropdown-submenu:focus>a{color:#fff;text-decoration:none;background-color:#0081c2;background-image:-moz-linear-gradient(top,#08c,#0077b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#0077b3));background-image:-webkit-linear-gradient(top,#08c,#0077b3);background-image:-o-linear-gradient(top,#08c,#0077b3);background-image:linear-gradient(to bottom,#08c,#0077b3);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0077b3',GradientType=0)}.dropdown-menu>.active>a,.dropdown-menu>.active>a:hover,.dropdown-menu>.active>a:focus{color:#fff;text-decoration:none;background-color:#0081c2;background-image:-moz-linear-gradient(top,#08c,#0077b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#0077b3));background-image:-webkit-linear-gradient(top,#08c,#0077b3);background-image:-o-linear-gradient(top,#08c,#0077b3);background-image:linear-gradient(to bottom,#08c,#0077b3);background-repeat:repeat-x;outline:0;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0077b3',GradientType=0)}.dropdown-menu>.disabled>a,.dropdown-menu>.disabled>a:hover,.dropdown-menu>.disabled>a:focus{color:#999}.dropdown-menu>.disabled>a:hover,.dropdown-menu>.disabled>a:focus{text-decoration:none;cursor:default;background-color:transparent;background-image:none;filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.open{*z-index:1000}.open>.dropdown-menu{display:block}.dropdown-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:990}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #000;content:""}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover>.dropdown-menu{display:block}.dropup .dropdown-submenu>.dropdown-menu{top:auto;bottom:0;margin-top:0;margin-bottom:-2px;-webkit-border-radius:5px 5px 5px 0;-moz-border-radius:5px 5px 5px 0;border-radius:5px 5px 5px 0}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown-submenu.pull-left{float:none}.dropdown-submenu.pull-left>.dropdown-menu{left:-100%;margin-left:10px;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.typeahead{z-index:1051;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 #e3e3e3;-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;-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;-o-transition:height .35s ease;transition:height .35s ease}.collapse.in{height:auto}.close{float:right;font-size:20px;font-weight:bold;line-height:20px;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover,.close:focus{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 12px;margin-bottom:0;*margin-left:.3em;font-size:14px;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:-moz-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(to bottom,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #ccc;*border:0;border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);border-bottom-color:#b3b3b3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',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:focus,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{color:#333;background-color:#e6e6e6;*background-color:#d9d9d9}.btn:active,.btn.active{background-color:#ccc \9}.btn:first-child{*margin-left:0}.btn:hover,.btn:focus{color:#333;text-decoration:none;background-position:0 -15px;-webkit-transition:background-position .1s linear;-moz-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-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-image:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-large{padding:11px 19px;font-size:17.5px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.btn-large [class^="icon-"],.btn-large [class*=" icon-"]{margin-top:4px}.btn-small{padding:2px 10px;font-size:11.9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.btn-small [class^="icon-"],.btn-small [class*=" icon-"]{margin-top:0}.btn-mini [class^="icon-"],.btn-mini [class*=" icon-"]{margin-top:-1px}.btn-mini{padding:0 6px;font-size:10.5px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}input[type="submit"].btn-block,input[type="reset"].btn-block,input[type="button"].btn-block{width:100%}.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-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-moz-linear-gradient(top,#08c,#04c);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;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='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:focus,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#f89406;background-image:-moz-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(to bottom,#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='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:focus,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#bd362f;background-image:-moz-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:linear-gradient(to bottom,#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='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:focus,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#51a351;background-image:-moz-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:linear-gradient(to bottom,#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='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:focus,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#2f96b4;background-image:-moz-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:linear-gradient(to bottom,#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='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:focus,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-moz-linear-gradient(top,#444,#222);background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#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='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:focus,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}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-link,.btn-link:active,.btn-link[disabled]{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover,.btn-link:focus{color:#005580;text-decoration:underline;background-color:transparent}.btn-link[disabled]:hover,.btn-link[disabled]:focus{color:#333;text-decoration:none}.btn-group{position:relative;display:inline-block;*display:inline;*margin-left:.3em;font-size:0;white-space:nowrap;vertical-align:middle;*zoom:1}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar>.btn+.btn,.btn-toolbar>.btn-group+.btn,.btn-toolbar>.btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu,.btn-group>.popover{font-size:14px}.btn-group>.btn-mini{font-size:10.5px}.btn-group>.btn-small{font-size:11.9px}.btn-group>.btn-large{font-size:17.5px}.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>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;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-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;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:#04c}.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:8px;margin-left:0}.btn-large .caret{margin-top:6px}.btn-large .caret{border-top-width:5px;border-right-width:5px;border-left-width:5px}.btn-mini .caret,.btn-small .caret{margin-top:8px}.dropup .btn-large .caret{border-bottom-width:5px}.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}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical>.btn{display:block;float:none;max-width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical>.btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical>.btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical>.btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical>.btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical>.btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;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,.alert h4{color:#c09853}.alert h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.alert-success{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.alert-success h4{color:#468847}.alert-danger,.alert-error{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.alert-danger h4,.alert-error h4{color:#b94a48}.alert-info{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.alert-info h4{color:#3a87ad}.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:20px;margin-left:0;list-style:none}.nav>li>a{display:block}.nav>li>a:hover,.nav>li>a:focus{text-decoration:none;background-color:#eee}.nav>li>a>img{max-width:none}.nav>.pull-right{float:right}.nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;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,.nav-list>.active>a:focus{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.2);background-color:#08c}.nav-list [class^="icon-"],.nav-list [class*=" icon-"]{margin-right:2px}.nav-list .divider{*width:100%;height:1px;margin:9px 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;line-height:0;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:20px;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,.nav-tabs>li>a:focus{border-color:#eee #eee #ddd}.nav-tabs>.active>a,.nav-tabs>.active>a:hover,.nav-tabs>.active>a:focus{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,.nav-pills>.active>a:focus{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-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft:4px}.nav-tabs.nav-stacked>li>a:hover,.nav-tabs.nav-stacked>li>a:focus{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 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret,.nav .dropdown-toggle:focus .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover,.nav>.dropdown.active>a:focus{cursor:pointer}.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover,.nav>li.dropdown.open.active>a:focus{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,.nav li.dropdown.open a:focus .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:1;filter:alpha(opacity=100)}.tabs-stacked .open>a:hover,.tabs-stacked .open>a:focus{border-color:#999}.tabbable{*zoom:1}.tabbable:before,.tabbable:after{display:table;line-height:0;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,.tabs-below>.nav-tabs>li>a:focus{border-top-color:#ddd;border-bottom-color:transparent}.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover,.tabs-below>.nav-tabs>.active>a:focus{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,.tabs-left>.nav-tabs>li>a:focus{border-color:#eee #ddd #eee #eee}.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover,.tabs-left>.nav-tabs .active>a:focus{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,.tabs-right>.nav-tabs>li>a:focus{border-color:#eee #eee #eee #ddd}.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover,.tabs-right>.nav-tabs .active>a:focus{border-color:#ddd #ddd #ddd transparent;*border-left-color:#fff}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover,.nav>.disabled>a:focus{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#f2f2f2);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#f2f2f2));background-image:-webkit-linear-gradient(top,#fff,#f2f2f2);background-image:-o-linear-gradient(top,#fff,#f2f2f2);background-image:linear-gradient(to bottom,#fff,#f2f2f2);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff',endColorstr='#fff2f2f2',GradientType=0);*zoom:1;-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar-inner:before,.navbar-inner:after{display:table;line-height:0;content:""}.navbar-inner:after{clear:both}.navbar .container{width:auto}.nav-collapse.collapse{height:auto;overflow:visible}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#777;text-shadow:0 1px 0 #fff}.navbar .brand:hover,.navbar .brand:focus{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px;color:#777}.navbar-link{color:#777}.navbar-link:hover,.navbar-link:focus{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:5px}.navbar .btn-group .btn,.navbar .input-prepend .btn,.navbar .input-append .btn,.navbar .input-prepend .btn-group,.navbar .input-append .btn-group{margin-top:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;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,.navbar-form .btn{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:5px;white-space:nowrap}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0}.navbar-search{position:relative;float:left;margin-top:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-bottom:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{border-width:0 0 1px}.navbar-fixed-bottom .navbar-inner{border-width:1px 0 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-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 10px rgba(0,0,0,0.1);box-shadow:0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:0 -1px 10px rgba(0,0,0,0.1);box-shadow:0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right;margin-right:0}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#777;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;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='#fff2f2f2',endColorstr='#ffe5e5e5',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:focus,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#ccc \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 .nav>li>.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 .nav>li>.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 .nav>li>.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 .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown>a:hover .caret,.navbar .nav li.dropdown>a:focus .caret{border-top-color:#333;border-bottom-color:#333}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#777;border-bottom-color:#777}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover,.navbar-inverse .brand:focus,.navbar-inverse .nav>li>a:focus{color:#fff}.navbar-inverse .brand{color:#999}.navbar-inverse .navbar-text{color:#999}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover,.navbar-inverse .navbar-link:focus{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>a:hover .caret,.navbar-inverse .nav li.dropdown>a:focus .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-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;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;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-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-moz-linear-gradient(top,#151515,#040404);background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #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='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:focus,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb>li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb>li>.divider{padding:0 5px;color:#ccc}.breadcrumb>.active{color:#999}.pagination{margin:20px 0}.pagination ul{display:inline-block;*display:inline;margin-bottom:0;margin-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;*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 ul>li{display:inline}.pagination ul>li>a,.pagination ul>li>span{float:left;padding:4px 12px;line-height:20px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination ul>li>a:hover,.pagination ul>li>a:focus,.pagination ul>.active>a,.pagination ul>.active>span{background-color:#f5f5f5}.pagination ul>.active>a,.pagination ul>.active>span{color:#999;cursor:default}.pagination ul>.disabled>span,.pagination ul>.disabled>a,.pagination ul>.disabled>a:hover,.pagination ul>.disabled>a:focus{color:#999;cursor:default;background-color:transparent}.pagination ul>li:first-child>a,.pagination ul>li:first-child>span{border-left-width:1px;-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}.pagination ul>li:last-child>a,.pagination ul>li:last-child>span{-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}.pagination-centered{text-align:center}.pagination-right{text-align:right}.pagination-large ul>li>a,.pagination-large ul>li>span{padding:11px 19px;font-size:17.5px}.pagination-large ul>li:first-child>a,.pagination-large ul>li:first-child>span{-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}.pagination-large ul>li:last-child>a,.pagination-large ul>li:last-child>span{-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}.pagination-mini ul>li:first-child>a,.pagination-small ul>li:first-child>a,.pagination-mini ul>li:first-child>span,.pagination-small ul>li:first-child>span{-webkit-border-bottom-left-radius:3px;border-bottom-left-radius:3px;-webkit-border-top-left-radius:3px;border-top-left-radius:3px;-moz-border-radius-bottomleft:3px;-moz-border-radius-topleft:3px}.pagination-mini ul>li:last-child>a,.pagination-small ul>li:last-child>a,.pagination-mini ul>li:last-child>span,.pagination-small ul>li:last-child>span{-webkit-border-top-right-radius:3px;border-top-right-radius:3px;-webkit-border-bottom-right-radius:3px;border-bottom-right-radius:3px;-moz-border-radius-topright:3px;-moz-border-radius-bottomright:3px}.pagination-small ul>li>a,.pagination-small ul>li>span{padding:2px 10px;font-size:11.9px}.pagination-mini ul>li>a,.pagination-mini ul>li>span{padding:0 6px;font-size:10.5px}.pager{margin:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;content:""}.pager:after{clear:both}.pager li{display:inline}.pager li>a,.pager li>span{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 li>a:hover,.pager li>a:focus{text-decoration:none;background-color:#f5f5f5}.pager .next>a,.pager .next>span{float:right}.pager .previous>a,.pager .previous>span{float:left}.pager .disabled>a,.pager .disabled>a:hover,.pager .disabled>a:focus,.pager .disabled>span{color:#999;cursor:default;background-color:#fff}.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:10%;left:50%;z-index:1050;width:560px;margin-left:-280px;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;outline:0;-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;-o-transition:opacity .3s linear,top .3s ease-out;transition:opacity .3s linear,top .3s ease-out}.modal.fade.in{top:10%}.modal-header{padding:9px 15px;border-bottom:1px solid #eee}.modal-header .close{margin-top:2px}.modal-header h3{margin:0;line-height:30px}.modal-body{position:relative;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;line-height:0;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}.modal-footer .btn-block+.btn-block{margin-left:0}.tooltip{position:absolute;z-index:1030;display:block;font-size:11px;line-height:1.4;opacity:0;filter:alpha(opacity=0);visibility:visible}.tooltip.in{opacity:.8;filter:alpha(opacity=80)}.tooltip.top{padding:5px 0;margin-top:-3px}.tooltip.right{padding:0 5px;margin-left:3px}.tooltip.bottom{padding:5px 0;margin-top:3px}.tooltip.left{padding:0 5px;margin-left:-3px}.tooltip-inner{max-width:200px;padding: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;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;max-width:276px;padding:1px;text-align:left;white-space:normal;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-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}.popover.top{margin-top:-10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-left:-10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-title:empty{display:none}.popover-content{padding:9px 14px}.popover .arrow,.popover .arrow:after{position:absolute;display:block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow{border-width:11px}.popover .arrow:after{border-width:10px;content:""}.popover.top .arrow{bottom:-11px;left:50%;margin-left:-11px;border-top-color:#999;border-top-color:rgba(0,0,0,0.25);border-bottom-width:0}.popover.top .arrow:after{bottom:1px;margin-left:-10px;border-top-color:#fff;border-bottom-width:0}.popover.right .arrow{top:50%;left:-11px;margin-top:-11px;border-right-color:#999;border-right-color:rgba(0,0,0,0.25);border-left-width:0}.popover.right .arrow:after{bottom:-10px;left:1px;border-right-color:#fff;border-left-width:0}.popover.bottom .arrow{top:-11px;left:50%;margin-left:-11px;border-bottom-color:#999;border-bottom-color:rgba(0,0,0,0.25);border-top-width:0}.popover.bottom .arrow:after{top:1px;margin-left:-10px;border-bottom-color:#fff;border-top-width:0}.popover.left .arrow{top:50%;right:-11px;margin-top:-11px;border-left-color:#999;border-left-color:rgba(0,0,0,0.25);border-right-width:0}.popover.left .arrow:after{right:1px;bottom:-10px;border-left-color:#fff;border-right-width:0}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}a.thumbnail:hover,a.thumbnail:focus{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;color:#555}.media,.media-body{overflow:hidden;*overflow:visible;zoom:1}.media,.media .media{margin-top:15px}.media:first-child{margin-top:0}.media-object{display:block}.media-heading{margin:0 0 5px}.media>.pull-left{margin-right:10px}.media>.pull-right{margin-left:10px}.media-list{margin-left:0;list-style:none}.label,.badge{display:inline-block;padding:2px 4px;font-size:11.844px;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{-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.badge{padding-right:9px;padding-left:9px;-webkit-border-radius:9px;-moz-border-radius:9px;border-radius:9px}.label:empty,.badge:empty{display:none}a.label:hover,a.label:focus,a.badge:hover,a.badge:focus{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}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-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:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-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(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#fff9f9f9',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{float:left;width:0;height:100%;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(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',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;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;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:-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);-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,.progress .bar-danger{background-color:#dd514c;background-image:-moz-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(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{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:-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,.progress .bar-success{background-color:#5eb95e;background-image:-moz-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(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{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:-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,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-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(to bottom,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff339bb9',GradientType=0)}.progress-info.progress-striped .bar,.progress-striped .bar-info{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:-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,.progress .bar-warning{background-color:#faa732;background-image:-moz-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(to bottom,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0)}.progress-warning.progress-striped .bar,.progress-striped .bar-warning{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:-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:20px}.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:20px;line-height:1}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel-inner>.item{position:relative;display:none;-webkit-transition:.6s ease-in-out left;-moz-transition:.6s ease-in-out left;-o-transition:.6s ease-in-out left;transition:.6s ease-in-out left}.carousel-inner>.item>img,.carousel-inner>.item>a>img{display:block;line-height:1}.carousel-inner>.active,.carousel-inner>.next,.carousel-inner>.prev{display:block}.carousel-inner>.active{left:0}.carousel-inner>.next,.carousel-inner>.prev{position:absolute;top:0;width:100%}.carousel-inner>.next{left:100%}.carousel-inner>.prev{left:-100%}.carousel-inner>.next.left,.carousel-inner>.prev.right{left:0}.carousel-inner>.active.left{left:-100%}.carousel-inner>.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,.carousel-control:focus{color:#fff;text-decoration:none;opacity:.9;filter:alpha(opacity=90)}.carousel-indicators{position:absolute;top:15px;right:15px;z-index:5;margin:0;list-style:none}.carousel-indicators li{display:block;float:left;width:10px;height:10px;margin-left:5px;text-indent:-999px;background-color:#ccc;background-color:rgba(255,255,255,0.25);border-radius:5px}.carousel-indicators .active{background-color:#fff}.carousel-caption{position:absolute;right:0;bottom:0;left:0;padding:15px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{line-height:20px;color:#fff}.carousel-caption h4{margin:0 0 5px}.carousel-caption p{margin-bottom:0}.hero-unit{padding:60px;margin-bottom:30px;font-size:18px;font-weight:200;line-height:30px;color:inherit;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 li{line-height:30px}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden}.affix{position:fixed} diff --git a/core/src/main/resources/spark/ui/static/webui.css b/core/src/main/resources/spark/ui/static/webui.css new file mode 100644 index 00000000000..c1ad4dbb17e --- /dev/null +++ b/core/src/main/resources/spark/ui/static/webui.css @@ -0,0 +1,41 @@ +.navbar .brand { + height: 50px; + width: 110px; + margin-left: 1px; + padding: 0; +} + +.version { + line-height: 30px; + vertical-align: bottom; + font-size: 12px; + padding: 0; + margin: 0; + font-weight: bold; + color: #777; +} + +.navbar-inner { + padding-top: 2px; + height: 50px; +} + +.navbar-inner .nav { + margin-top: 5px; + font-size: 15px; +} + + +#infolist { + margin-left: 400px; + margin-top: 14px; +} + +#infolist li { + display: inline; + list-style-type: none; + list-style-position: outside; + padding-right: 20px; + padding-top: 10px; + padding-bottom: 10px; +} 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 fb2c803a912..33a16b5d849 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -77,7 +77,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { {executorTable}
    ; - UIUtils.sparkPage(content, "Application Info: " + app.desc.name) + UIUtils.basicSparkPage(content, "Application Info: " + app.desc.name) } def executorRow(executor: ExecutorInfo): Seq[Node] = { 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 4892f258e1a..2ed566a5bc3 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -79,7 +79,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { {completedAppsTable}
    ; - UIUtils.sparkPage(content, "Spark Master: " + state.uri) + UIUtils.basicSparkPage(content, "Spark Master: " + state.uri) } def workerRow(worker: WorkerInfo): Seq[Node] = { 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 e95a74a37f8..c65d5b4fafe 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -70,7 +70,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { ; - UIUtils.sparkPage(content, "Spark Worker on %s:%s".format(workerState.host, workerState.port)) + UIUtils.basicSparkPage(content, "Spark Worker on %s:%s".format(workerState.host, workerState.port)) } def executorRow(executor: ExecutorRunner): Seq[Node] = { diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index fde36067407..5e20f41501e 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -107,35 +107,76 @@ private[spark] object JettyUtils extends Logging { } } +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
  • + } - /** 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 = -
    -
    -
    - + + + + + + + + {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 9b34cdd27fe..f5ed08744ca 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 518f48cb818..c9294a7261a 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 772c669a1ad..d284134391f 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 dd9fd2e4c54..65952f711a0 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] = { From 62c2c6b8561a00b2103fe5fb80b42b9975a5b8db Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 27 Jun 2013 21:47:22 -0700 Subject: [PATCH 0197/2521] 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 5e20f41501e..8cc50bc5b0b 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) } } From c767e7437059aa35bca3e8bb93264b35853c7a8f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 27 Jun 2013 21:48:58 -0700 Subject: [PATCH 0198/2521] 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 4000c4d5209..699901f1a13 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") From c537e869f3a12e75fac862cc6fdcf27a3e1e01b4 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 27 Jun 2013 22:02:03 -0700 Subject: [PATCH 0199/2521] Missing logo file --- .../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 0000000000000000000000000000000000000000..6c5f0993c43f0d9e1e1a04391d8a9157f0fdfba8 GIT binary patch literal 3536 zcmV;>4KMPEP)4Tx05}naRo`#hR1`jmZ&IWdKOk5~hl<6oRa0BJ8yc;~21%2p?MfD<>DVeH z9(p*dx19w`~g7O0}n_%Aq@s%d)fBDv`JHkDym6Hd+5XuAtvnwRpGmK zVkc9?T=n|PIo~X-eVh__(Z?q}P9Z-Dj?gOW6|D%o20XmjW-qs4UjrD(li^iv8@eK9k+ZFm zVRFymFOPAzG5-%Pn|1W;U4vNroTa&AxDScmEA~{ri9gr1^c?U@uwSpaNnw8l_>cP1 zd;)kMQS_;jeRSUEM_*s96y65j1$)tOrwdK{YIQMt92l|D^(E_=$Rjw{b!QT@q!)ni zR`|5oW9X5n$Wv+HVc@|^eX5yXnsHX8PF3UX~a6)MwxDE0HaPjyrlI!;jX{6Kvuh*8ej?;85ekN$?5uuCiS zBTvvVG+XTxAO{m@bvM#Jr)z6J><&E22D|vq?Y?Vkbo_DijopiF$2PET#mZ8eu=y$(ArYkv7@Ex`GL?QCc!_*KFrd&;n1r7 zqW-CFs9&fT)ZaU5gc&=gBz-DaCw(vdOp0__x+47~U6sC(E(JNe@4cTT*n6*E zVH4eoU1-&7pEV~_PRe`a7v+@vy!^5}8?Y3)UmlaER00009a7bBm000XU000XU0RWnu7ytkXQb|NXRA>e5 zSRhOo=*hmAER0R0YUQ5&?|AZ?L`ST`WM6a>^FLsRE~`yXWtl?w+2db2a{%eVhd4JE8MirO_aSq+gq*ou-ZU=tMS$2;L~N>BV;{wvcP<7Z_^#Xj=fhGn zif=t*TpK?89DUkm5wUAvW9vl!o*TLbli^DTS`@^q*4Y};iIMBs7&6=iN$VNr&8F$Q zeF7+21l#uUaYC;V zR6w;;vZ>ARVX2PzR~H40(fs;pcjB_RthZ<291>Y;0`Izl!WlITbxo_lI`?&3=ri-Q zdrM3BoZ|dR7T)zchbJeD0l%5zgZ99lU_ohXlQoXqb6|EQkc%K)^YM<$;~?fE;l1%) zcQtND#8gYWyL{{M-8)MsE@uvh&K~gh;o3s?-kqiGm76!GEigU)|GmvX&Llr~)cVWG@3)*vA*QX@6--0RtTDtPQ6}8;?wwlz#6MCK zis=Rh+j;Tocp^xq^?r@Eq#%F$_ZMdWU%#ti4P~wa0B*i`^=3!bw7k3=En)mAD{|HE2!z z;Wk>Q0D!d@-k;w5$I+X1te7IWpg%3&s-)QGYO?>Q#!i57Ur*XMg^+EIi}-v1l7N=2 zjhUD@m+ss+Rc4$d8v;1 zYBDA=_}HfZrO9=oT>a}qG(cDXi93OfW z`SI|h^~GyPJ0$P!?WH}glXO&?@V6!}475XGPSkEJOI!pzF)l!NVEjVXz53$#xTE=` zQ)ZqPPqgXUaP?I!@S6Zwnm2r8X;!|5hdBpHiygYE#>4dKB$n=fA< zH{v!3|7p8`FC#;{hC?hY!iS>sqsqjfNxZ z_Z+_mtOaX|N(NveI;)-ljf+7Zc9NI?qY$`(&96vY$gs-J(B{`gE)!2$gKII~_!vf) zj0(p`KweK@FzI=y2Wu^jV(2(~*N4F&nF(M#q)!>;69D6AiI_%)H+{Gx7ss{~g6Oa4 zhygc#Br@v4ZC2BOQ68CpD7t{(n>>U5mOv3>l?01l!uPo9e=jh~dJ#2H~EYB%0A_<@(%v0D?Aw1w&ZxILfi zcER+6bap=xbBFeZ?xSfO&?h|wGeP47Tm(;FPevNq@fsJc`Ouqva(49KkNSjB9Yi(4 zS`rlDPpdt^NJ>Qx(0f8l4R;NM9@Va;+Ch|dDAvc5Q$gbpesQC>Ts$JLb7ahwnDG$= zbGNbD_#6G0cYaEx73|zej{r9gG;i^JzmWvjR_Qfb=VNIPn$LHTzu-3PlGdG}(AMpA zE=L(&3u{_O)4U-WhlGbuFB4Xo&fG;eu9L1N0F_0awI;{{kprwY!v|kD!t=E=+fk@V zu+M7bxGCP2CQ2+nEG}$ICjtmSwxrFPn@(>MploEFT1H9{O1goeLXO9R{uQq~YQ}## zCi&Xc#9H)0VC{d8JAs%6#3(r*yD7)emzy4A7F{3!sRh)LtCe{=cn($3IZy+H3-3-_ z0J)}kp2TfNORX_JVaz{ZO!T#Z7e;?_*2O1{JZcxWSbRXt*D!AldDy;JA6ai1fIPui zdYsPaU2Ef^;i_TW0Ku3$L`Nw)X3|Zu{)*ajnFqt%PxQ~Kb1>Q645cy#-2gw(GH#k? z4p~L>h4y@GKC3URrJ--g=o3vx`bKr&liiVZm&uF5LlDV%Y5Y%UshP+M&3YmoQNnCD zwOgZuP>M6ioHtn{doy%+W=K3on@-0(%4YS2ECkrA$}MAc=5}bLfs5rNV2uH)xDdqj zMI>J@L@Aw)q~mhhP1z5|vzOkRjF+zcR}LniFwwU-#u^j?r4#~`7G46olDPJ-7s?Dm zQW#u)A}Q_^3DB<%wD+^XlD<)Hx8mcWGc|9iy1zjxGq4>z%i7Y%4Q$WmeLReLvNq9S z?dOje?4xC*hqkbpbA9O9qR39K`*75h_OyE98AU~!s&*yaIsH6=wUnaGYqd*F!*VCf zG{C5texA%5V8kF3Gzz8_-i%5%Gaf0uOGTa=H0-`C2q11u#0rsJ<~lQ`UFHq{5_`Ld zrTr4~e%51Lfp&r8y8`VBvD@(foXg>qSyzO9lVzjpa0000< KMNUMnLSTaP)3PW4 literal 0 HcmV?d00001 From 4974b658edf2716ff3c6f2e6863cddb2a4ddf891 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 27 Jun 2013 22:16:40 -0700 Subject: [PATCH 0200/2521] 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 7c06a550622..805466ea2c0 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 From 249f0e54ba6c358b3d786b1ba42e5d2ea22ed9db Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 28 Jun 2013 13:25:26 -0700 Subject: [PATCH 0201/2521] Minor changes from Matei's review --- core/src/main/scala/spark/ui/JettyUtils.scala | 2 +- .../main/scala/spark/ui/jobs/IndexPage.scala | 21 ++++++++----------- .../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 8cc50bc5b0b..9e4b8843790 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 f5ed08744ca..27c017d7eec 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 a18bf0f81e2..f584d1e187e 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) From 8113c55df8f4b5f34140ddba5e58e132e3dc2d23 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Fri, 28 Jun 2013 13:46:21 -0700 Subject: [PATCH 0202/2521] [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 ea67a3e6b8c..a8e72143fbc 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 From a80b28a579936251874b74e0d73c72425f8c273e Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Fri, 28 Jun 2013 18:18:51 -0700 Subject: [PATCH 0203/2521] Renamed several functions and classes and improved documentation --- graph/src/main/scala/spark/graph/Edge.scala | 6 +++++ .../scala/spark/graph/EdgeDirection.scala | 15 ++++++++++++ ...geWithVertices.scala => EdgeTriplet.scala} | 4 ++-- graph/src/main/scala/spark/graph/Graph.scala | 17 ++++++++----- graph/src/main/scala/spark/graph/Pregel.scala | 4 ++-- graph/src/main/scala/spark/graph/Vertex.scala | 6 ++++- ...VerticesRDD.scala => EdgeTripletRDD.scala} | 18 +++++++------- .../scala/spark/graph/impl/GraphImpl.scala | 24 +++++++++---------- 8 files changed, 62 insertions(+), 32 deletions(-) rename graph/src/main/scala/spark/graph/{EdgeWithVertices.scala => EdgeTriplet.scala} (67%) rename graph/src/main/scala/spark/graph/impl/{EdgeWithVerticesRDD.scala => EdgeTripletRDD.scala} (76%) diff --git a/graph/src/main/scala/spark/graph/Edge.scala b/graph/src/main/scala/spark/graph/Edge.scala index 8f022e812a5..cb057a467a4 100644 --- a/graph/src/main/scala/spark/graph/Edge.scala +++ b/graph/src/main/scala/spark/graph/Edge.scala @@ -1,6 +1,12 @@ package spark.graph +/** + * A single directed edge consisting of a source id, target id, + * and the data associated with the Edgee. + * + * @tparam ED type of the edge attribute + */ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] ( var src: Vid = 0, var dst: Vid = 0, diff --git a/graph/src/main/scala/spark/graph/EdgeDirection.scala b/graph/src/main/scala/spark/graph/EdgeDirection.scala index a0c52c7038b..38caac44d6f 100644 --- a/graph/src/main/scala/spark/graph/EdgeDirection.scala +++ b/graph/src/main/scala/spark/graph/EdgeDirection.scala @@ -1,6 +1,10 @@ package spark.graph +/** + * The direction of directed edge relative to a vertex used to select + * the set of adjacent neighbors when running a neighborhood query. + */ sealed abstract class EdgeDirection { def reverse: EdgeDirection = this match { case EdgeDirection.In => EdgeDirection.In @@ -11,7 +15,18 @@ sealed abstract class EdgeDirection { object EdgeDirection { + /** + * Edges arriving at a vertex. + */ case object In extends EdgeDirection + + /** + * Edges originating from a vertex + */ case object Out extends EdgeDirection + + /** + * All edges adjacent to a vertex + */ case object Both extends EdgeDirection } diff --git a/graph/src/main/scala/spark/graph/EdgeWithVertices.scala b/graph/src/main/scala/spark/graph/EdgeTriplet.scala similarity index 67% rename from graph/src/main/scala/spark/graph/EdgeWithVertices.scala rename to graph/src/main/scala/spark/graph/EdgeTriplet.scala index a731f73709c..9a819481ba8 100644 --- a/graph/src/main/scala/spark/graph/EdgeWithVertices.scala +++ b/graph/src/main/scala/spark/graph/EdgeTriplet.scala @@ -1,8 +1,8 @@ package spark.graph -class EdgeWithVertices[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD, - @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] { +class EdgeTriplet[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD, + @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] { var src: Vertex[VD] = _ var dst: Vertex[VD] = _ var data: ED = _ diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 421055d319e..d0742a1c8fd 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -20,6 +20,8 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { /** * Get the vertices and their data. * + * @return An RDD containing the vertices in this graph + * * @see Vertex for the vertex type. * * @todo should vertices return tuples instead of vertex objects? @@ -30,13 +32,14 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * Get the Edges and their data as an RDD. The entries in the RDD contain * just the source id and target id along with the edge data. * + * @return An RDD containing the edges in this graph * * @see Edge for the edge type. * @see edgesWithVertices to get an RDD which contains all the edges along * with their vertex data. * * @todo Should edges return 3 tuples instead of Edge objects? In this case - * we could rename EdgeWithVertices to Edge? + * we could rename EdgeTriplet to Edge? */ def edges(): RDD[Edge[ED]] @@ -44,6 +47,8 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * Get the edges with the vertex data associated with the adjacent pair of * vertices. * + * @return An RDD containing edge triplets. + * * @example This operation might be used to evaluate a graph coloring where * we would like to check that both vertices are a different color. * {{{ @@ -56,7 +61,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @see edges() If only the edge data and adjacent vertex ids are required. * */ - def edgesWithVertices(): RDD[EdgeWithVertices[VD, ED]] + def triplets(): RDD[EdgeTriplet[VD, ED]] /** * Return a graph that is cached when first created. This is used to pin a @@ -133,8 +138,8 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * }}} * */ - def mapEdgesWithVertices[ED2: ClassManifest]( - map: EdgeWithVertices[VD, ED] => ED2): Graph[VD, ED2] + def mapTriplets[ED2: ClassManifest]( + map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] /** @@ -183,7 +188,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * */ def aggregateNeighbors[VD2: ClassManifest]( - mapFunc: (Vid, EdgeWithVertices[VD, ED]) => Option[VD2], + mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[VD2], mergeFunc: (VD2, VD2) => VD2, direction: EdgeDirection) : RDD[(Vid, VD2)] @@ -232,7 +237,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * */ def aggregateNeighbors[VD2: ClassManifest]( - mapFunc: (Vid, EdgeWithVertices[VD, ED]) => Option[VD2], + mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[VD2], reduceFunc: (VD2, VD2) => VD2, default: VD2, // Should this be a function or a value? direction: EdgeDirection) diff --git a/graph/src/main/scala/spark/graph/Pregel.scala b/graph/src/main/scala/spark/graph/Pregel.scala index cd2400e5faf..699affba816 100644 --- a/graph/src/main/scala/spark/graph/Pregel.scala +++ b/graph/src/main/scala/spark/graph/Pregel.scala @@ -8,7 +8,7 @@ object Pregel { def iterate[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( vprog: ( Vertex[VD], A) => VD, - sendMsg: (Vid, EdgeWithVertices[VD, ED]) => Option[A], + sendMsg: (Vid, EdgeTriplet[VD, ED]) => Option[A], mergeMsg: (A, A) => A, initialMsg: A, numIter: Int) : Graph[VD, ED] = { @@ -16,7 +16,7 @@ object Pregel { var g = graph.cache var i = 0 - def mapF(vid: Vid, edge: EdgeWithVertices[VD,ED]) = sendMsg(edge.otherVertex(vid).id, edge) + def mapF(vid: Vid, edge: EdgeTriplet[VD,ED]) = sendMsg(edge.otherVertex(vid).id, edge) def runProg(v: Vertex[VD], msg: Option[A]): VD = { if (msg.isEmpty) v.data else vprog(v, msg.get) diff --git a/graph/src/main/scala/spark/graph/Vertex.scala b/graph/src/main/scala/spark/graph/Vertex.scala index 543cc8e9427..5323643bece 100644 --- a/graph/src/main/scala/spark/graph/Vertex.scala +++ b/graph/src/main/scala/spark/graph/Vertex.scala @@ -1,6 +1,10 @@ package spark.graph - +/** + * A graph vertex consists of a vertex id and attribute. + * + * @tparam VD the type of the vertex attribute. + */ case class Vertex[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD] ( var id: Vid = 0, var data: VD = nullValue[VD]) { diff --git a/graph/src/main/scala/spark/graph/impl/EdgeWithVerticesRDD.scala b/graph/src/main/scala/spark/graph/impl/EdgeTripletRDD.scala similarity index 76% rename from graph/src/main/scala/spark/graph/impl/EdgeWithVerticesRDD.scala rename to graph/src/main/scala/spark/graph/impl/EdgeTripletRDD.scala index 5370b4e1604..f157dd40560 100644 --- a/graph/src/main/scala/spark/graph/impl/EdgeWithVerticesRDD.scala +++ b/graph/src/main/scala/spark/graph/impl/EdgeTripletRDD.scala @@ -7,7 +7,7 @@ import spark.graph._ private[graph] -class EdgeWithVerticesPartition(idx: Int, val vPart: Partition, val ePart: Partition) +class EdgeTripletPartition(idx: Int, val vPart: Partition, val ePart: Partition) extends Partition { override val index: Int = idx override def hashCode(): Int = idx @@ -18,10 +18,10 @@ class EdgeWithVerticesPartition(idx: Int, val vPart: Partition, val ePart: Parti * A RDD that brings together edge data with its associated vertex data. */ private[graph] -class EdgeWithVerticesRDD[VD: ClassManifest, ED: ClassManifest]( +class EdgeTripletRDD[VD: ClassManifest, ED: ClassManifest]( vTableReplicated: RDD[(Vid, VD)], eTable: RDD[(Pid, EdgePartition[ED])]) - extends RDD[(VertexHashMap[VD], Iterator[EdgeWithVertices[VD, ED]])](eTable.context, Nil) { + extends RDD[(VertexHashMap[VD], Iterator[EdgeTriplet[VD, ED]])](eTable.context, Nil) { println(vTableReplicated.partitioner.get.numPartitions) println(eTable.partitioner.get.numPartitions) @@ -33,18 +33,18 @@ class EdgeWithVerticesRDD[VD: ClassManifest, ED: ClassManifest]( } override def getPartitions = Array.tabulate[Partition](eTable.partitions.size) { - i => new EdgeWithVerticesPartition(i, eTable.partitions(i), vTableReplicated.partitions(i)) + i => new EdgeTripletPartition(i, eTable.partitions(i), vTableReplicated.partitions(i)) } override val partitioner = eTable.partitioner override def getPreferredLocations(s: Partition) = - eTable.preferredLocations(s.asInstanceOf[EdgeWithVerticesPartition].ePart) + eTable.preferredLocations(s.asInstanceOf[EdgeTripletPartition].ePart) override def compute(s: Partition, context: TaskContext) - : Iterator[(VertexHashMap[VD], Iterator[EdgeWithVertices[VD, ED]])] = { + : Iterator[(VertexHashMap[VD], Iterator[EdgeTriplet[VD, ED]])] = { - val split = s.asInstanceOf[EdgeWithVerticesPartition] + val split = s.asInstanceOf[EdgeTripletPartition] // Fetch the vertices and put them in a hashmap. // TODO: use primitive hashmaps for primitive VD types. @@ -55,9 +55,9 @@ class EdgeWithVerticesRDD[VD: ClassManifest, ED: ClassManifest]( .asInstanceOf[(Pid, EdgePartition[ED])] // Return an iterator that looks up the hash map to find matching vertices for each edge. - val iter = new Iterator[EdgeWithVertices[VD, ED]] { + val iter = new Iterator[EdgeTriplet[VD, ED]] { private var pos = 0 - private val e = new EdgeWithVertices[VD, ED] + private val e = new EdgeTriplet[VD, ED] e.src = new Vertex[VD] e.dst = new Vertex[VD] diff --git a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala index 4565f94e833..6ee7b8a0620 100644 --- a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala @@ -78,8 +78,8 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } /** Return a RDD that brings edges with its source and destination vertices together. */ - override def edgesWithVertices: RDD[EdgeWithVertices[VD, ED]] = { - (new EdgeWithVerticesRDD(vTableReplicated, eTable)).mapPartitions { part => part.next._2 } + override def triplets: RDD[EdgeTriplet[VD, ED]] = { + (new EdgeTripletRDD(vTableReplicated, eTable)).mapPartitions { part => part.next._2 } } override def mapVertices[VD2: ClassManifest](f: Vertex[VD] => VD2): Graph[VD2, ED] = { @@ -90,9 +90,9 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( newGraph(vertices, edges.map(e => Edge(e.src, e.dst, f(e)))) } - override def mapEdgesWithVertices[ED2: ClassManifest](f: EdgeWithVertices[VD, ED] => ED2): + override def mapTriplets[ED2: ClassManifest](f: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] = { - newGraph(vertices, edgesWithVertices.map(e => Edge(e.src.id, e.dst.id, f(e)))) + newGraph(vertices, triplets.map(e => Edge(e.src.id, e.dst.id, f(e)))) } @@ -101,7 +101,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( ////////////////////////////////////////////////////////////////////////////////////////////////// override def aggregateNeighbors[VD2: ClassManifest]( - mapFunc: (Vid, EdgeWithVertices[VD, ED]) => Option[VD2], + mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[VD2], reduceFunc: (VD2, VD2) => VD2, default: VD2, gatherDirection: EdgeDirection) @@ -114,13 +114,13 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( part.map { v => (v._1, MutableTuple2(v._2, Option.empty[VD2])) } }, preservesPartitioning = true) - (new EdgeWithVerticesRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable)) + (new EdgeTripletRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable)) .mapPartitions { part => val (vmap, edges) = part.next() - val edgeSansAcc = new EdgeWithVertices[VD, ED]() + val edgeSansAcc = new EdgeTriplet[VD, ED]() edgeSansAcc.src = new Vertex[VD] edgeSansAcc.dst = new Vertex[VD] - edges.foreach { e: EdgeWithVertices[MutableTuple2[VD, Option[VD2]], ED] => + edges.foreach { e: EdgeTriplet[MutableTuple2[VD, Option[VD2]], ED] => edgeSansAcc.data = e.data edgeSansAcc.src.data = e.src.data._1 edgeSansAcc.dst.data = e.dst.data._1 @@ -158,7 +158,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( * As a consequence, the resulting table may be much smaller than the set of vertices. */ override def aggregateNeighbors[VD2: ClassManifest]( - mapFunc: (Vid, EdgeWithVertices[VD, ED]) => Option[VD2], + mapFunc: (Vid, EdgeTriplet[VD, ED]) => Option[VD2], reduceFunc: (VD2, VD2) => VD2, gatherDirection: EdgeDirection): RDD[(Vid, VD2)] = { @@ -169,13 +169,13 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( part.map { v => (v._1, MutableTuple2(v._2, Option.empty[VD2])) } }, preservesPartitioning = true) - (new EdgeWithVerticesRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable)) + (new EdgeTripletRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable)) .mapPartitions { part => val (vmap, edges) = part.next() - val edgeSansAcc = new EdgeWithVertices[VD, ED]() + val edgeSansAcc = new EdgeTriplet[VD, ED]() edgeSansAcc.src = new Vertex[VD] edgeSansAcc.dst = new Vertex[VD] - edges.foreach { e: EdgeWithVertices[MutableTuple2[VD, Option[VD2]], ED] => + edges.foreach { e: EdgeTriplet[MutableTuple2[VD, Option[VD2]], ED] => edgeSansAcc.data = e.data edgeSansAcc.src.data = e.src.data._1 edgeSansAcc.dst.data = e.dst.data._1 From 473961d82e70f4cdb9e3d12b0dfcb4b70e6121ab Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 29 Jun 2013 08:38:04 -0700 Subject: [PATCH 0204/2521] Styling for progress bar --- .../main/resources/spark/ui/static/webui.css | 8 ++++++ .../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 c1ad4dbb17e..f7537bb7661 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 27c017d7eec..cd04542faab 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 _ => From e721ff7e5a2d738750c40e95d6ba53898eaa7051 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 29 Jun 2013 11:26:30 -0700 Subject: [PATCH 0205/2521] Allowing details for failed stages --- .../scala/spark/scheduler/DAGScheduler.scala | 6 +- .../cluster/ClusterTaskSetManager.scala | 1 + .../scheduler/local/LocalTaskSetManager.scala | 1 + .../scala/spark/ui/jobs/JobProgressUI.scala | 30 ++++--- .../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 82d419453b5..d9ddc41aa2f 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 d72b0bfc9f5..6965cde5da3 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 70b69bb26fe..499116f6534 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 f584d1e187e..aafa4140559 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 c9294a7261a..ed96fc2994e 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("")} } } From ae12d163dc2462ededefc8d31900803cf9a782a5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 29 Jun 2013 15:22:15 -0700 Subject: [PATCH 0206/2521] Added the BytecodeUtils class for analyzing bytecode. --- graph/src/main/scala/spark/graph/Graph.scala | 6 +- .../spark/graph/util/BytecodeUtils.scala | 113 ++++++++++++++++++ .../test/scala/spark/graph/GraphSuite.scala | 74 ++++++------ .../spark/graph/util/BytecodeUtilsSuite.scala | 93 ++++++++++++++ 4 files changed, 246 insertions(+), 40 deletions(-) create mode 100644 graph/src/main/scala/spark/graph/util/BytecodeUtils.scala create mode 100644 graph/src/test/scala/spark/graph/util/BytecodeUtilsSuite.scala diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 421055d319e..7d296bc9dc8 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -204,7 +204,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @param mapFunc the function applied to each edge adjacent to each vertex. * The mapFunc can optionally return None in which case it does not * contribute to the final sum. - * @param mergeFunc the function used to merge the results of each map + * @param reduceFunc the function used to merge the results of each map * operation. * @param default the default value to use for each vertex if it has no * neighbors or the map function repeatedly evaluates to none @@ -247,7 +247,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * * @tparam U the type of entry in the table of updates * @tparam VD2 the new vertex value type - * @param tlb the table to join with the vertices in the graph. The table + * @param table the table to join with the vertices in the graph. The table * should contain at most one entry for each vertex. * @param mapFunc the function used to compute the new vertex values. The * map function is invoked for all vertices, even those that do not have a @@ -282,7 +282,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * map function is skipped and the old value is used. * * @tparam U the type of entry in the table of updates - * @param tlb the table to join with the vertices in the graph. The table + * @param table the table to join with the vertices in the graph. The table * should contain at most one entry for each vertex. * @param mapFunc the function used to compute the new vertex values. The * map function is invoked only for vertices with a corresponding entry in diff --git a/graph/src/main/scala/spark/graph/util/BytecodeUtils.scala b/graph/src/main/scala/spark/graph/util/BytecodeUtils.scala new file mode 100644 index 00000000000..268a3c2bcf5 --- /dev/null +++ b/graph/src/main/scala/spark/graph/util/BytecodeUtils.scala @@ -0,0 +1,113 @@ +package spark.graph.util + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} + +import scala.collection.mutable.HashSet + +import org.objectweb.asm.{ClassReader, MethodVisitor} +import org.objectweb.asm.commons.EmptyVisitor +import org.objectweb.asm.Opcodes._ + +import spark.Utils + + +private[graph] object BytecodeUtils { + + /** + * Test whether the given closure invokes the specified method in the specified class. + */ + def invokedMethod(closure: AnyRef, targetClass: Class[_], targetMethod: String): Boolean = { + if (_invokedMethod(closure.getClass, "apply", targetClass, targetMethod)) { + true + } else { + // look at closures enclosed in this closure + for (f <- closure.getClass.getDeclaredFields + if f.getType.getName.startsWith("scala.Function")) { + f.setAccessible(true) + if (invokedMethod(f.get(closure), targetClass, targetMethod)) { + return true + } + } + return false + } + } + + private def _invokedMethod(cls: Class[_], method: String, + targetClass: Class[_], targetMethod: String): Boolean = { + + val seen = new HashSet[(Class[_], String)] + var stack = List[(Class[_], String)]((cls, method)) + + while (stack.nonEmpty) { + val (c, m) = stack.head + stack = stack.tail + seen.add((c, m)) + val finder = new MethodInvocationFinder(c.getName, m) + getClassReader(c).accept(finder, 0) + for (classMethod <- finder.methodsInvoked) { + println(classMethod) + if (classMethod._1 == targetClass && classMethod._2 == targetMethod) { + return true + } else if (!seen.contains(classMethod)) { + stack = classMethod :: stack + } + } + } + return false + } + + /** + * Get an ASM class reader for a given class from the JAR that loaded it. + */ + private def getClassReader(cls: Class[_]): ClassReader = { + // Copy data over, before delegating to ClassReader - else we can run out of open file handles. + val className = cls.getName.replaceFirst("^.*\\.", "") + ".class" + val resourceStream = cls.getResourceAsStream(className) + // todo: Fixme - continuing with earlier behavior ... + if (resourceStream == null) return new ClassReader(resourceStream) + + val baos = new ByteArrayOutputStream(128) + Utils.copyStream(resourceStream, baos, true) + new ClassReader(new ByteArrayInputStream(baos.toByteArray)) + } + + /** + * Given the class name, return whether we should look into the class or not. This is used to + * skip examing a large quantity of Java or Scala classes that we know for sure wouldn't access + * the closures. Note that the class name is expected in ASM style (i.e. use "/" instead of "."). + */ + private def skipClass(className: String): Boolean = { + val c = className + c.startsWith("java/") || c.startsWith("scala/") || c.startsWith("javax/") + } + + /** + * Find the set of methods invoked by the specified method in the specified class. + * For example, after running the visitor, + * MethodInvocationFinder("spark/graph/Foo", "test") + * its methodsInvoked variable will contain the set of methods invoked directly by + * Foo.test(). Interface invocations are not returned as part of the result set because we cannot + * determine the actual metod invoked by inspecting the bytecode. + */ + private class MethodInvocationFinder(className: String, methodName: String) extends EmptyVisitor { + + val methodsInvoked = new HashSet[(Class[_], String)] + + override def visitMethod(access: Int, name: String, desc: String, + sig: String, exceptions: Array[String]): MethodVisitor = { + if (name == methodName) { + new EmptyVisitor { + override def visitMethodInsn(op: Int, owner: String, name: String, desc: String) { + if (op == INVOKEVIRTUAL || op == INVOKESPECIAL || op == INVOKESTATIC) { + if (!skipClass(owner)) { + methodsInvoked.add((Class.forName(owner.replace("/", ".")), name)) + } + } + } + } + } else { + null + } + } + } +} diff --git a/graph/src/test/scala/spark/graph/GraphSuite.scala b/graph/src/test/scala/spark/graph/GraphSuite.scala index 64a7aa063b0..4eb469a71f4 100644 --- a/graph/src/test/scala/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/spark/graph/GraphSuite.scala @@ -6,41 +6,41 @@ import spark.SparkContext class GraphSuite extends FunSuite with LocalSparkContext { - - test("graph partitioner") { - sc = new SparkContext("local", "test") - val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two"))) - val edges = sc.parallelize(Seq(Edge(1, 2, "onlyedge"))) - var g = new Graph(vertices, edges) - - g = g.withPartitioner(4, 7) - assert(g.numVertexPartitions === 4) - assert(g.numEdgePartitions === 7) - - g = g.withVertexPartitioner(5) - assert(g.numVertexPartitions === 5) - - g = g.withEdgePartitioner(8) - assert(g.numEdgePartitions === 8) - - g = g.mapVertices(x => x) - assert(g.numVertexPartitions === 5) - assert(g.numEdgePartitions === 8) - - g = g.mapEdges(x => x) - assert(g.numVertexPartitions === 5) - assert(g.numEdgePartitions === 8) - - val updates = sc.parallelize(Seq((1, " more"))) - g = g.updateVertices( - updates, - (v, u: Option[String]) => if (u.isDefined) v.data + u.get else v.data) - assert(g.numVertexPartitions === 5) - assert(g.numEdgePartitions === 8) - - g = g.reverse - assert(g.numVertexPartitions === 5) - assert(g.numEdgePartitions === 8) - - } +// +// test("graph partitioner") { +// sc = new SparkContext("local", "test") +// val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two"))) +// val edges = sc.parallelize(Seq(Edge(1, 2, "onlyedge"))) +// var g = new Graph(vertices, edges) +// +// g = g.withPartitioner(4, 7) +// assert(g.numVertexPartitions === 4) +// assert(g.numEdgePartitions === 7) +// +// g = g.withVertexPartitioner(5) +// assert(g.numVertexPartitions === 5) +// +// g = g.withEdgePartitioner(8) +// assert(g.numEdgePartitions === 8) +// +// g = g.mapVertices(x => x) +// assert(g.numVertexPartitions === 5) +// assert(g.numEdgePartitions === 8) +// +// g = g.mapEdges(x => x) +// assert(g.numVertexPartitions === 5) +// assert(g.numEdgePartitions === 8) +// +// val updates = sc.parallelize(Seq((1, " more"))) +// g = g.updateVertices( +// updates, +// (v, u: Option[String]) => if (u.isDefined) v.data + u.get else v.data) +// assert(g.numVertexPartitions === 5) +// assert(g.numEdgePartitions === 8) +// +// g = g.reverse +// assert(g.numVertexPartitions === 5) +// assert(g.numEdgePartitions === 8) +// +// } } diff --git a/graph/src/test/scala/spark/graph/util/BytecodeUtilsSuite.scala b/graph/src/test/scala/spark/graph/util/BytecodeUtilsSuite.scala new file mode 100644 index 00000000000..8d18cf39e89 --- /dev/null +++ b/graph/src/test/scala/spark/graph/util/BytecodeUtilsSuite.scala @@ -0,0 +1,93 @@ +package spark.graph.util + +import org.scalatest.FunSuite + + +class BytecodeUtilsSuite extends FunSuite { + + import BytecodeUtilsSuite.TestClass + + test("closure invokes a method") { + val c1 = {e: TestClass => println(e.foo); println(e.bar); println(e.baz); } + assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) + assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) + assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) + + val c2 = {e: TestClass => println(e.foo); println(e.bar); } + assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "foo")) + assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "bar")) + assert(!BytecodeUtils.invokedMethod(c2, classOf[TestClass], "baz")) + + val c3 = {e: TestClass => println(e.foo); } + assert(BytecodeUtils.invokedMethod(c3, classOf[TestClass], "foo")) + assert(!BytecodeUtils.invokedMethod(c3, classOf[TestClass], "bar")) + assert(!BytecodeUtils.invokedMethod(c3, classOf[TestClass], "baz")) + } + + test("closure inside a closure invokes a method") { + val c1 = {e: TestClass => println(e.foo); println(e.bar); println(e.baz); } + val c2 = {e: TestClass => c1(e); println(e.foo); } + assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "foo")) + assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "bar")) + assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "baz")) + } + + test("closure inside a closure inside a closure invokes a method") { + val c1 = {e: TestClass => println(e.baz); } + val c2 = {e: TestClass => c1(e); println(e.foo); } + val c3 = {e: TestClass => c2(e) } + assert(BytecodeUtils.invokedMethod(c3, classOf[TestClass], "foo")) + assert(!BytecodeUtils.invokedMethod(c3, classOf[TestClass], "bar")) + assert(BytecodeUtils.invokedMethod(c3, classOf[TestClass], "baz")) + } + + test("closure calling a function that invokes a method") { + def zoo(e: TestClass) { + println(e.baz) + } + val c1 = {e: TestClass => zoo(e)} + assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) + assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) + assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) + } + + test("closure calling a function that invokes a method which uses another closure") { + val c2 = {e: TestClass => println(e.baz)} + def zoo(e: TestClass) { + c2(e) + } + val c1 = {e: TestClass => zoo(e)} + assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) + assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) + assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) + } + + test("nested closure") { + val c2 = {e: TestClass => println(e.baz)} + def zoo(e: TestClass, c: TestClass => Unit) { + c(e) + } + val c1 = {e: TestClass => zoo(e, c2)} + assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) + assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) + assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) + } + + // The following doesn't work yet, because the byte code doesn't contain any information + // about what exactly "c" is. +// test("invoke interface") { +// val c1 = {e: TestClass => c(e)} +// assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) +// assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) +// assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) +// } + + private val c = {e: TestClass => println(e.baz)} +} + + +object BytecodeUtilsSuite { + class TestClass(val foo: Int, val bar: Long) { + def baz: Boolean = false + } +} From 4358acfe07e991090fbe009aafe3f5110fbf0c40 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 29 Jun 2013 15:25:06 -0700 Subject: [PATCH 0207/2521] Initialize Twitter4J OAuth from system properties instead of prompting --- .../spark/streaming/StreamingContext.scala | 4 ++- .../api/java/JavaStreamingContext.scala | 23 ++++++------- .../dstream/TwitterInputDStream.scala | 32 ++++++------------- .../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 e61438fe3ac..36b841af8fa 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 c4a223b4195..ed7b789d981 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 e0c654d385f..ff7a58be452 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 e5fdbe1b7af..4cf10582a95 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java @@ -1267,7 +1267,7 @@ public void testFileStream() { @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 From 5cfcd3c336cc13e9fd448ae122216e4b583b77b4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 29 Jun 2013 15:37:27 -0700 Subject: [PATCH 0208/2521] 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 3bcb2a3f345..7a31be98b2b 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 07572201ded..5e4692162e9 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( From 758ceff778a5590297fe9b712d9642f009f9b628 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 29 Jun 2013 15:39:48 -0700 Subject: [PATCH 0209/2521] Updated BytecodeUtils to ASM4. --- .../main/scala/spark/graph/util/BytecodeUtils.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/graph/src/main/scala/spark/graph/util/BytecodeUtils.scala b/graph/src/main/scala/spark/graph/util/BytecodeUtils.scala index 268a3c2bcf5..ac3a1fb9573 100644 --- a/graph/src/main/scala/spark/graph/util/BytecodeUtils.scala +++ b/graph/src/main/scala/spark/graph/util/BytecodeUtils.scala @@ -4,8 +4,7 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.collection.mutable.HashSet -import org.objectweb.asm.{ClassReader, MethodVisitor} -import org.objectweb.asm.commons.EmptyVisitor +import org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor} import org.objectweb.asm.Opcodes._ import spark.Utils @@ -45,7 +44,7 @@ private[graph] object BytecodeUtils { val finder = new MethodInvocationFinder(c.getName, m) getClassReader(c).accept(finder, 0) for (classMethod <- finder.methodsInvoked) { - println(classMethod) + //println(classMethod) if (classMethod._1 == targetClass && classMethod._2 == targetMethod) { return true } else if (!seen.contains(classMethod)) { @@ -89,14 +88,15 @@ private[graph] object BytecodeUtils { * Foo.test(). Interface invocations are not returned as part of the result set because we cannot * determine the actual metod invoked by inspecting the bytecode. */ - private class MethodInvocationFinder(className: String, methodName: String) extends EmptyVisitor { + private class MethodInvocationFinder(className: String, methodName: String) + extends ClassVisitor(ASM4) { val methodsInvoked = new HashSet[(Class[_], String)] override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { if (name == methodName) { - new EmptyVisitor { + new MethodVisitor(ASM4) { override def visitMethodInsn(op: Int, owner: String, name: String, desc: String) { if (op == INVOKEVIRTUAL || op == INVOKESPECIAL || op == INVOKESTATIC) { if (!skipClass(owner)) { From 79b5eaa4e2a32817a50e583554a53ed7ab72e0b2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 29 Jun 2013 15:58:59 -0700 Subject: [PATCH 0210/2521] Added a 64bit string hash function. --- .../scala/spark/graph/util/HashUtils.scala | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) create mode 100644 graph/src/main/scala/spark/graph/util/HashUtils.scala diff --git a/graph/src/main/scala/spark/graph/util/HashUtils.scala b/graph/src/main/scala/spark/graph/util/HashUtils.scala new file mode 100644 index 00000000000..0dfaef4c48d --- /dev/null +++ b/graph/src/main/scala/spark/graph/util/HashUtils.scala @@ -0,0 +1,21 @@ +package spark.graph.util + + +object HashUtils { + + /** + * Compute a 64-bit hash value for the given string. + * See http://stackoverflow.com/questions/1660501/what-is-a-good-64bit-hash-function-in-java-for-textual-strings + */ + def hash(str: String): Long = { + var h = 1125899906842597L + val len = str.length + var i = 0 + + while (i < len) { + h = 31 * h + str(i) + i += 1 + } + h + } +} From 2964df7a4e9d39c6206709680b9828e0cb7b1430 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sat, 29 Jun 2013 21:04:35 -0700 Subject: [PATCH 0211/2521] Commenting out unused test code. --- .../test/scala/spark/graph/GraphSuite.scala | 72 +++++++++---------- 1 file changed, 36 insertions(+), 36 deletions(-) diff --git a/graph/src/test/scala/spark/graph/GraphSuite.scala b/graph/src/test/scala/spark/graph/GraphSuite.scala index 64a7aa063b0..3d250aa18ce 100644 --- a/graph/src/test/scala/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/spark/graph/GraphSuite.scala @@ -7,40 +7,40 @@ import spark.SparkContext class GraphSuite extends FunSuite with LocalSparkContext { - test("graph partitioner") { - sc = new SparkContext("local", "test") - val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two"))) - val edges = sc.parallelize(Seq(Edge(1, 2, "onlyedge"))) - var g = new Graph(vertices, edges) - - g = g.withPartitioner(4, 7) - assert(g.numVertexPartitions === 4) - assert(g.numEdgePartitions === 7) - - g = g.withVertexPartitioner(5) - assert(g.numVertexPartitions === 5) - - g = g.withEdgePartitioner(8) - assert(g.numEdgePartitions === 8) - - g = g.mapVertices(x => x) - assert(g.numVertexPartitions === 5) - assert(g.numEdgePartitions === 8) - - g = g.mapEdges(x => x) - assert(g.numVertexPartitions === 5) - assert(g.numEdgePartitions === 8) - - val updates = sc.parallelize(Seq((1, " more"))) - g = g.updateVertices( - updates, - (v, u: Option[String]) => if (u.isDefined) v.data + u.get else v.data) - assert(g.numVertexPartitions === 5) - assert(g.numEdgePartitions === 8) - - g = g.reverse - assert(g.numVertexPartitions === 5) - assert(g.numEdgePartitions === 8) - - } +// test("graph partitioner") { +// sc = new SparkContext("local", "test") +// val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two"))) +// val edges = sc.parallelize(Seq(Edge(1, 2, "onlyedge"))) +// var g = Graph(vertices, edges) +// +// g = g.withPartitioner(4, 7) +// assert(g.numVertexPartitions === 4) +// assert(g.numEdgePartitions === 7) +// +// g = g.withVertexPartitioner(5) +// assert(g.numVertexPartitions === 5) +// +// g = g.withEdgePartitioner(8) +// assert(g.numEdgePartitions === 8) +// +// g = g.mapVertices(x => x) +// assert(g.numVertexPartitions === 5) +// assert(g.numEdgePartitions === 8) +// +// g = g.mapEdges(x => x) +// assert(g.numVertexPartitions === 5) +// assert(g.numEdgePartitions === 8) +// +// val updates = sc.parallelize(Seq((1, " more"))) +// g = g.updateVertices( +// updates, +// (v, u: Option[String]) => if (u.isDefined) v.data + u.get else v.data) +// assert(g.numVertexPartitions === 5) +// assert(g.numEdgePartitions === 8) +// +// g = g.reverse +// assert(g.numVertexPartitions === 5) +// assert(g.numEdgePartitions === 8) +// +// } } From f24548da88cb4283f5214643de4c9a0320db4143 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sat, 29 Jun 2013 21:04:53 -0700 Subject: [PATCH 0212/2521] Adding graph cosntruction code to graph singleton object. --- graph/src/main/scala/spark/graph/Graph.scala | 22 ++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index d0742a1c8fd..359f295f568 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -326,5 +326,27 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { object Graph { + import spark.graph.impl._ + import spark.SparkContext._ + + def apply(rawEdges: RDD[(Int,Int)], uniqueEdges: Boolean = true): Graph[Int, Int] = { + // Reduce to unique edges + val edges = + if(uniqueEdges) rawEdges.map{ case (s,t) => ((s,t),1) }.reduceByKey( _ + _ ) + .map{ case ((s,t), cnt) => Edge(s,t,cnt) } + else rawEdges.map{ case (s,t) => Edge(s,t,1) } + // Determine unique vertices + val vertices = edges.flatMap{ case Edge(s, t, cnt) => Array((s,1), (t,1)) }.reduceByKey( _ + _ ) + .map{ case (id, deg) => Vertex(id, deg) } + // Return graph + new GraphImpl(vertices, edges) + } + + def apply[VD: ClassManifest, ED: ClassManifest](vertices: RDD[Vertex[VD]], edges: RDD[Edge[ED]]): Graph[VD, ED] = { + new GraphImpl(vertices, edges) + + } + + implicit def graphToGraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) = g.ops } From ce7e270bb4c6374b0ad0cce0eabd4ebf1566718a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 29 Jun 2013 21:28:22 -0700 Subject: [PATCH 0213/2521] Added graph package to the classpath. --- bin/compute-classpath.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 3a788802904..4b772e6f2bc 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" +GRAPH_DIR="$FWDIR/graph" 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:$GRAPH_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do CLASSPATH="$CLASSPATH:$jar" done From 6acc2a7b3d3c2a1b449c2c50ece7ad34b0863ed1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 29 Jun 2013 21:28:31 -0700 Subject: [PATCH 0214/2521] Various minor changes. --- .../src/main/scala/spark/graph/Analytics.scala | 4 ++++ graph/src/main/scala/spark/graph/Graph.scala | 6 +++--- graph/src/main/scala/spark/graph/GraphOps.scala | 4 ++-- graph/src/main/scala/spark/graph/Pregel.scala | 17 +++++++++-------- graph/src/main/scala/spark/graph/Vertex.scala | 2 +- .../scala/spark/graph/impl/EdgePartition.scala | 8 ++++---- .../main/scala/spark/graph/impl/GraphImpl.scala | 12 ++++-------- 7 files changed, 27 insertions(+), 26 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index 05275bec683..8f76622db0e 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -7,6 +7,10 @@ import spark.SparkContext._ object Analytics extends Logging { + def main(args: Array[String]) { + //pregelPagerank() + } + // /** // * Compute the PageRank of a graph returning the pagerank of each vertex as an RDD // */ diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index fb5a86c41fc..506e53df082 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -26,7 +26,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * * @todo should vertices return tuples instead of vertex objects? */ - def vertices(): RDD[Vertex[VD]] + def vertices: RDD[Vertex[VD]] /** * Get the Edges and their data as an RDD. The entries in the RDD contain @@ -41,7 +41,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @todo Should edges return 3 tuples instead of Edge objects? In this case * we could rename EdgeTriplet to Edge? */ - def edges(): RDD[Edge[ED]] + def edges: RDD[Edge[ED]] /** * Get the edges with the vertex data associated with the adjacent pair of @@ -61,7 +61,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * @see edges() If only the edge data and adjacent vertex ids are required. * */ - def triplets(): RDD[EdgeTriplet[VD, ED]] + def triplets: RDD[EdgeTriplet[VD, ED]] /** * Return a graph that is cached when first created. This is used to pin a diff --git a/graph/src/main/scala/spark/graph/GraphOps.scala b/graph/src/main/scala/spark/graph/GraphOps.scala index 4fba8d19768..d98cd8d44cb 100644 --- a/graph/src/main/scala/spark/graph/GraphOps.scala +++ b/graph/src/main/scala/spark/graph/GraphOps.scala @@ -14,11 +14,11 @@ class GraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) { } lazy val outDegrees: RDD[(Vid, Int)] = { - g.aggregateNeighbors((vid,edge) => Some(1), _+_, EdgeDirection.Out) + g.aggregateNeighbors((vid, edge) => Some(1), _+_, EdgeDirection.Out) } lazy val degrees: RDD[(Vid, Int)] = { - g.aggregateNeighbors((vid,edge) => Some(1), _+_, EdgeDirection.Both) + g.aggregateNeighbors((vid, edge) => Some(1), _+_, EdgeDirection.Both) } def collectNeighborIds(edgeDirection: EdgeDirection) : RDD[(Vid, Array[Vid])] = { diff --git a/graph/src/main/scala/spark/graph/Pregel.scala b/graph/src/main/scala/spark/graph/Pregel.scala index 699affba816..0a564b8041f 100644 --- a/graph/src/main/scala/spark/graph/Pregel.scala +++ b/graph/src/main/scala/spark/graph/Pregel.scala @@ -1,19 +1,20 @@ package spark.graph -import scala.collection.JavaConversions._ import spark.RDD object Pregel { def iterate[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( - vprog: ( Vertex[VD], A) => VD, - sendMsg: (Vid, EdgeTriplet[VD, ED]) => Option[A], - mergeMsg: (A, A) => A, - initialMsg: A, - numIter: Int) : Graph[VD, ED] = { - - var g = graph.cache + vprog: (Vertex[VD], A) => VD, + sendMsg: (Vid, EdgeTriplet[VD, ED]) => Option[A], + mergeMsg: (A, A) => A, + initialMsg: A, + numIter: Int) + : Graph[VD, ED] = { + + var g = graph + //var g = graph.cache() var i = 0 def mapF(vid: Vid, edge: EdgeTriplet[VD,ED]) = sendMsg(edge.otherVertex(vid).id, edge) diff --git a/graph/src/main/scala/spark/graph/Vertex.scala b/graph/src/main/scala/spark/graph/Vertex.scala index 5323643bece..32653571f79 100644 --- a/graph/src/main/scala/spark/graph/Vertex.scala +++ b/graph/src/main/scala/spark/graph/Vertex.scala @@ -9,7 +9,7 @@ case class Vertex[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD var id: Vid = 0, var data: VD = nullValue[VD]) { - def this(tuple: Tuple2[Vid, VD]) = this(tuple._1, tuple._2) + def this(tuple: (Vid, VD)) = this(tuple._1, tuple._2) def tuple = (id, data) } diff --git a/graph/src/main/scala/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/spark/graph/impl/EdgePartition.scala index e5ed2db0f29..0008534c0f8 100644 --- a/graph/src/main/scala/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/spark/graph/impl/EdgePartition.scala @@ -11,8 +11,8 @@ import spark.graph._ * A partition of edges in 3 large columnar arrays. */ private[graph] -class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest] -{ +class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest] { + val srcIds: IntArrayList = new IntArrayList val dstIds: IntArrayList = new IntArrayList // TODO: Specialize data. @@ -33,7 +33,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) def size: Int = srcIds.size def iterator = new Iterator[Edge[ED]] { - private var edge = new Edge[ED] + private val edge = new Edge[ED] private var pos = 0 override def hasNext: Boolean = pos < EdgePartition.this.size @@ -46,4 +46,4 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) edge } } -} \ No newline at end of file +} diff --git a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala index 6ee7b8a0620..e18a2f6b097 100644 --- a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala @@ -2,11 +2,7 @@ package spark.graph.impl import scala.collection.JavaConversions._ -import spark.ClosureCleaner -import spark.HashPartitioner -import spark.Partitioner -import spark.RDD -import spark.SparkContext +import spark.{ClosureCleaner, HashPartitioner, RDD} import spark.SparkContext._ import spark.graph._ @@ -31,7 +27,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( def withPartitioner(numVertexPartitions: Int, numEdgePartitions: Int): Graph[VD, ED] = { if (_cached) { - (new GraphImpl(numVertexPartitions, numEdgePartitions, null, null, _rawVTable, _rawETable)) + new GraphImpl(numVertexPartitions, numEdgePartitions, null, null, _rawVTable, _rawETable) .cache() } else { new GraphImpl(numVertexPartitions, numEdgePartitions, _rawVertices, _rawEdges, null, null) @@ -73,13 +69,13 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( if (!_cached && _rawEdges != null) { _rawEdges } else { - eTable.mapPartitions { iter => iter.next._2.iterator } + eTable.mapPartitions { iter => iter.next()._2.iterator } } } /** Return a RDD that brings edges with its source and destination vertices together. */ override def triplets: RDD[EdgeTriplet[VD, ED]] = { - (new EdgeTripletRDD(vTableReplicated, eTable)).mapPartitions { part => part.next._2 } + new EdgeTripletRDD(vTableReplicated, eTable).mapPartitions { part => part.next()._2 } } override def mapVertices[VD2: ClassManifest](f: Vertex[VD] => VD2): Graph[VD2, ED] = { From f269e5975bee4581297e017fc0617df00e726c62 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sat, 29 Jun 2013 21:53:13 -0700 Subject: [PATCH 0215/2521] Adding additional assertions and documenting the edge triplet class --- .../main/scala/spark/graph/EdgeTriplet.scala | 48 ++++++++++++++++--- 1 file changed, 42 insertions(+), 6 deletions(-) diff --git a/graph/src/main/scala/spark/graph/EdgeTriplet.scala b/graph/src/main/scala/spark/graph/EdgeTriplet.scala index 9a819481ba8..3ed8052794a 100644 --- a/graph/src/main/scala/spark/graph/EdgeTriplet.scala +++ b/graph/src/main/scala/spark/graph/EdgeTriplet.scala @@ -1,17 +1,53 @@ package spark.graph - +/** + * An edge triplet represents two vertices and edge along with their attributes. + * + * @tparam VD the type of the vertex attribute. + * @tparam ED the type of the edge attribute + */ class EdgeTriplet[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) VD, @specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] { + /** + * The vertex (id and attribute) corresponding to the source vertex. + */ var src: Vertex[VD] = _ + + /** + * The vertex (id and attribute) corresponding to the target vertex. + */ var dst: Vertex[VD] = _ + + /** + * The attribute associated with the edge. + */ var data: ED = _ - def otherVertex(vid: Vid): Vertex[VD] = if (src.id == vid) dst else src + /** + * Given one vertex in the edge return the other vertex. + * + * @param vid the id one of the two vertices on the edge. + * @return the other vertex on the edge. + */ + def otherVertex(vid: Vid): Vertex[VD] = + if (src.id == vid) dst else { assert(dst.id == vid); src } + + /** + * Get the vertex object for the given vertex in the edge. + * + * @param vid the id of one of the two vertices on the edge + * @return the vertex object with that id. + */ + def vertex(vid: Vid): Vertex[VD] = + if (src.id == vid) src else { assert(dst.id == vid); dst } - def vertex(vid: Vid): Vertex[VD] = if (src.id == vid) src else dst + /** + * Return the relative direction of the edge to the corresponding vertex. + * + * @param vid the id of one of the two vertices in the edge. + * @return the relative direction of the edge to the corresponding vertex. + */ + def relativeDirection(vid: Vid): EdgeDirection = + if (vid == src.id) EdgeDirection.Out else { assert(vid == dst.id); EdgeDirection.In } - def relativeDirection(vid: Vid): EdgeDirection = { - if (vid == src.id) EdgeDirection.Out else EdgeDirection.In - } } From f776301241ee92eccd4f1c90d511b3ae568242eb Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sat, 29 Jun 2013 21:53:38 -0700 Subject: [PATCH 0216/2521] Resurrecting the GraphLab gather-apply-scatter api --- .../src/main/scala/spark/graph/GraphLab.scala | 232 +++++++++--------- 1 file changed, 119 insertions(+), 113 deletions(-) diff --git a/graph/src/main/scala/spark/graph/GraphLab.scala b/graph/src/main/scala/spark/graph/GraphLab.scala index 9c157b9361d..504cd162aba 100644 --- a/graph/src/main/scala/spark/graph/GraphLab.scala +++ b/graph/src/main/scala/spark/graph/GraphLab.scala @@ -3,119 +3,125 @@ package spark.graph import scala.collection.JavaConversions._ import spark.RDD - +/** + * This object implement the graphlab gather-apply-scatter api. + */ object GraphLab { - // def iterateGA2[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( - // gather: (Vid, EdgeWithVertices[VD, ED]) => A, - // merge: (A, A) => A, - // default: A, - // apply: (Vertex[VD], A) => VD, - // numIter: Int, - // gatherDirection: EdgeDirection = EdgeDirection.In) : Graph[VD, ED] = { - - // var g = graph.cache() - - // var i = 0 - // while (i < numIter) { - - // val accUpdates: RDD[(Vid, A)] = - // g.aggregateNeighbors(gather, merge, default, gatherDirection) - - // def applyFunc(v: Vertex[VD], update: Option[A]): VD = { apply(v, update.get) } - // g = g.updateVertices(accUpdates, applyFunc).cache() - - // i += 1 - // } - // g - // } - - // def iterateGA[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( - // gatherFunc: (Vid, EdgeWithVertices[VD, ED]) => A, - // mergeFunc: (A, A) => A, - // applyFunc: (Vertex[VD], Option[A]) => VD, - // numIter: Int, - // gatherDirection: EdgeDirection = EdgeDirection.In) : Graph[VD, ED] = { - - // var g = graph.cache() - - // def someGather(vid: Vid, edge: EdgeWithVertices[VD, ED]) = Some(gatherFunc(vid, edge)) - - // var i = 0 - // while (i < numIter) { - - // val accUpdates: RDD[(Vid, A)] = - // g.flatMapReduceNeighborhood(someGather, mergeFunc, gatherDirection) - - // g = g.updateVertices(accUpdates, applyFunc).cache() - - // i += 1 - // } - // g - // } - - // def iterateGAS[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( - // gatherFunc: (Vid, EdgeWithVertices[VD, ED]) => A, - // mergeFunc: (A, A) => A, - // applyFunc: (Vertex[VD], Option[A]) => VD, - // scatterFunc: (Vid, EdgeWithVertices[VD, ED]) => Boolean, - // numIter: Int, - // gatherDirection: EdgeDirection = EdgeDirection.In, - // scatterDirection: EdgeDirection = EdgeDirection.Out) : Graph[VD, ED] = { - - // var g = graph.mapVertices{ case Vertex(id,data) => Vertex(id, (true, data)) }.cache() - - // def gather(vid: Vid, e: EdgeWithVertices[(Boolean, VD), ED]) = { - // if(e.vertex(vid).data._1) { - // val edge = new EdgeWithVertices[VD,ED] - // edge.src = Vertex(e.src.id, e.src.data._2) - // edge.dst = Vertex(e.dst.id, e.dst.data._2) - // Some(gatherFunc(vid, edge)) - // } else { - // None - // } - // } - - // def apply(v: Vertex[(Boolean, VD)], accum: Option[A]) = { - // if(v.data._1) (true, applyFunc(Vertex(v.id, v.data._2), accum)) - // else (false, v.data._2) - // } - - // def scatter(rawVid: Vid, e: EdgeWithVertices[(Boolean, VD),ED]) = { - // val vid = e.otherVertex(rawVid).id - // if(e.vertex(vid).data._1) { - // val edge = new EdgeWithVertices[VD,ED] - // edge.src = Vertex(e.src.id, e.src.data._2) - // edge.dst = Vertex(e.dst.id, e.dst.data._2) - // Some(scatterFunc(vid, edge)) - // } else { - // None - // } - // } - - // def applyActive(v: Vertex[(Boolean, VD)], accum: Option[Boolean]) = - // (accum.getOrElse(false), v.data._2) - - // var i = 0 - // var numActive = g.numVertices - // while (i < numIter && numActive > 0) { - - // val accUpdates: RDD[(Vid, A)] = - // g.flatMapReduceNeighborhood(gather, mergeFunc, gatherDirection) - - // g = g.updateVertices(accUpdates, apply).cache() - - // // Scatter is basically a gather in the opposite direction so we reverse the edge direction - // val activeVertices: RDD[(Vid, Boolean)] = - // g.flatMapReduceNeighborhood(scatter, _ || _, scatterDirection.reverse) - - // g = g.updateVertices(activeVertices, applyActive).cache() - - // numActive = g.vertices.map(v => if (v.data._1) 1 else 0).reduce( _ + _ ) - // println("Number active vertices: " + numActive) - // i += 1 - // } - - // g.mapVertices(v => Vertex(v.id, v.data._2)) - // } + /** + * Execute the GraphLab Gather-Apply-Scatter API + * + * @todo finish documenting GraphLab Gather-Apply-Scatter API + * + * @param graph The graph on which to execute the GraphLab API + * @param gatherFunc The gather function is executed on each edge triplet + * adjacent to a vertex and returns an accumulator which + * is then merged using the merge function. + * @param mergeFunc An accumulative associative operation on the result of + * the gather type. + * @param applyFunc Takes a vertex and the final result of the merge operations + * on the adjacent edges and returns a new vertex value. + * @param scatterFunc Executed after the apply function the scatter function takes + * a triplet and signals whether the neighboring vertex program + * must be recomputed. + * @param numIter The maximum number of iterations to run. + * @param gatherDirection The direction of edges to consider during the gather phase + * @param scatterDirection The direction of edges to consider during the scatter phase + * + * @tparam VD The graph vertex attribute type + * @tparam ED The graph edge attribute type + * @tparam A The type accumulated during the gather phase + * @return the resulting graph after the algorithm converges + */ + def apply[VD: ClassManifest, ED: ClassManifest, A: ClassManifest](graph: Graph[VD, ED])( + gatherFunc: (Vid, EdgeTriplet[VD, ED]) => A, + mergeFunc: (A, A) => A, + applyFunc: (Vertex[VD], Option[A]) => VD, + scatterFunc: (Vid, EdgeTriplet[VD, ED]) => Boolean, + numIter: Int, + gatherDirection: EdgeDirection = EdgeDirection.In, + scatterDirection: EdgeDirection = EdgeDirection.Out): Graph[VD, ED] = { + + + // Add an active attribute to all vertices to track convergence. + var activeGraph = graph.mapVertices { + case Vertex(id, data) => (true, data) + }.cache() + + // The gather function wrapper strips the active attribute and + // only invokes the gather function on active vertices + def gather(vid: Vid, e: EdgeTriplet[(Boolean, VD), ED]) = { + if (e.vertex(vid).data._1) { + val edge = new EdgeTriplet[VD,ED] + edge.src = Vertex(e.src.id, e.src.data._2) + edge.dst = Vertex(e.dst.id, e.dst.data._2) + edge.data = e.data + Some(gatherFunc(vid, edge)) + } else { + None + } + } + + // The apply function wrapper strips the vertex of the active attribute + // and only invokes the apply function on active vertices + def apply(v: Vertex[(Boolean, VD)], accum: Option[A]) = { + if (v.data._1) (true, applyFunc(Vertex(v.id, v.data._2), accum)) + else (false, v.data._2) + } + + // The scatter function wrapper strips the vertex of the active attribute + // and only invokes the scatter function on active vertices + def scatter(rawVid: Vid, e: EdgeTriplet[(Boolean, VD), ED]) = { + val vid = e.otherVertex(rawVid).id + if (e.vertex(vid).data._1) { + val edge = new EdgeTriplet[VD,ED] + edge.src = Vertex(e.src.id, e.src.data._2) + edge.dst = Vertex(e.dst.id, e.dst.data._2) + edge.data = e.data +// val src = Vertex(e.src.id, e.src.data._2) +// val dst = Vertex(e.dst.id, e.dst.data._2) +// val edge = new EdgeTriplet[VD,ED](src, dst, e.data) + Some(scatterFunc(vid, edge)) + } else { + None + } + } + + // Used to set the active status of vertices for the next round + def applyActive(v: Vertex[(Boolean, VD)], accum: Option[Boolean]) = + (accum.getOrElse(false), v.data._2) + + // Main Loop --------------------------------------------------------------------- + var i = 0 + var numActive = activeGraph.numVertices + while (i < numIter && numActive > 0) { + + val accUpdates: RDD[(Vid, A)] = + activeGraph.aggregateNeighbors(gather, mergeFunc, gatherDirection) + + activeGraph = activeGraph.leftJoinVertices(accUpdates, apply).cache() + + // Scatter is basically a gather in the opposite direction so we reverse the edge direction + val activeVertices: RDD[(Vid, Boolean)] = + activeGraph.aggregateNeighbors(scatter, _ || _, scatterDirection.reverse) + + activeGraph = activeGraph.leftJoinVertices(activeVertices, applyActive).cache() + + numActive = activeGraph.vertices.map(v => if (v.data._1) 1 else 0).reduce(_ + _) + println("Number active vertices: " + numActive) + i += 1 + } + + // Remove the active attribute from the vertex data before returning the graph + activeGraph.mapVertices(v => v.data._2) + } } + + + + + + + + + From 03d0b858c807339b4221bedffa29ac76eef5352e Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 30 Jun 2013 15:38:58 -0700 Subject: [PATCH 0217/2521] 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 +++++++++----- .../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 70a9d7698c4..366afb2a2a7 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 9ac875de3a1..8844057a5c3 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 2de512f8965..ae61769e315 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 dc57035ebaf..eab8a0ff204 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 32c7ab86e99..5ffca54481b 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 From 5bbd0eec84867937713ceb8438f25a943765a084 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 30 Jun 2013 17:00:26 -0700 Subject: [PATCH 0218/2521] 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 37565ca8279..b8936314ecc 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 ae61769e315..3266db7af14 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 From 39ae073b5cd0dcfe4a00d9f205c88bad9df37870 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 30 Jun 2013 17:11:14 -0700 Subject: [PATCH 0219/2521] 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 7a31be98b2b..48e623fa1cb 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 From 0791581346517c8fa55540703f667f30abba73a0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 30 Jun 2013 23:07:40 -0700 Subject: [PATCH 0220/2521] More bug fixes --- .../main/scala/spark/graph/Analytics.scala | 2 +- graph/src/main/scala/spark/graph/Graph.scala | 23 +++++++++++-------- .../spark/graph/impl/EdgePartition.scala | 14 +++++++---- .../scala/spark/graph/impl/GraphImpl.scala | 11 +++++---- .../test/scala/spark/graph/GraphSuite.scala | 23 +++++++++++++++++++ 5 files changed, 52 insertions(+), 21 deletions(-) diff --git a/graph/src/main/scala/spark/graph/Analytics.scala b/graph/src/main/scala/spark/graph/Analytics.scala index 8f76622db0e..8acf863ff83 100644 --- a/graph/src/main/scala/spark/graph/Analytics.scala +++ b/graph/src/main/scala/spark/graph/Analytics.scala @@ -51,7 +51,7 @@ object Analytics extends Logging { (me_id, edge) => Some(edge.src.data._2 / edge.src.data._1), // gather (a: Double, b: Double) => a + b, // merge 1.0, - numIter).mapVertices{ case Vertex(id, (outDeg, r)) => Vertex(id, r) } + numIter).mapVertices{ case Vertex(id, (outDeg, r)) => r } } // /** diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index ef058a1fb53..6724e4ede5e 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -252,6 +252,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { * * @tparam U the type of entry in the table of updates * @tparam VD2 the new vertex value type + * * @param table the table to join with the vertices in the graph. The table * should contain at most one entry for each vertex. * @param mapFunc the function used to compute the new vertex values. The @@ -329,24 +330,26 @@ object Graph { import spark.graph.impl._ import spark.SparkContext._ - def apply(rawEdges: RDD[(Int,Int)], uniqueEdges: Boolean = true): Graph[Int, Int] = { - // Reduce to unique edges - val edges = - if(uniqueEdges) rawEdges.map{ case (s,t) => ((s,t),1) }.reduceByKey( _ + _ ) - .map{ case ((s,t), cnt) => Edge(s,t,cnt) } - else rawEdges.map{ case (s,t) => Edge(s,t,1) } + def apply(rawEdges: RDD[(Int, Int)], uniqueEdges: Boolean = true): Graph[Int, Int] = { + // Reduce to unique edges. + val edges: RDD[Edge[Int]] = + if (uniqueEdges) { + rawEdges.map((_, 1)).reduceByKey(_ + _).map { case ((s, t), cnt) => Edge(s, t, cnt) } + } else { + rawEdges.map { case (s, t) => Edge(s, t, 1) } + } // Determine unique vertices - val vertices = edges.flatMap{ case Edge(s, t, cnt) => Array((s,1), (t,1)) }.reduceByKey( _ + _ ) + val vertices: RDD[Vertex[Int]] = edges.flatMap{ case Edge(s, t, cnt) => Array((s, 1), (t, 1)) } + .reduceByKey(_ + _) .map{ case (id, deg) => Vertex(id, deg) } // Return graph new GraphImpl(vertices, edges) } - def apply[VD: ClassManifest, ED: ClassManifest](vertices: RDD[Vertex[VD]], edges: RDD[Edge[ED]]): Graph[VD, ED] = { + def apply[VD: ClassManifest, ED: ClassManifest]( + vertices: RDD[Vertex[VD]], edges: RDD[Edge[ED]]): Graph[VD, ED] = { new GraphImpl(vertices, edges) - } - implicit def graphToGraphOps[VD: ClassManifest, ED: ClassManifest](g: Graph[VD, ED]) = g.ops } diff --git a/graph/src/main/scala/spark/graph/impl/EdgePartition.scala b/graph/src/main/scala/spark/graph/impl/EdgePartition.scala index 0008534c0f8..0e092541c9f 100644 --- a/graph/src/main/scala/spark/graph/impl/EdgePartition.scala +++ b/graph/src/main/scala/spark/graph/impl/EdgePartition.scala @@ -1,6 +1,6 @@ package spark.graph.impl -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.ArrayBuilder import it.unimi.dsi.fastutil.ints.IntArrayList @@ -13,21 +13,25 @@ import spark.graph._ private[graph] class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassManifest] { + private var _data: Array[ED] = _ + private var _dataBuilder = ArrayBuilder.make[ED] + val srcIds: IntArrayList = new IntArrayList val dstIds: IntArrayList = new IntArrayList - // TODO: Specialize data. - val data: ArrayBuffer[ED] = new ArrayBuffer[ED] + + def data: Array[ED] = _data /** Add a new edge to the partition. */ def add(src: Vid, dst: Vid, d: ED) { srcIds.add(src) dstIds.add(dst) - data += d + _dataBuilder += d } def trim() { srcIds.trim() dstIds.trim() + _data = _dataBuilder.result() } def size: Int = srcIds.size @@ -41,7 +45,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) override def next(): Edge[ED] = { edge.src = srcIds.get(pos) edge.dst = dstIds.get(pos) - edge.data = data(pos) + edge.data = _data(pos) pos += 1 edge } diff --git a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala index e18a2f6b097..3f5cbc3d75b 100644 --- a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala @@ -150,7 +150,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } /** - * Same as mapReduceNeighborhood but map function can return none and there is no default value. + * Same as aggregateNeighbors but map function can return none and there is no default value. * As a consequence, the resulting table may be much smaller than the set of vertices. */ override def aggregateNeighbors[VD2: ClassManifest]( @@ -165,7 +165,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( part.map { v => (v._1, MutableTuple2(v._2, Option.empty[VD2])) } }, preservesPartitioning = true) - (new EdgeTripletRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable)) + new EdgeTripletRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable) .mapPartitions { part => val (vmap, edges) = part.next() val edgeSansAcc = new EdgeTriplet[VD, ED]() @@ -188,7 +188,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } if (gatherDirection == EdgeDirection.Out || gatherDirection == EdgeDirection.Both) { e.dst.data._2 = - if (e.dst.data._2.isEmpty) { + if (e.src.data._2.isEmpty) { mapFunc(edgeSansAcc.src.id, edgeSansAcc) } else { val tmp = mapFunc(edgeSansAcc.src.id, edgeSansAcc) @@ -218,7 +218,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } }, preservesPartitioning = true).cache() - new GraphImpl(newVTable.partitions.size, eTable.partitions.size, null, null, newVTable, eTable) + new GraphImpl(newVTable.partitions.length, eTable.partitions.length, null, null, newVTable, eTable) } override def joinVertices[U: ClassManifest]( @@ -239,7 +239,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } }, preservesPartitioning = true).cache() - new GraphImpl(newVTable.partitions.size, eTable.partitions.size, null, null, newVTable, eTable) + new GraphImpl(newVTable.partitions.length, eTable.partitions.length, null, null, newVTable, eTable) } @@ -307,6 +307,7 @@ object GraphImpl { .mapPartitionsWithIndex({ (pid, iter) => val edgePartition = new EdgePartition[ED] iter.foreach { case (_, (src, dst, data)) => edgePartition.add(src, dst, data) } + edgePartition.trim() Iterator((pid, edgePartition)) }, preservesPartitioning = true) } diff --git a/graph/src/test/scala/spark/graph/GraphSuite.scala b/graph/src/test/scala/spark/graph/GraphSuite.scala index 3d250aa18ce..87c8c158af5 100644 --- a/graph/src/test/scala/spark/graph/GraphSuite.scala +++ b/graph/src/test/scala/spark/graph/GraphSuite.scala @@ -3,10 +3,33 @@ package spark.graph import org.scalatest.FunSuite import spark.SparkContext +import spark.graph.impl.GraphImpl class GraphSuite extends FunSuite with LocalSparkContext { + test("aggregateNeighbors") { + + } + + test("joinVertices") { + sc = new SparkContext("local", "test") + val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two"), Vertex(3, "three")), 2) + val edges = sc.parallelize((Seq(Edge(1, 2, "onetwo")))) + val g: Graph[String, String] = new GraphImpl(vertices, edges) + + val tbl = sc.parallelize(Seq((1, 10), (2, 20))) + val g1 = g.joinVertices(tbl, (v: Vertex[String], u: Int) => v.data + u) + + val v = g1.vertices.collect().sortBy(_.id) + assert(v(0).data === "one10") + assert(v(1).data === "two20") + assert(v(2).data === "three") + + val e = g1.edges.collect() + assert(e(0).data === "onetwo") + } + // test("graph partitioner") { // sc = new SparkContext("local", "test") // val vertices = sc.parallelize(Seq(Vertex(1, "one"), Vertex(2, "two"))) From 3296d132b6ce042843de6e7384800e089b49e5fa Mon Sep 17 00:00:00 2001 From: root Date: Mon, 1 Jul 2013 02:45:00 +0000 Subject: [PATCH 0221/2521] Fix performance bug with new Python code not using buffered streams --- core/src/main/scala/spark/SparkEnv.scala | 3 +- .../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 7ccde2e8182..ec59b4f48fd 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 63140cf37f5..3f283afa62f 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) From ec31e68d5df259e6df001529235d8c906ff02a6f Mon Sep 17 00:00:00 2001 From: root Date: Mon, 1 Jul 2013 06:20:14 +0000 Subject: [PATCH 0222/2521] 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 --- .../scala/spark/api/python/PythonRDD.scala | 10 +++-- .../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 3f283afa62f..31d8ea89d43 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 8844411d738..85d1dfeac8b 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 78a2da1e18e..78c9457b84d 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 handle_sigchld(*args): 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 handle_sigchld(*args): # 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 shutdown(): 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 handle_sigchld(*args): 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 handle_sigchld(*args): 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) From 7cd490ef5ba28df31f5e061eff83c855731dfca4 Mon Sep 17 00:00:00 2001 From: root Date: Mon, 1 Jul 2013 06:25:17 +0000 Subject: [PATCH 0223/2521] 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 7f1e7cf93dc..e8aaac74d03 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: From 2943edf8eece20b21ff568b401cc5e8323ce9c07 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 1 Jul 2013 00:24:30 -0700 Subject: [PATCH 0224/2521] Fixed another bug .. --- graph/src/main/scala/spark/graph/impl/GraphImpl.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala index 3f5cbc3d75b..711446d8e0c 100644 --- a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala @@ -110,7 +110,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( part.map { v => (v._1, MutableTuple2(v._2, Option.empty[VD2])) } }, preservesPartitioning = true) - (new EdgeTripletRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable)) + new EdgeTripletRDD[MutableTuple2[VD, Option[VD2]], ED](newVTable, eTable) .mapPartitions { part => val (vmap, edges) = part.next() val edgeSansAcc = new EdgeTriplet[VD, ED]() @@ -187,7 +187,7 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( } } if (gatherDirection == EdgeDirection.Out || gatherDirection == EdgeDirection.Both) { - e.dst.data._2 = + e.src.data._2 = if (e.src.data._2.isEmpty) { mapFunc(edgeSansAcc.src.id, edgeSansAcc) } else { From 5de326db7d5235dbf259a4c8b388cb24c40fd2ec Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 1 Jul 2013 09:19:45 -0700 Subject: [PATCH 0225/2521] 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 bac984b5c94..8de3aa91a4d 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 ed96fc2994e..c6f87fc6520 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("")} + } } From 735c951a09f71512cf7be834802d1e88567ca9ae Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 1 Jul 2013 09:33:22 -0700 Subject: [PATCH 0226/2521] Adding test script --- .../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 00000000000..24cfe36aaac --- /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) + } + } + } +} From 8688689387eb6e2ac34add5e460dc54fe32b6def Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 1 Jul 2013 13:40:12 -0700 Subject: [PATCH 0227/2521] Various formatting changes --- core/src/main/scala/spark/Utils.scala | 8 ++++---- .../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 6966ee9ee94..af926b1e36f 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 cd04542faab..1e675ab2cb2 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 127ab5ebc21..aa191f628b9 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") } } From 30b90342410a196d99d25113fbc1e37ce2a8200f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 1 Jul 2013 13:48:01 -0700 Subject: [PATCH 0228/2521] 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 b6a26245fc9..6ae1cef9402 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() } From 1025d7d1efb20e3683a604259c92eeef400cc3fa Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 1 Jul 2013 14:40:53 -0700 Subject: [PATCH 0229/2521] 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 9e4b8843790..57f85bafd2e 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 00000000000..c853b44b765 --- /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 00000000000..7b79290d1bd --- /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 From 9a42d04efa1ff620306ec8864c9f95186fc1a199 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 1 Jul 2013 14:43:13 -0700 Subject: [PATCH 0230/2521] 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 57f85bafd2e..bc6f9c10d5a 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 } From 2f2c7e6a294ae9297ae7c9b84cfc352384d373aa Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 1 Jul 2013 16:23:23 -0700 Subject: [PATCH 0231/2521] Added a correctEdges function. --- graph/src/main/scala/spark/graph/Graph.scala | 1 + graph/src/main/scala/spark/graph/impl/GraphImpl.scala | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/graph/src/main/scala/spark/graph/Graph.scala b/graph/src/main/scala/spark/graph/Graph.scala index 6724e4ede5e..f51171f0d62 100644 --- a/graph/src/main/scala/spark/graph/Graph.scala +++ b/graph/src/main/scala/spark/graph/Graph.scala @@ -141,6 +141,7 @@ abstract class Graph[VD: ClassManifest, ED: ClassManifest] { def mapTriplets[ED2: ClassManifest]( map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2] + def correctEdges(): Graph[VD, ED] /** * Construct a new graph with all the edges reversed. If this graph contains diff --git a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala index 711446d8e0c..775d0686e75 100644 --- a/graph/src/main/scala/spark/graph/impl/GraphImpl.scala +++ b/graph/src/main/scala/spark/graph/impl/GraphImpl.scala @@ -91,6 +91,12 @@ class GraphImpl[VD: ClassManifest, ED: ClassManifest] protected ( newGraph(vertices, triplets.map(e => Edge(e.src.id, e.dst.id, f(e)))) } + override def correctEdges(): Graph[VD, ED] = { + val sc = vertices.context + val vset = sc.broadcast(vertices.map(_.id).collect().toSet) + val newEdges = edges.filter(e => vset.value.contains(e.src) && vset.value.contains(e.dst)) + Graph(vertices, newEdges) + } ////////////////////////////////////////////////////////////////////////////////////////////////// // Lower level transformation methods From 6fdbc68f2c5b220d1618d5a78d46aa0f844cae45 Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Mon, 1 Jul 2013 16:05:55 -0700 Subject: [PATCH 0232/2521] 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 3e5271ec2f9..78ec58729b3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -193,6 +193,11 @@ hadoop-yarn-common provided
    + + org.apache.hbase + hbase + 0.94.6 + From 8ca1cc1786a376b227ba69872efbf8e26952b80d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 2 Jul 2013 15:56:34 -0700 Subject: [PATCH 0233/2521] Adding truncation for log files --- core/src/main/scala/spark/Utils.scala | 14 +++++++++ .../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 af926b1e36f..c1616de6419 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 2e6279566c5..16564d56197 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 aa191f628b9..e4bb3abc339 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) + } } From 39e2325675e40c2231e26a7e693035f54039b1d5 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 2 Jul 2013 16:28:40 -0700 Subject: [PATCH 0234/2521] 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 c1616de6419..c5faf79ac31 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 = { 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: [PATCH 0235/2521] 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. --- .../scala/spark/deploy/SparkHadoopUtil.scala | 4 ++++ .../scala/spark/deploy/SparkHadoopUtil.scala | 7 +++++++ .../scala/spark/deploy/yarn/Client.scala | 14 +++++++++----- .../scala/spark/deploy/yarn/ClientArguments.scala | 8 +------- .../scala/spark/deploy/yarn/WorkerRunnable.scala | 12 +++++++----- .../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 ++ .../scala/spark/scheduler/InputFormatInfo.scala | 11 +++-------- .../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 a0fb4fe25d1..f1c86de4cc9 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 b96c047e10e..301a57fffa2 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 f20cc31c7c4..514c17f241e 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 24110558e7d..07e7edea363 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 e22d256a844..cc6f3344a18 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 a0fb4fe25d1..f1c86de4cc9 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 b3976011849..c9d698fc090 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 c3a56938b5c..6c37203707b 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 cbf5512e247..07c103503c8 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 30a56d7135a..17d0ea4f80f 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 20ee1d3c5de..8d0a83d4399 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) From 96130c30d90abf155731346488f79c034bcaaf6a Mon Sep 17 00:00:00 2001 From: Gavin Li Date: Wed, 3 Jul 2013 05:49:04 +0000 Subject: [PATCH 0236/2521] add compression codec trait and snappy compression --- .../scala/spark/storage/BlockManager.scala | 20 +++++++++++++++++-- .../spark/storage/CompressionCodec.scala | 13 ++++++++++++ .../main/scala/spark/storage/DiskStore.scala | 1 - .../spark/storage/LZFCompressionCodec.scala | 16 +++++++++++++++ .../storage/SnappyCompressionCodec.scala | 18 +++++++++++++++++ 5 files changed, 65 insertions(+), 3 deletions(-) create mode 100644 core/src/main/scala/spark/storage/CompressionCodec.scala create mode 100644 core/src/main/scala/spark/storage/LZFCompressionCodec.scala create mode 100644 core/src/main/scala/spark/storage/SnappyCompressionCodec.scala diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 9b39d3aadfd..2d4a3502c69 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -141,6 +141,8 @@ private[spark] class BlockManager( val metadataCleaner = new MetadataCleaner("BlockManager", this.dropOldBlocks) initialize() + var compressionCodec: CompressionCodec = null + /** * Construct a BlockManager with a memory limit set based on system properties. */ @@ -902,8 +904,15 @@ private[spark] 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 (compressionCodec == null) { + compressionCodec = Class.forName(System.getProperty("spark.storage.compression.codec", + "spark.storage.LZFCompressionCodec"), true, Thread.currentThread.getContextClassLoader) + .newInstance().asInstanceOf[CompressionCodec] + } + if (shouldCompress(blockId)) { - (new LZFOutputStream(s)).setFinishBlockOnFlush(true) + //(new LZFOutputStream(s)).setFinishBlockOnFlush(true) + compressionCodec.compressionOutputStream(s) } else { s } @@ -913,7 +922,14 @@ private[spark] class BlockManager( * Wrap an input stream for compression if block compression is enabled for its block type */ def wrapForCompression(blockId: String, s: InputStream): InputStream = { - if (shouldCompress(blockId)) new LZFInputStream(s) else s + if (compressionCodec == null) { + compressionCodec = Class.forName(System.getProperty("spark.storage.compression.codec", + "spark.storage.LZFCompressionCodec"), true, Thread.currentThread.getContextClassLoader) + .newInstance().asInstanceOf[CompressionCodec] + } + + if (shouldCompress(blockId)) /*new LZFInputStream(s) */ + compressionCodec.compressionInputStream(s) else s } def dataSerialize( diff --git a/core/src/main/scala/spark/storage/CompressionCodec.scala b/core/src/main/scala/spark/storage/CompressionCodec.scala new file mode 100644 index 00000000000..cd80de33f61 --- /dev/null +++ b/core/src/main/scala/spark/storage/CompressionCodec.scala @@ -0,0 +1,13 @@ +package spark.storage + +import java.io.{InputStream, OutputStream} + + +/** + * CompressionCodec allows the customization of the compression codec + */ +trait CompressionCodec { + def compressionOutputStream(s: OutputStream): OutputStream + + def compressionInputStream(s: InputStream): InputStream +} diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index da859eebcb9..221e285192f 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -49,7 +49,6 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) override def close() { if (initialized) { objOut.close() - bs.close() channel = null bs = null objOut = null diff --git a/core/src/main/scala/spark/storage/LZFCompressionCodec.scala b/core/src/main/scala/spark/storage/LZFCompressionCodec.scala new file mode 100644 index 00000000000..3328b949efe --- /dev/null +++ b/core/src/main/scala/spark/storage/LZFCompressionCodec.scala @@ -0,0 +1,16 @@ +package spark.storage + +import java.io.{InputStream, OutputStream} + +import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} + +/** + * LZF implementation of [[spark.storage.CompressionCodec]] + */ +class LZFCompressionCodec extends CompressionCodec { + def compressionOutputStream(s: OutputStream): OutputStream = + (new LZFOutputStream(s)).setFinishBlockOnFlush(true) + + def compressionInputStream(s: InputStream): InputStream = + new LZFInputStream(s) +} diff --git a/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala b/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala new file mode 100644 index 00000000000..dc8546b0393 --- /dev/null +++ b/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala @@ -0,0 +1,18 @@ +package spark.storage + +import java.io.{InputStream, OutputStream} + +import org.xerial.snappy.SnappyOutputStream + +/** + * Snappy implementation of [[spark.storage.CompressionCodec]] + * block size can be configured by spark.snappy.block.size + */ +class SnappyCompressionCodec extends CompressionCodec { + def compressionOutputStream(s: OutputStream): OutputStream = + new SnappyOutputStream(s, + System.getProperty("spark.snappy.block.size", "32768").toInt) + + def compressionInputStream(s: InputStream): InputStream = + new SnappyInputStream(s) +} From 04567a1771ec02e1efcc3ce078291a975fbb4e68 Mon Sep 17 00:00:00 2001 From: Mingfei Date: Wed, 3 Jul 2013 17:43:37 +0800 Subject: [PATCH 0237/2521] 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 5e4692162e9..f65e398d954 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, From 94238aae57475030f6e88102a83c7809c5835494 Mon Sep 17 00:00:00 2001 From: Gavin Li Date: Wed, 3 Jul 2013 18:08:38 +0000 Subject: [PATCH 0238/2521] fix dependencies --- core/src/main/scala/spark/storage/SnappyCompressionCodec.scala | 2 +- project/SparkBuild.scala | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala b/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala index dc8546b0393..62b00ef3f6c 100644 --- a/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala +++ b/core/src/main/scala/spark/storage/SnappyCompressionCodec.scala @@ -2,7 +2,7 @@ package spark.storage import java.io.{InputStream, OutputStream} -import org.xerial.snappy.SnappyOutputStream +import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} /** * Snappy implementation of [[spark.storage.CompressionCodec]] diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 07572201ded..f824826af3c 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -162,7 +162,8 @@ object SparkBuild extends Build { "cc.spray" % "spray-json_2.9.2" % "1.1.1" excludeAll(excludeNetty), "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", + "org.xerial.snappy" % "snappy-java" % "1.0.5" ) ++ ( if (HADOOP_MAJOR_VERSION == "2") { if (HADOOP_YARN) { From 0f06d6217d0da67633d9ede73df67b94133fd3d5 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 4 Jul 2013 01:05:39 -0700 Subject: [PATCH 0239/2521] 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 e93cc3b4850..0cff0d5b019 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) } From 6ccfb73ca92a72d5cca6a504d9ee332a16d9293d Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Thu, 4 Jul 2013 19:19:44 +0800 Subject: [PATCH 0240/2521] 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 00000000000..04a6b418dc1 --- /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 5a688b0ebb9..6e573b18836 100644 --- a/core/src/test/resources/fairscheduler.xml +++ b/core/src/test/resources/fairscheduler.xml @@ -1,3 +1,4 @@ + 2 From c0c3155c3c0e37ba31cbeeccb21654bdf5ec0cfc Mon Sep 17 00:00:00 2001 From: Lian Cheng Date: Fri, 5 Jul 2013 00:54:10 +0800 Subject: [PATCH 0241/2521] 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 3e965e82ac6..87f304b6cd6 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() } } From 7687ed529217161ea681939870f6e1e042139e65 Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Thu, 4 Jul 2013 13:48:33 -0700 Subject: [PATCH 0242/2521] 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 48e623fa1cb..5b4c1d48f79 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 From e4ff544a8d43a23ad2110556102bb83e34ae71b4 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 5 Jul 2013 10:34:45 +0800 Subject: [PATCH 0243/2521] 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 f7d60be5dbf..51b10ed0450 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() { From 729e463f649332c5480d2d175d42d4ba0dd3cb74 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 11:13:41 -0700 Subject: [PATCH 0244/2521] Import RidgeRegression example Conflicts: run --- .../main/scala/spark/ml/RidgeRegression.scala | 110 ++++++++++++++++++ .../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 00000000000..7896873d447 --- /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 00000000000..22a1e4613b9 --- /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 5e4692162e9..731671c23b4 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( From 6a9a9a364ce3b158c4162e401f90eb4d305104e8 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 4 Jun 2013 16:27:02 -0700 Subject: [PATCH 0245/2521] Minor clean up of the RidgeRegression code. I am not even sure why I did this :s. --- .../main/scala/spark/ml/RidgeRegression.scala | 38 +++++++++---------- .../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 7896873d447..b8b632e1113 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 22a1e4613b9..ff8640bb50e 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 731671c23b4..aa877ad4a7c 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", From c070decb8e746bb295b3d95f49713d0d389f92b3 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 6 Jun 2013 11:04:33 -0700 Subject: [PATCH 0246/2521] Add methods to normalize the data before training Also update model after training based appropriately. --- .../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 b8b632e1113..881316c0fc5 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() } } From c8169c0a3331c8c89cd963f8f14ea69dbe3ef966 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 6 Jun 2013 11:05:11 -0700 Subject: [PATCH 0247/2521] 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 00000000000..fdd16e36b49 --- /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 From 4dc13bf5be713443f22ca9c27530fb1d3cf48993 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 7 Jun 2013 15:10:13 -0700 Subject: [PATCH 0248/2521] Revert back to closed form CV error --- .../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 881316c0fc5..2288a284e21 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 } From b9d9b6f98168ae211e36e94ee4c2d8defa525169 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sat, 8 Jun 2013 17:04:30 -0700 Subject: [PATCH 0249/2521] Add a unit test for Ridge Regression --- ml/src/test/resources/log4j.properties | 11 +++++ .../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 00000000000..390c92763cb --- /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 00000000000..cc46602eddf --- /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") + } +} From 282c8ed78843fc9a3555b4b6dbdb709d2632d360 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 10 Jun 2013 23:16:11 -0700 Subject: [PATCH 0250/2521] 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 ++++ .../main/scala/spark/ml/GradientDescent.scala | 46 ++++++ .../scala/spark/ml/LogisticRegression.scala | 77 ++++++++++ .../ml/LogisticRegressionGenerator.scala | 40 ++++++ ml/src/main/scala/spark/ml/MLUtils.scala | 27 ++++ ml/src/main/scala/spark/ml/Regression.scala | 53 +++++++ .../main/scala/spark/ml/RidgeRegression.scala | 134 +++++------------- .../spark/ml/RidgeRegressionGenerator.scala | 24 +--- ml/src/main/scala/spark/ml/Updater.scala | 16 +++ .../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 00000000000..a7e83271338 --- /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 00000000000..80d4c44a134 --- /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 00000000000..5647bf5a84b --- /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 00000000000..ff46cfa46d7 --- /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 00000000000..eab5db61bdd --- /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 00000000000..e19d58396b5 --- /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 2288a284e21..c030223b85b 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 ff8640bb50e..e6d3e94787d 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 00000000000..bacb0593774 --- /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 cc46602eddf..50ef4720751 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 From 48770419bdc09b2cf1ddef3cbdd2c1d23f1748ac Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 10 Jun 2013 23:17:06 -0700 Subject: [PATCH 0251/2521] 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 00000000000..29bcb8acbaa --- /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 From fd137bd7c6b924dfb818b8ff45b9c6bec0d47700 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 12 Jun 2013 16:30:04 -0700 Subject: [PATCH 0252/2521] Address Reynold's comments. Also use a builder pattern to construct the regression classes. --- ml/src/main/scala/spark/ml/Gradient.scala | 19 +++-- .../main/scala/spark/ml/GradientDescent.scala | 21 ++++- .../scala/spark/ml/LogisticRegression.scala | 80 ++++++++++++++---- ml/src/main/scala/spark/ml/MLUtils.scala | 22 +++-- ml/src/main/scala/spark/ml/Regression.scala | 82 +++++++++++-------- .../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 a7e83271338..fe361870001 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 80d4c44a134..a03285bd531 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 5647bf5a84b..e8e4f96dd31 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 eab5db61bdd..d62198f4260 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 e19d58396b5..26a3a05545d 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 c030223b85b..f67fb451347 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 bacb0593774..3952008e28d 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) } From ed32ec2b3bf93b1090ade63f2be25c2deba890b9 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 12 Jun 2013 16:31:39 -0700 Subject: [PATCH 0253/2521] 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 50ef4720751..eb67974cf0d 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) From 2d0e64900e798db6de5d6f7f725c372a3f20cdd1 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 13 Jun 2013 09:53:34 -0700 Subject: [PATCH 0254/2521] Convert regression classes to builder pattern. Remove extraneous methods and classes --- .../scala/spark/ml/LogisticRegression.scala | 77 ++++++++----------- .../main/scala/spark/ml/RidgeRegression.scala | 64 ++++++--------- .../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 e8e4f96dd31..ddc138cacf8 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 f67fb451347..42b370d43b7 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 eb67974cf0d..06b5b1ae311 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) From 6aadaf4d71e9ed95363638673bda48cecd483c03 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 13 Jun 2013 13:17:20 -0700 Subject: [PATCH 0255/2521] Move normalization to MLUtils and remove Regression trait. --- .../scala/spark/ml/LogisticRegression.scala | 64 +++++++----------- ml/src/main/scala/spark/ml/MLUtils.scala | 46 +++++++++++++ ml/src/main/scala/spark/ml/Regression.scala | 43 ++---------- .../main/scala/spark/ml/RidgeRegression.scala | 66 +++++++------------ .../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 ddc138cacf8..76c4fe54c8e 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 d62198f4260..b2361e55646 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 26a3a05545d..f23524bfa8d 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 42b370d43b7..36db2570d99 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 06b5b1ae311..07da5081b3c 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) From 3a6924cb8ff082eed0243b0684de88a858beee7d Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 13 Jun 2013 13:25:10 -0700 Subject: [PATCH 0256/2521] 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 a03285bd531..564f89e5eeb 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 76c4fe54c8e..c134f8d77d4 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]) { From 76acc9fe9d76c43ee063642db3ea2a9190d97eb6 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 13 Jun 2013 14:15:03 -0700 Subject: [PATCH 0257/2521] Make regression arguments private and add method to predict one point --- .../scala/spark/ml/LogisticRegression.scala | 13 ++++++-- ml/src/main/scala/spark/ml/Regression.scala | 31 +++++-------------- .../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 c134f8d77d4..4dc883fbf80 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 f23524bfa8d..9d7424d656b 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 36db2570d99..628fc259337 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) From 6dd3a816c8d438e1d8beebe0a1cbf5d90ef6e3bc Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 13 Jun 2013 15:11:21 -0700 Subject: [PATCH 0258/2521] 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 4dc883fbf80..3417bfe8822 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 628fc259337..3cbb0653c3d 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) From 43b398db6a2008a9dc0343df0ec0b24f96fa65c1 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 14 Jun 2013 16:13:12 -0700 Subject: [PATCH 0259/2521] Fix logistic regression to not center data. Also add a feature to get the intercept correct and test these using a small unit test. --- .../scala/spark/ml/LogisticRegression.scala | 17 ++---- .../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 3417bfe8822..14c237c5edc 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 00000000000..d66c2c8b353 --- /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") + } +} From 39ed41652beaad63c03540411a51ed82c1126e6d Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sat, 15 Jun 2013 09:03:20 -0700 Subject: [PATCH 0260/2521] Move to regression, util and gradient packages --- ml/src/main/scala/spark/ml/{ => gradient}/Gradient.scala | 2 +- .../main/scala/spark/ml/{ => gradient}/GradientDescent.scala | 2 +- ml/src/main/scala/spark/ml/{ => gradient}/Updater.scala | 2 +- .../scala/spark/ml/{ => regression}/LogisticRegression.scala | 4 +++- .../ml/{ => regression}/LogisticRegressionGenerator.scala | 3 ++- ml/src/main/scala/spark/ml/{ => regression}/Regression.scala | 2 +- .../scala/spark/ml/{ => regression}/RidgeRegression.scala | 3 ++- .../spark/ml/{ => regression}/RidgeRegressionGenerator.scala | 3 ++- ml/src/main/scala/spark/ml/{ => util}/MLUtils.scala | 2 +- .../spark/ml/{ => regression}/LogisticRegressionSuite.scala | 2 +- .../spark/ml/{ => regression}/RidgeRegressionSuite.scala | 2 +- 11 files changed, 16 insertions(+), 11 deletions(-) rename ml/src/main/scala/spark/ml/{ => gradient}/Gradient.scala (97%) rename ml/src/main/scala/spark/ml/{ => gradient}/GradientDescent.scala (98%) rename ml/src/main/scala/spark/ml/{ => gradient}/Updater.scala (97%) rename ml/src/main/scala/spark/ml/{ => regression}/LogisticRegression.scala (97%) rename ml/src/main/scala/spark/ml/{ => regression}/LogisticRegressionGenerator.scala (95%) rename ml/src/main/scala/spark/ml/{ => regression}/Regression.scala (95%) rename ml/src/main/scala/spark/ml/{ => regression}/RidgeRegression.scala (98%) rename ml/src/main/scala/spark/ml/{ => regression}/RidgeRegressionGenerator.scala (96%) rename ml/src/main/scala/spark/ml/{ => util}/MLUtils.scala (99%) rename ml/src/test/scala/spark/ml/{ => regression}/LogisticRegressionSuite.scala (98%) rename ml/src/test/scala/spark/ml/{ => regression}/RidgeRegressionSuite.scala (98%) diff --git a/ml/src/main/scala/spark/ml/Gradient.scala b/ml/src/main/scala/spark/ml/gradient/Gradient.scala similarity index 97% rename from ml/src/main/scala/spark/ml/Gradient.scala rename to ml/src/main/scala/spark/ml/gradient/Gradient.scala index fe361870001..50795c01a25 100644 --- a/ml/src/main/scala/spark/ml/Gradient.scala +++ b/ml/src/main/scala/spark/ml/gradient/Gradient.scala @@ -1,4 +1,4 @@ -package spark.ml +package spark.ml.gradient import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/GradientDescent.scala b/ml/src/main/scala/spark/ml/gradient/GradientDescent.scala similarity index 98% rename from ml/src/main/scala/spark/ml/GradientDescent.scala rename to ml/src/main/scala/spark/ml/gradient/GradientDescent.scala index 564f89e5eeb..8cd2a69a3a1 100644 --- a/ml/src/main/scala/spark/ml/GradientDescent.scala +++ b/ml/src/main/scala/spark/ml/gradient/GradientDescent.scala @@ -1,4 +1,4 @@ -package spark.ml +package spark.ml.gradient import spark.{Logging, RDD, SparkContext} import spark.SparkContext._ diff --git a/ml/src/main/scala/spark/ml/Updater.scala b/ml/src/main/scala/spark/ml/gradient/Updater.scala similarity index 97% rename from ml/src/main/scala/spark/ml/Updater.scala rename to ml/src/main/scala/spark/ml/gradient/Updater.scala index 3952008e28d..27f03897b00 100644 --- a/ml/src/main/scala/spark/ml/Updater.scala +++ b/ml/src/main/scala/spark/ml/gradient/Updater.scala @@ -1,4 +1,4 @@ -package spark.ml +package spark.ml.gradient import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/LogisticRegression.scala b/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala similarity index 97% rename from ml/src/main/scala/spark/ml/LogisticRegression.scala rename to ml/src/main/scala/spark/ml/regression/LogisticRegression.scala index 14c237c5edc..1508f6934a0 100644 --- a/ml/src/main/scala/spark/ml/LogisticRegression.scala +++ b/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala @@ -1,6 +1,8 @@ -package spark.ml +package spark.ml.regression import spark.{Logging, RDD, SparkContext} +import spark.ml.gradient._ +import spark.ml.util.MLUtils import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/LogisticRegressionGenerator.scala b/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala similarity index 95% rename from ml/src/main/scala/spark/ml/LogisticRegressionGenerator.scala rename to ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala index ff46cfa46d7..1617eac2054 100644 --- a/ml/src/main/scala/spark/ml/LogisticRegressionGenerator.scala +++ b/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala @@ -1,6 +1,7 @@ -package spark.ml +package spark.ml.regression import spark.{RDD, SparkContext} +import spark.ml.util.MLUtils import org.apache.commons.math3.distribution.NormalDistribution import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/Regression.scala b/ml/src/main/scala/spark/ml/regression/Regression.scala similarity index 95% rename from ml/src/main/scala/spark/ml/Regression.scala rename to ml/src/main/scala/spark/ml/regression/Regression.scala index 9d7424d656b..4a20f513b7b 100644 --- a/ml/src/main/scala/spark/ml/Regression.scala +++ b/ml/src/main/scala/spark/ml/regression/Regression.scala @@ -1,4 +1,4 @@ -package spark.ml +package spark.ml.regression import spark.RDD diff --git a/ml/src/main/scala/spark/ml/RidgeRegression.scala b/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala similarity index 98% rename from ml/src/main/scala/spark/ml/RidgeRegression.scala rename to ml/src/main/scala/spark/ml/regression/RidgeRegression.scala index 3cbb0653c3d..16d146ac273 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegression.scala +++ b/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala @@ -1,7 +1,8 @@ -package spark.ml +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 diff --git a/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala b/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala similarity index 96% rename from ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala rename to ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala index e6d3e94787d..ac7f1e73203 100644 --- a/ml/src/main/scala/spark/ml/RidgeRegressionGenerator.scala +++ b/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala @@ -1,6 +1,7 @@ -package spark.ml +package spark.ml.regression import spark.{RDD, SparkContext} +import spark.ml.util.MLUtils import org.apache.commons.math3.distribution.NormalDistribution import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/MLUtils.scala b/ml/src/main/scala/spark/ml/util/MLUtils.scala similarity index 99% rename from ml/src/main/scala/spark/ml/MLUtils.scala rename to ml/src/main/scala/spark/ml/util/MLUtils.scala index b2361e55646..76e1bbd36d0 100644 --- a/ml/src/main/scala/spark/ml/MLUtils.scala +++ b/ml/src/main/scala/spark/ml/util/MLUtils.scala @@ -1,4 +1,4 @@ -package spark.ml +package spark.ml.util import spark.{RDD, SparkContext} import spark.SparkContext._ diff --git a/ml/src/test/scala/spark/ml/LogisticRegressionSuite.scala b/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala similarity index 98% rename from ml/src/test/scala/spark/ml/LogisticRegressionSuite.scala rename to ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala index d66c2c8b353..ce388ecc267 100644 --- a/ml/src/test/scala/spark/ml/LogisticRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala @@ -1,4 +1,4 @@ -package spark.ml +package spark.ml.regression import spark.SparkContext import spark.SparkContext._ diff --git a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala b/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala similarity index 98% rename from ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala rename to ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala index 07da5081b3c..6d5f13d6f5f 100644 --- a/ml/src/test/scala/spark/ml/RidgeRegressionSuite.scala +++ b/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala @@ -1,4 +1,4 @@ -package spark.ml +package spark.ml.regression import spark.SparkContext import spark.SparkContext._ From 05be233ce2716fe57cf44433d52734ded29e3506 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 21 Jun 2013 15:58:01 +0200 Subject: [PATCH 0261/2521] Removed dependency on Apache Commons Math --- .../regression/LogisticRegressionGenerator.scala | 12 ++++++------ .../ml/regression/RidgeRegressionGenerator.scala | 12 ++++++------ .../ml/regression/LogisticRegressionSuite.scala | 11 ++++++----- .../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 1617eac2054..6d37aad047c 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 ac7f1e73203..75854fe1de8 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 ce388ecc267..53d97899797 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 6d5f13d6f5f..795cda13794 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 aa877ad4a7c..5dbb5d4a65e 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" ) ) From d903b3887f13b5ef500cd31227b4563d372e6cd8 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 21 Jun 2013 19:09:17 +0200 Subject: [PATCH 0262/2521] 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 ++++++++++++++++++ .../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 00000000000..e476cc23e04 --- /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 00000000000..84339780e24 --- /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 00000000000..52a8984bc79 --- /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 00000000000..70f350857fa --- /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") + } +} + From 6f0ebb2db24272390ebd481e50a08336e37d2b1d Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 21 Jun 2013 19:13:22 +0200 Subject: [PATCH 0263/2521] 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 84339780e24..dca5bfc8f11 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} From 3c046a6eca182d720b58f5cdfd17b6201664e716 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 21 Jun 2013 19:24:06 +0200 Subject: [PATCH 0264/2521] 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 dca5bfc8f11..a1ae28a38da 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)) } From d3ce898b8eec066d69892d29b0c6531790902d78 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 21 Jun 2013 20:00:56 +0200 Subject: [PATCH 0265/2521] Scaffolding and model for K-means --- .../scala/spark/ml/clustering/KMeans.scala | 117 ++++++++++++++++++ .../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 00000000000..fd15da9b43e --- /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 00000000000..f2f26c47b87 --- /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 76e1bbd36d0..48292649c39 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 + } } From 43dae967d7095d4ebd1ab5919e8907da640e48d3 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 15:16:24 -0400 Subject: [PATCH 0266/2521] Renamed "als" package to "recommendation" --- .../spark/ml/{als => recommendation}/ALS.scala | 14 +++++++------- .../MatrixFactorizationModel.scala | 2 +- .../ml/{als => recommendation}/ALSSuite.scala | 2 +- 3 files changed, 9 insertions(+), 9 deletions(-) rename ml/src/main/scala/spark/ml/{als => recommendation}/ALS.scala (98%) rename ml/src/main/scala/spark/ml/{als => recommendation}/MatrixFactorizationModel.scala (95%) rename ml/src/test/scala/spark/ml/{als => recommendation}/ALSSuite.scala (98%) diff --git a/ml/src/main/scala/spark/ml/als/ALS.scala b/ml/src/main/scala/spark/ml/recommendation/ALS.scala similarity index 98% rename from ml/src/main/scala/spark/ml/als/ALS.scala rename to ml/src/main/scala/spark/ml/recommendation/ALS.scala index a1ae28a38da..6717315fa7b 100644 --- a/ml/src/main/scala/spark/ml/als/ALS.scala +++ b/ml/src/main/scala/spark/ml/recommendation/ALS.scala @@ -1,4 +1,4 @@ -package spark.ml.als +package spark.ml.recommendation import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.util.Random @@ -15,7 +15,7 @@ import org.jblas.{DoubleMatrix, SimpleBlas, Solve} * 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( +private[recommendation] case class OutLinkBlock( elementIds: Array[Int], shouldSend: Array[BitSet]) @@ -29,7 +29,7 @@ private[als] case class OutLinkBlock( * 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( +private[recommendation] case class InLinkBlock( elementIds: Array[Int], ratingsForBlock: Array[Array[(Array[Int], Array[Double])]]) @@ -51,7 +51,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l { 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. */ @@ -198,7 +198,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l lambda: Double) : RDD[(Int, Array[Array[Double]])] = { - val numBlocks = products.partitions.size + 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) { @@ -224,7 +224,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l 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 @@ -321,7 +321,7 @@ object ALS { ratings: RDD[(Int, Int, Double)], rank: Int, iterations: Int, - lambda: Double, + lambda: Double, blocks: Int) : MatrixFactorizationModel = { diff --git a/ml/src/main/scala/spark/ml/als/MatrixFactorizationModel.scala b/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala similarity index 95% rename from ml/src/main/scala/spark/ml/als/MatrixFactorizationModel.scala rename to ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala index 52a8984bc79..7bd33a87ee2 100644 --- a/ml/src/main/scala/spark/ml/als/MatrixFactorizationModel.scala +++ b/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala @@ -1,4 +1,4 @@ -package spark.ml.als +package spark.ml.recommendation import spark.RDD import spark.SparkContext._ diff --git a/ml/src/test/scala/spark/ml/als/ALSSuite.scala b/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala similarity index 98% rename from ml/src/test/scala/spark/ml/als/ALSSuite.scala rename to ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala index 70f350857fa..957ab51feba 100644 --- a/ml/src/test/scala/spark/ml/als/ALSSuite.scala +++ b/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala @@ -1,4 +1,4 @@ -package spark.ml.als +package spark.ml.recommendation import scala.util.Random From 6586c5e28b4ccbb569ca8cf149bd181d0d8e1e33 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 16:52:51 -0400 Subject: [PATCH 0267/2521] 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 f336c2ea1ea..8e1e18c06c5 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() From 39684eafe3dbfdc0563c95c05e69350fe39c71f9 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 18:08:52 -0400 Subject: [PATCH 0268/2521] Formatting --- ml/src/main/scala/spark/ml/recommendation/ALS.scala | 2 +- .../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 6717315fa7b..abf54216a58 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 7bd33a87ee2..cfdf2ba5231 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 = { From 52f491125ecc26821842fe88786b4b0e7c01499b Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 18:09:19 -0400 Subject: [PATCH 0269/2521] Implementation of k-means and k-means|| --- .../scala/spark/ml/clustering/KMeans.scala | 230 ++++++++++++++++-- .../spark/ml/clustering/KMeansModel.scala | 13 +- .../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 fd15da9b43e..d1786b0812b 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 f2f26c47b87..84bfd0f99a3 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 00000000000..03129ef552d --- /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 48292649c39..6efa7c81ad2 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 } From 496c7548bb3af6581f9bdb53cfa5be600d5849fb Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sat, 22 Jun 2013 13:51:30 -0700 Subject: [PATCH 0270/2521] Change test to use fewer iterations --- .../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 53d97899797..6216523a9e4 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) From cffe3340c5346ab7c21d82cd9c456e9ad17e2e32 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 22 Jun 2013 23:17:15 -0400 Subject: [PATCH 0271/2521] Fix logistic regression test failure and test suite cleanup --- .../spark/ml/recommendation/ALSSuite.scala | 16 +++++------ .../regression/LogisticRegressionSuite.scala | 28 ++++++++++--------- .../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 957ab51feba..c450d5315ce 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 6216523a9e4..55f2c5c18e7 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 795cda13794..aed5cbec246 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") } } From 652ea0f1d86e77503942a7846f89236b15da2ce9 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 11:15:13 -0700 Subject: [PATCH 0272/2521] 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 8e1e18c06c5..106fb2960f5 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 d8db69b1c91..919bbb2ed8d 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") + } + } } From e7d49388e3df2cd3abb8fedc8a097831d4f33e3e Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 23 Jun 2013 00:01:10 -0400 Subject: [PATCH 0273/2521] Added unit test for K-means, and fixed some bugs --- .../scala/spark/ml/clustering/KMeans.scala | 2 +- .../spark/ml/clustering/KMeansModel.scala | 11 +- .../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 d1786b0812b..8c4bec2a266 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 84bfd0f99a3..8244ccc55be 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 00000000000..f3bd1d599f5 --- /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) + } +} From 9441d3ef09273436044a8a8b37706eab8df58995 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 23 Jun 2013 00:50:14 -0400 Subject: [PATCH 0274/2521] 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 ++++++----- .../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 8c4bec2a266..d35f942c01c 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 abf54216a58..8d5c16847a6 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 c450d5315ce..f3f56c43574 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) } /** From 09f187a400623020b520a4cc5c8fc5814746dab1 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 30 Jun 2013 14:40:24 -0700 Subject: [PATCH 0275/2521] Add top-level methods for regression methods. Also add multiple versions of them to make it easier to call them from java. --- .../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 1508f6934a0..435669f72a7 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 16d146ac273..dae224144e7 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() } } From 0e33c88cbd85d3c9aa73b4c66a03372169abf3a1 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 30 Jun 2013 14:43:08 -0700 Subject: [PATCH 0276/2521] Rename package gradient to optimization --- .../scala/spark/ml/{gradient => optimization}/Gradient.scala | 2 +- .../spark/ml/{gradient => optimization}/GradientDescent.scala | 2 +- .../scala/spark/ml/{gradient => optimization}/Updater.scala | 2 +- ml/src/main/scala/spark/ml/regression/LogisticRegression.scala | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) rename ml/src/main/scala/spark/ml/{gradient => optimization}/Gradient.scala (97%) rename ml/src/main/scala/spark/ml/{gradient => optimization}/GradientDescent.scala (98%) rename ml/src/main/scala/spark/ml/{gradient => optimization}/Updater.scala (97%) diff --git a/ml/src/main/scala/spark/ml/gradient/Gradient.scala b/ml/src/main/scala/spark/ml/optimization/Gradient.scala similarity index 97% rename from ml/src/main/scala/spark/ml/gradient/Gradient.scala rename to ml/src/main/scala/spark/ml/optimization/Gradient.scala index 50795c01a25..6d062ebddfe 100644 --- a/ml/src/main/scala/spark/ml/gradient/Gradient.scala +++ b/ml/src/main/scala/spark/ml/optimization/Gradient.scala @@ -1,4 +1,4 @@ -package spark.ml.gradient +package spark.ml.optimization import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/gradient/GradientDescent.scala b/ml/src/main/scala/spark/ml/optimization/GradientDescent.scala similarity index 98% rename from ml/src/main/scala/spark/ml/gradient/GradientDescent.scala rename to ml/src/main/scala/spark/ml/optimization/GradientDescent.scala index 8cd2a69a3a1..d959ebf71c4 100644 --- a/ml/src/main/scala/spark/ml/gradient/GradientDescent.scala +++ b/ml/src/main/scala/spark/ml/optimization/GradientDescent.scala @@ -1,4 +1,4 @@ -package spark.ml.gradient +package spark.ml.optimization import spark.{Logging, RDD, SparkContext} import spark.SparkContext._ diff --git a/ml/src/main/scala/spark/ml/gradient/Updater.scala b/ml/src/main/scala/spark/ml/optimization/Updater.scala similarity index 97% rename from ml/src/main/scala/spark/ml/gradient/Updater.scala rename to ml/src/main/scala/spark/ml/optimization/Updater.scala index 27f03897b00..dfc7bf20259 100644 --- a/ml/src/main/scala/spark/ml/gradient/Updater.scala +++ b/ml/src/main/scala/spark/ml/optimization/Updater.scala @@ -1,4 +1,4 @@ -package spark.ml.gradient +package spark.ml.optimization import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala b/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala index 435669f72a7..3c471ab652b 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 From 399bd65ef5f780c2796f6facf9fac8fd9ec2c2f4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 11:27:06 -0700 Subject: [PATCH 0277/2521] 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 919bbb2ed8d..e41ae385c0d 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) From 43b24635ee45a845f2432bc13c11fcf2eb02f2f3 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 11:38:53 -0700 Subject: [PATCH 0278/2521] Renamed ML package to MLlib and added it to classpath --- bin/compute-classpath.cmd | 2 ++ bin/compute-classpath.sh | 2 ++ {ml => mllib}/data/als/test.data | 0 {ml => mllib}/data/lr-data/random.data | 0 {ml => mllib}/data/ridge-data/lpsa.data | 0 .../src/main/scala/spark/ml/clustering/KMeans.scala | 9 +++++---- .../src/main/scala/spark/ml/clustering/KMeansModel.scala | 4 ++-- .../src/main/scala/spark/ml/clustering/LocalKMeans.scala | 4 ++-- .../src/main/scala/spark/ml/optimization/Gradient.scala | 2 +- .../scala/spark/ml/optimization/GradientDescent.scala | 2 +- .../src/main/scala/spark/ml/optimization/Updater.scala | 2 +- .../src/main/scala/spark/ml/recommendation/ALS.scala | 2 +- .../ml/recommendation/MatrixFactorizationModel.scala | 2 +- .../scala/spark/ml/regression/LogisticRegression.scala | 6 +++--- .../ml/regression/LogisticRegressionGenerator.scala | 4 ++-- .../src/main/scala/spark/ml/regression/Regression.scala | 2 +- .../main/scala/spark/ml/regression/RidgeRegression.scala | 4 ++-- .../spark/ml/regression/RidgeRegressionGenerator.scala | 4 ++-- {ml => mllib}/src/main/scala/spark/ml/util/MLUtils.scala | 2 +- {ml => mllib}/src/test/resources/log4j.properties | 0 .../src/test/scala/spark/ml/clustering/KMeansSuite.scala | 2 +- .../test/scala/spark/ml/recommendation/ALSSuite.scala | 2 +- .../spark/ml/regression/LogisticRegressionSuite.scala | 2 +- .../scala/spark/ml/regression/RidgeRegressionSuite.scala | 2 +- project/SparkBuild.scala | 8 ++++---- 25 files changed, 37 insertions(+), 32 deletions(-) rename {ml => mllib}/data/als/test.data (100%) rename {ml => mllib}/data/lr-data/random.data (100%) rename {ml => mllib}/data/ridge-data/lpsa.data (100%) rename {ml => mllib}/src/main/scala/spark/ml/clustering/KMeans.scala (97%) rename {ml => mllib}/src/main/scala/spark/ml/clustering/KMeansModel.scala (92%) rename {ml => mllib}/src/main/scala/spark/ml/clustering/LocalKMeans.scala (97%) rename {ml => mllib}/src/main/scala/spark/ml/optimization/Gradient.scala (96%) rename {ml => mllib}/src/main/scala/spark/ml/optimization/GradientDescent.scala (98%) rename {ml => mllib}/src/main/scala/spark/ml/optimization/Updater.scala (96%) rename {ml => mllib}/src/main/scala/spark/ml/recommendation/ALS.scala (99%) rename {ml => mllib}/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala (95%) rename {ml => mllib}/src/main/scala/spark/ml/regression/LogisticRegression.scala (98%) rename {ml => mllib}/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala (94%) rename {ml => mllib}/src/main/scala/spark/ml/regression/Regression.scala (94%) rename {ml => mllib}/src/main/scala/spark/ml/regression/RidgeRegression.scala (98%) rename {ml => mllib}/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala (96%) rename {ml => mllib}/src/main/scala/spark/ml/util/MLUtils.scala (99%) rename {ml => mllib}/src/test/resources/log4j.properties (100%) rename {ml => mllib}/src/test/scala/spark/ml/clustering/KMeansSuite.scala (99%) rename {ml => mllib}/src/test/scala/spark/ml/recommendation/ALSSuite.scala (98%) rename {ml => mllib}/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala (98%) rename {ml => mllib}/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala (97%) diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 6e7efbd3349..44826f339c1 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 3a788802904..75c58d11813 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/mllib/data/als/test.data similarity index 100% rename from ml/data/als/test.data rename to mllib/data/als/test.data diff --git a/ml/data/lr-data/random.data b/mllib/data/lr-data/random.data similarity index 100% rename from ml/data/lr-data/random.data rename to mllib/data/lr-data/random.data diff --git a/ml/data/ridge-data/lpsa.data b/mllib/data/ridge-data/lpsa.data similarity index 100% rename from ml/data/ridge-data/lpsa.data rename to mllib/data/ridge-data/lpsa.data diff --git a/ml/src/main/scala/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/spark/ml/clustering/KMeans.scala similarity index 97% rename from ml/src/main/scala/spark/ml/clustering/KMeans.scala rename to mllib/src/main/scala/spark/ml/clustering/KMeans.scala index d35f942c01c..6d78f926c2c 100644 --- a/ml/src/main/scala/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/spark/ml/clustering/KMeans.scala @@ -1,4 +1,4 @@ -package spark.ml.clustering +package spark.mllib.clustering import scala.collection.mutable.ArrayBuffer import scala.util.Random @@ -6,7 +6,7 @@ import scala.util.Random import spark.{SparkContext, RDD} import spark.SparkContext._ import spark.Logging -import spark.ml.util.MLUtils +import spark.mllib.util.MLUtils import org.jblas.DoubleMatrix @@ -270,7 +270,8 @@ object KMeans { /** * 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) = + private[mllib] def findClosest(centers: Array[Array[Double]], point: Array[Double]) + : (Int, Double) = { var bestDistance = Double.PositiveInfinity var bestIndex = 0 @@ -287,7 +288,7 @@ object KMeans { /** * 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 = { + 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)) diff --git a/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala b/mllib/src/main/scala/spark/ml/clustering/KMeansModel.scala similarity index 92% rename from ml/src/main/scala/spark/ml/clustering/KMeansModel.scala rename to mllib/src/main/scala/spark/ml/clustering/KMeansModel.scala index 8244ccc55be..4fd0646160e 100644 --- a/ml/src/main/scala/spark/ml/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/spark/ml/clustering/KMeansModel.scala @@ -1,8 +1,8 @@ -package spark.ml.clustering +package spark.mllib.clustering import spark.RDD import spark.SparkContext._ -import spark.ml.util.MLUtils +import spark.mllib.util.MLUtils /** diff --git a/ml/src/main/scala/spark/ml/clustering/LocalKMeans.scala b/mllib/src/main/scala/spark/ml/clustering/LocalKMeans.scala similarity index 97% rename from ml/src/main/scala/spark/ml/clustering/LocalKMeans.scala rename to mllib/src/main/scala/spark/ml/clustering/LocalKMeans.scala index 03129ef552d..e12b3be2512 100644 --- a/ml/src/main/scala/spark/ml/clustering/LocalKMeans.scala +++ b/mllib/src/main/scala/spark/ml/clustering/LocalKMeans.scala @@ -1,4 +1,4 @@ -package spark.ml.clustering +package spark.mllib.clustering import scala.util.Random @@ -8,7 +8,7 @@ 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 { +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. diff --git a/ml/src/main/scala/spark/ml/optimization/Gradient.scala b/mllib/src/main/scala/spark/ml/optimization/Gradient.scala similarity index 96% rename from ml/src/main/scala/spark/ml/optimization/Gradient.scala rename to mllib/src/main/scala/spark/ml/optimization/Gradient.scala index 6d062ebddfe..90b0999a5ec 100644 --- a/ml/src/main/scala/spark/ml/optimization/Gradient.scala +++ b/mllib/src/main/scala/spark/ml/optimization/Gradient.scala @@ -1,4 +1,4 @@ -package spark.ml.optimization +package spark.mllib.optimization import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/ml/optimization/GradientDescent.scala similarity index 98% rename from ml/src/main/scala/spark/ml/optimization/GradientDescent.scala rename to mllib/src/main/scala/spark/ml/optimization/GradientDescent.scala index d959ebf71c4..eff853f379b 100644 --- a/ml/src/main/scala/spark/ml/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/ml/optimization/GradientDescent.scala @@ -1,4 +1,4 @@ -package spark.ml.optimization +package spark.mllib.optimization import spark.{Logging, RDD, SparkContext} import spark.SparkContext._ diff --git a/ml/src/main/scala/spark/ml/optimization/Updater.scala b/mllib/src/main/scala/spark/ml/optimization/Updater.scala similarity index 96% rename from ml/src/main/scala/spark/ml/optimization/Updater.scala rename to mllib/src/main/scala/spark/ml/optimization/Updater.scala index dfc7bf20259..ea80bfcbfd4 100644 --- a/ml/src/main/scala/spark/ml/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/ml/optimization/Updater.scala @@ -1,4 +1,4 @@ -package spark.ml.optimization +package spark.mllib.optimization import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/spark/ml/recommendation/ALS.scala similarity index 99% rename from ml/src/main/scala/spark/ml/recommendation/ALS.scala rename to mllib/src/main/scala/spark/ml/recommendation/ALS.scala index 8d5c16847a6..0c6fa6f741d 100644 --- a/ml/src/main/scala/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/ml/recommendation/ALS.scala @@ -1,4 +1,4 @@ -package spark.ml.recommendation +package spark.mllib.recommendation import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.util.Random diff --git a/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala similarity index 95% rename from ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala rename to mllib/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala index cfdf2ba5231..fb812a6dbeb 100644 --- a/ml/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala @@ -1,4 +1,4 @@ -package spark.ml.recommendation +package spark.mllib.recommendation import spark.RDD import spark.SparkContext._ diff --git a/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/ml/regression/LogisticRegression.scala similarity index 98% rename from ml/src/main/scala/spark/ml/regression/LogisticRegression.scala rename to mllib/src/main/scala/spark/ml/regression/LogisticRegression.scala index 3c471ab652b..448ab9dce90 100644 --- a/ml/src/main/scala/spark/ml/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/ml/regression/LogisticRegression.scala @@ -1,8 +1,8 @@ -package spark.ml.regression +package spark.mllib.regression import spark.{Logging, RDD, SparkContext} -import spark.ml.optimization._ -import spark.ml.util.MLUtils +import spark.mllib.optimization._ +import spark.mllib.util.MLUtils import org.jblas.DoubleMatrix diff --git a/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala similarity index 94% rename from ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala rename to mllib/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala index 6d37aad047c..9f6abab70b9 100644 --- a/ml/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala @@ -1,11 +1,11 @@ -package spark.ml.regression +package spark.mllib.regression import scala.util.Random import org.jblas.DoubleMatrix import spark.{RDD, SparkContext} -import spark.ml.util.MLUtils +import spark.mllib.util.MLUtils object LogisticRegressionGenerator { diff --git a/ml/src/main/scala/spark/ml/regression/Regression.scala b/mllib/src/main/scala/spark/ml/regression/Regression.scala similarity index 94% rename from ml/src/main/scala/spark/ml/regression/Regression.scala rename to mllib/src/main/scala/spark/ml/regression/Regression.scala index 4a20f513b7b..f79974c1915 100644 --- a/ml/src/main/scala/spark/ml/regression/Regression.scala +++ b/mllib/src/main/scala/spark/ml/regression/Regression.scala @@ -1,4 +1,4 @@ -package spark.ml.regression +package spark.mllib.regression import spark.RDD diff --git a/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala similarity index 98% rename from ml/src/main/scala/spark/ml/regression/RidgeRegression.scala rename to mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala index dae224144e7..2d07c771410 100644 --- a/ml/src/main/scala/spark/ml/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala @@ -1,8 +1,8 @@ -package spark.ml.regression +package spark.mllib.regression import spark.{Logging, RDD, SparkContext} import spark.SparkContext._ -import spark.ml.util.MLUtils +import spark.mllib.util.MLUtils import org.jblas.DoubleMatrix import org.jblas.Solve diff --git a/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala similarity index 96% rename from ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala rename to mllib/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala index 75854fe1de8..c9ac4a8b07c 100644 --- a/ml/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala @@ -1,11 +1,11 @@ -package spark.ml.regression +package spark.mllib.regression import scala.util.Random import org.jblas.DoubleMatrix import spark.{RDD, SparkContext} -import spark.ml.util.MLUtils +import spark.mllib.util.MLUtils object RidgeRegressionGenerator { diff --git a/ml/src/main/scala/spark/ml/util/MLUtils.scala b/mllib/src/main/scala/spark/ml/util/MLUtils.scala similarity index 99% rename from ml/src/main/scala/spark/ml/util/MLUtils.scala rename to mllib/src/main/scala/spark/ml/util/MLUtils.scala index 6efa7c81ad2..0a4a037c713 100644 --- a/ml/src/main/scala/spark/ml/util/MLUtils.scala +++ b/mllib/src/main/scala/spark/ml/util/MLUtils.scala @@ -1,4 +1,4 @@ -package spark.ml.util +package spark.mllib.util import spark.{RDD, SparkContext} import spark.SparkContext._ diff --git a/ml/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties similarity index 100% rename from ml/src/test/resources/log4j.properties rename to mllib/src/test/resources/log4j.properties diff --git a/ml/src/test/scala/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/spark/ml/clustering/KMeansSuite.scala similarity index 99% rename from ml/src/test/scala/spark/ml/clustering/KMeansSuite.scala rename to mllib/src/test/scala/spark/ml/clustering/KMeansSuite.scala index f3bd1d599f5..ae7cf57c42c 100644 --- a/ml/src/test/scala/spark/ml/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/spark/ml/clustering/KMeansSuite.scala @@ -1,4 +1,4 @@ -package spark.ml.clustering +package spark.mllib.clustering import scala.util.Random diff --git a/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/spark/ml/recommendation/ALSSuite.scala similarity index 98% rename from ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala rename to mllib/src/test/scala/spark/ml/recommendation/ALSSuite.scala index f3f56c43574..2ada9ae76b5 100644 --- a/ml/src/test/scala/spark/ml/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/spark/ml/recommendation/ALSSuite.scala @@ -1,4 +1,4 @@ -package spark.ml.recommendation +package spark.mllib.recommendation import scala.util.Random diff --git a/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala similarity index 98% rename from ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala rename to mllib/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala index 55f2c5c18e7..04d3400cb42 100644 --- a/ml/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala @@ -1,4 +1,4 @@ -package spark.ml.regression +package spark.mllib.regression import scala.util.Random diff --git a/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala similarity index 97% rename from ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala rename to mllib/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala index aed5cbec246..df41dbbdff8 100644 --- a/ml/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala @@ -1,4 +1,4 @@ -package spark.ml.regression +package spark.mllib.regression import scala.util.Random diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5dbb5d4a65e..c487f34d4a0 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" ) From de67deeaabc12a62dadee5ec302fe58bee0b3498 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 17:16:49 -0700 Subject: [PATCH 0279/2521] Addressed style comments from Ryan LeCompte --- .../main/scala/spark/ml/clustering/KMeans.scala | 14 ++++++-------- .../main/scala/spark/ml/recommendation/ALS.scala | 4 +++- .../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 6d78f926c2c..b0e141ff329 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 0c6fa6f741d..6c9fb2359cb 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 2d07c771410..a6ececbeb67 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. * From 653043beb6681c57a02a3e8dde837d7dbc1e44bf Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 17:18:55 -0700 Subject: [PATCH 0280/2521] Renamed files to match package --- mllib/src/main/scala/spark/{ml => mllib}/clustering/KMeans.scala | 0 .../main/scala/spark/{ml => mllib}/clustering/KMeansModel.scala | 0 .../main/scala/spark/{ml => mllib}/clustering/LocalKMeans.scala | 0 .../main/scala/spark/{ml => mllib}/optimization/Gradient.scala | 0 .../scala/spark/{ml => mllib}/optimization/GradientDescent.scala | 0 .../src/main/scala/spark/{ml => mllib}/optimization/Updater.scala | 0 mllib/src/main/scala/spark/{ml => mllib}/recommendation/ALS.scala | 0 .../{ml => mllib}/recommendation/MatrixFactorizationModel.scala | 0 .../scala/spark/{ml => mllib}/regression/LogisticRegression.scala | 0 .../{ml => mllib}/regression/LogisticRegressionGenerator.scala | 0 .../main/scala/spark/{ml => mllib}/regression/Regression.scala | 0 .../scala/spark/{ml => mllib}/regression/RidgeRegression.scala | 0 .../spark/{ml => mllib}/regression/RidgeRegressionGenerator.scala | 0 mllib/src/main/scala/spark/{ml => mllib}/util/MLUtils.scala | 0 .../test/scala/spark/{ml => mllib}/clustering/KMeansSuite.scala | 0 .../test/scala/spark/{ml => mllib}/recommendation/ALSSuite.scala | 0 .../spark/{ml => mllib}/regression/LogisticRegressionSuite.scala | 0 .../spark/{ml => mllib}/regression/RidgeRegressionSuite.scala | 0 18 files changed, 0 insertions(+), 0 deletions(-) rename mllib/src/main/scala/spark/{ml => mllib}/clustering/KMeans.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/clustering/KMeansModel.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/clustering/LocalKMeans.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/optimization/Gradient.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/optimization/GradientDescent.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/optimization/Updater.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/recommendation/ALS.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/recommendation/MatrixFactorizationModel.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/regression/LogisticRegression.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/regression/LogisticRegressionGenerator.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/regression/Regression.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/regression/RidgeRegression.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/regression/RidgeRegressionGenerator.scala (100%) rename mllib/src/main/scala/spark/{ml => mllib}/util/MLUtils.scala (100%) rename mllib/src/test/scala/spark/{ml => mllib}/clustering/KMeansSuite.scala (100%) rename mllib/src/test/scala/spark/{ml => mllib}/recommendation/ALSSuite.scala (100%) rename mllib/src/test/scala/spark/{ml => mllib}/regression/LogisticRegressionSuite.scala (100%) rename mllib/src/test/scala/spark/{ml => mllib}/regression/RidgeRegressionSuite.scala (100%) diff --git a/mllib/src/main/scala/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/clustering/KMeans.scala rename to mllib/src/main/scala/spark/mllib/clustering/KMeans.scala diff --git a/mllib/src/main/scala/spark/ml/clustering/KMeansModel.scala b/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/clustering/KMeansModel.scala rename to mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala diff --git a/mllib/src/main/scala/spark/ml/clustering/LocalKMeans.scala b/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/clustering/LocalKMeans.scala rename to mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala diff --git a/mllib/src/main/scala/spark/ml/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/optimization/Gradient.scala rename to mllib/src/main/scala/spark/mllib/optimization/Gradient.scala diff --git a/mllib/src/main/scala/spark/ml/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/optimization/GradientDescent.scala rename to mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala diff --git a/mllib/src/main/scala/spark/ml/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/optimization/Updater.scala rename to mllib/src/main/scala/spark/mllib/optimization/Updater.scala diff --git a/mllib/src/main/scala/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/recommendation/ALS.scala rename to mllib/src/main/scala/spark/mllib/recommendation/ALS.scala diff --git a/mllib/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/recommendation/MatrixFactorizationModel.scala rename to mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala diff --git a/mllib/src/main/scala/spark/ml/regression/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/regression/LogisticRegression.scala rename to mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala diff --git a/mllib/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/regression/LogisticRegressionGenerator.scala rename to mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala diff --git a/mllib/src/main/scala/spark/ml/regression/Regression.scala b/mllib/src/main/scala/spark/mllib/regression/Regression.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/regression/Regression.scala rename to mllib/src/main/scala/spark/mllib/regression/Regression.scala diff --git a/mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/regression/RidgeRegression.scala rename to mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala diff --git a/mllib/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/regression/RidgeRegressionGenerator.scala rename to mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala diff --git a/mllib/src/main/scala/spark/ml/util/MLUtils.scala b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala similarity index 100% rename from mllib/src/main/scala/spark/ml/util/MLUtils.scala rename to mllib/src/main/scala/spark/mllib/util/MLUtils.scala diff --git a/mllib/src/test/scala/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala similarity index 100% rename from mllib/src/test/scala/spark/ml/clustering/KMeansSuite.scala rename to mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala diff --git a/mllib/src/test/scala/spark/ml/recommendation/ALSSuite.scala b/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala similarity index 100% rename from mllib/src/test/scala/spark/ml/recommendation/ALSSuite.scala rename to mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala diff --git a/mllib/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala similarity index 100% rename from mllib/src/test/scala/spark/ml/regression/LogisticRegressionSuite.scala rename to mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala diff --git a/mllib/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala similarity index 100% rename from mllib/src/test/scala/spark/ml/regression/RidgeRegressionSuite.scala rename to mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala From 84b7fc54e6777167f96742cd61326581f342fb03 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 5 Jul 2013 17:21:01 -0700 Subject: [PATCH 0281/2521] Enforcing correct sort order for formatted strings --- .../main/scala/spark/deploy/master/ui/IndexPage.scala | 10 +++++++--- .../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 2ed566a5bc3..c6de2bafa33 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 c65d5b4fafe..e466129c1ae 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 c6f87fc6520..49e84880cf6 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 65952f711a0..0cb1e47ea55 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)} + } From 8bbe907556041443a411b69c95d7a9cd3eb69dcc Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 5 Jul 2013 17:25:23 -0700 Subject: [PATCH 0282/2521] Replaced string constants in 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 ae7cf57c42c..cb096f39a97 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))) } From 757e56dfc7bd900d5b3f3f145eabe8198bfbe7cc Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Fri, 5 Jul 2013 19:54:28 -0700 Subject: [PATCH 0283/2521] make binSearch a tail-recursive method --- .../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 a6ececbeb67..8343f281390 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 From 280418ac452b029b15a83d8e2fe05a96417294d1 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 5 Jul 2013 21:38:21 -0700 Subject: [PATCH 0284/2521] Reduced the number of Iterator to ArrayBuffer copies in NetworkReceiver. --- .../dstream/NetworkInputDStream.scala | 18 ++++++------------ .../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 26805e96210..122a529bb76 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 b3201d0b28d..036c95a860c 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() = { From 37abe84212c6dad6fb87a3b47666d6a3c14c1f66 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 5 Jul 2013 22:54:06 -0700 Subject: [PATCH 0285/2521] Tracking some task metrics even during failures. --- core/src/main/scala/spark/TaskEndReason.scala | 4 +++- core/src/main/scala/spark/executor/Executor.scala | 10 ++++++++-- .../main/scala/spark/scheduler/DAGScheduler.scala | 2 +- .../scheduler/cluster/ClusterTaskSetManager.scala | 2 +- .../spark/scheduler/local/LocalScheduler.scala | 14 ++++++++++---- .../scheduler/local/LocalTaskSetManager.scala | 2 +- .../main/scala/spark/ui/UIWorkloadGenerator.scala | 12 +++++++++++- .../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 8140cba0841..bb75ec208cb 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 8bebfafce44..f01be68d141 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 d9ddc41aa2f..1945a4a5147 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 6965cde5da3..fe6420a522c 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 9d375e1db84..b000e328e67 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 499116f6534..f12fec41d50 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 24cfe36aaac..8bbc6ce88ea 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 aafa4140559..36b1cd00edd 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 } From 44a2440039a35784b7dfed2e36b96096c3424d33 Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Sun, 7 Jul 2013 01:33:09 +0800 Subject: [PATCH 0286/2521] 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 51b10ed0450..cbd375e5c14 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 } From 32b9d21a97d1c93f174551000d06cc429f317827 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 6 Jul 2013 16:36:13 -0700 Subject: [PATCH 0287/2521] 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. --- .../main/scala/spark/ui/jobs/JobProgressUI.scala | 14 +++++++------- .../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 36b1cd00edd..84730cc091b 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 49e84880cf6..51b82b6a8c2 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 => From fd6665122b4bd3aa2d56ec2f6c6028e1b8a18b7f Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 6 Jul 2013 16:45:15 -0700 Subject: [PATCH 0288/2521] 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 3184fd376a7..017c2424649 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 From f78f8d0b416ef4d88883d8f32382661f4c2ac52d Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Sat, 6 Jul 2013 16:46:53 -0700 Subject: [PATCH 0289/2521] fix formatting and use Vector instead of List to maintain order --- .../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 8343f281390..36cda721ddb 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. */ From a948f0672541bd68be020f07134659aee2f38403 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 02:30:45 -0700 Subject: [PATCH 0290/2521] 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 ++ .../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 ec59b4f48fd..16b00d15aab 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 55bb61b0ccf..cb85419ae43 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 f1a35bced3a..9c24ca430da 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 76d5258b02b..bd184222ed3 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() From 7d6d9e6ab226579518f1c7fbe108c4e66acc6ed0 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 02:45:55 -0700 Subject: [PATCH 0291/2521] 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 5e84b3a66a1..31c3dd75fb2 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() } From 3350ad0d7fc3ecece78f87d7aa6a727e48b21c8c Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 03:32:26 -0700 Subject: [PATCH 0292/2521] Catch RejectedExecution exception in Checkpoint handler. --- .../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 66e67cbfa1d..450e48d66e2 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() { From d362d0f4117268bdef265041ff291700ddd49b43 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 03:33:32 -0700 Subject: [PATCH 0293/2521] 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 1c9ca508112..d263bb00e9c 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() } From 4af0d63cb14db902cbd1dbdeeb68f1fcec4b2e97 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 7 Jul 2013 10:42:43 -0700 Subject: [PATCH 0294/2521] 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 16b00d15aab..ec59b4f48fd 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 cb85419ae43..939f26b6f48 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()) } From be123aa6ef90480ad61663eed6e8ea479b047fad Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Sun, 7 Jul 2013 15:35:06 -0700 Subject: [PATCH 0295/2521] update to use ListBuffer, faster than Vector for append operations --- .../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 36cda721ddb..f66025bc0bb 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 From 8c1d1c98e0cea8d2b20ac10b84aa76d8e22a1661 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Mon, 8 Jul 2013 12:25:46 -0700 Subject: [PATCH 0296/2521] Explicitly set class loader for MesosSchedulerDriver callbacks. --- .../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 ca7fab4cc5f..e73b780fcbc 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 { From bf4c9a5e0fca2dfc960120a7f3c5fab0b87e3850 Mon Sep 17 00:00:00 2001 From: Ameet Talwalkar Date: Mon, 8 Jul 2013 14:37:42 -0700 Subject: [PATCH 0297/2521] renamed with labeled prefix --- .../scala/spark/mllib/regression/LogisticRegression.scala | 2 +- .../spark/mllib/regression/LogisticRegressionGenerator.scala | 2 +- .../main/scala/spark/mllib/regression/RidgeRegression.scala | 2 +- .../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 448ab9dce90..e4db7bb9b73 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 9f6abab70b9..6e7c023bac7 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 f66025bc0bb..5f813df402a 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 c9ac4a8b07c..b83f505d8e4 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 0a4a037c713..08a031dded1 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) } From afdaf430bd8d7da3ee3323cf03b00d4214159626 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Mon, 8 Jul 2013 15:38:29 -0700 Subject: [PATCH 0298/2521] 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 5edafb3706f..39dba46d7a1 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 78ec58729b3..7d9769e8e4c 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 4dc9a19d51f..07725f9484f 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 From 0b39d66f3f815f2d406d83a41db0f46d097baaa7 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Mon, 8 Jul 2013 16:07:09 -0700 Subject: [PATCH 0299/2521] 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 39dba46d7a1..dbb2da9a9cd 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 7d9769e8e4c..1976765c3dd 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 017c2424649..63dd80b5fea 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 07725f9484f..2fb5bbdeb5c 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -61,7 +61,6 @@ org.scala-lang scala-library - 2.9.3 com.typesafe.akka From c1d44be80580f0fad6bb1805bbcf74a34f536d8c Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Tue, 9 Jul 2013 15:18:28 +0800 Subject: [PATCH 0300/2521] 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 f19648ec686..9bc692d480d 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) + } } } ) } From e47253e0cca1359b49f113dbf258c2c204e3bfc1 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Tue, 9 Jul 2013 01:13:12 -0700 Subject: [PATCH 0301/2521] Reset ClassLoader in MesosSchedulerBackend, too. (per review comments). Also set ClassLoader for all mesos callbacks, not just statusUpdate, registered. --- .../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 e73b780fcbc..e83368b98dd 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) { From aaa7b081df760a29ce5cdcd51d6b71422cba68d5 Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Tue, 9 Jul 2013 20:03:01 +0800 Subject: [PATCH 0302/2521] 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 9bc692d480d..776db201f95 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) } From 13fc6f248c7231501d8c53ad13641fa996e06be4 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 9 Jul 2013 13:42:51 -0700 Subject: [PATCH 0303/2521] Clean commit of log paging --- .../deploy/master/ui/ApplicationPage.scala | 8 ++- .../spark/deploy/worker/ui/IndexPage.scala | 8 ++- .../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 33a16b5d849..ea884215328 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 e466129c1ae..7cf98b473e3 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 16564d56197..5b0c785b008 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()) } From b6072b58bf795093df0e5f8424413fcd0cce6323 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 9 Jul 2013 17:25:10 -0700 Subject: [PATCH 0304/2521] Fixes style, makes "std__-page" consistent, reads only parts of files --- core/src/main/scala/spark/Utils.scala | 14 +++++ .../deploy/master/ui/ApplicationPage.scala | 8 +-- .../spark/deploy/worker/ui/IndexPage.scala | 4 +- .../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 64547bbdcd2..dfc30469cf8 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 ea884215328..cc32728c1c6 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 7cf98b473e3..d532aa9e956 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 5b0c785b008..b018f80c859 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) } From ce18b50d5ff37dc5c558d0602321a61887dd8b48 Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Wed, 10 Jul 2013 19:11:43 +0800 Subject: [PATCH 0305/2521] set SUCCEEDED for all master in shutdown hook --- .../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 776db201f95..68bb36d3166 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) } } } ) From 620a6974c6603f1c0e5a7cea8f0387a5d18f2e5e Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 10:20:53 -0700 Subject: [PATCH 0306/2521] Allows for larger files, refactors lastNBytes, removes old Log column, fixes imports, uses map --- core/src/main/scala/spark/Utils.scala | 21 +++++++------------ .../deploy/master/ui/ApplicationPage.scala | 12 +++-------- .../spark/deploy/worker/ui/IndexPage.scala | 12 +++-------- .../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 dfc30469cf8..849edc13ee3 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 cc32728c1c6..49ced0d3209 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 d532aa9e956..af9943853f4 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 b018f80c859..602881d5f16 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() From 24705d0f46ce536bf829660f4506dcffd9ff799a Mon Sep 17 00:00:00 2001 From: seanm Date: Wed, 10 Jul 2013 10:33:11 -0700 Subject: [PATCH 0307/2521] 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 106fb2960f5..af52040fa68 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 e41ae385c0d..fe17d1d5e71 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) { From ee4ce2fc51112387f28d7b422969ca2e9736e95f Mon Sep 17 00:00:00 2001 From: seanm Date: Wed, 10 Jul 2013 10:46:04 -0700 Subject: [PATCH 0308/2521] adding takeOrdered to java API --- .../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 b555f2030a4..94b95af714d 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) + } } From cfb6447ac4903a870dd268836dc0d8952491d591 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 11:47:57 -0700 Subject: [PATCH 0309/2521] Fixed for nonexistent bytes, added unit tests, changed stdout-page to stdout --- core/src/main/scala/spark/Utils.scala | 6 ++-- .../deploy/master/ui/ApplicationPage.scala | 2 +- .../spark/deploy/worker/ui/IndexPage.scala | 2 +- .../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 849edc13ee3..512ac92d890 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 49ced0d3209..dae99957795 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 af9943853f4..a9c6c6d519e 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 602881d5f16..24356b0f631 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 e4bb3abc339..b7a822c4bc1 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) + } } From 04263e4d4609355d8e0779dea472fbb1aff5ef4f Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 13:15:42 -0700 Subject: [PATCH 0310/2521] 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 512ac92d890..c5627c8419b 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 b7a822c4bc1..ab174732e35 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) From 0d4580360ba4bbcd2f73877743d746c071a92e34 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 13:24:26 -0700 Subject: [PATCH 0311/2521] 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 c5627c8419b..1c5af2700a6 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 24356b0f631..1f4ba5de601 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)}> } From 5f8a20b4a8a4cb7e4b14ee989f1b4f83981ce61a Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 13:53:39 -0700 Subject: [PATCH 0312/2521] 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 4a113e16bf5..73050a1d3bc 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 ab174732e35..ef2e9c2ec41 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) - } } From dbe948d9a2198845feb7c8bfa738286db41344d9 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 14:15:41 -0700 Subject: [PATCH 0313/2521] 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 73050a1d3bc..942702c826d 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 ef2e9c2ec41..cd7d1059280 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") { From f5f3b272f874a3ac22f4e0b2caab9053657a9124 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 14:52:29 -0700 Subject: [PATCH 0314/2521] 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 1c5af2700a6..1da9c9574e8 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 942702c826d..d83a0307fac 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 cd7d1059280..fc0c1607202 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") { From 24196c91f0bd560c08399eed3cb1903248ca497a Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 15:27:52 -0700 Subject: [PATCH 0315/2521] Changed buffer to 10,000 bytes, created scrollbar for fixed-height log --- .../main/scala/spark/deploy/master/ui/ApplicationPage.scala | 4 ++-- core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala | 4 ++-- .../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 dae99957795..27549e3b4a9 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 a9c6c6d519e..313ecec0849 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 1f4ba5de601..490fe15fcf7 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) From 74bd3fc6806a4185a95a19c107e1a4776582db44 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 10 Jul 2013 15:44:28 -0700 Subject: [PATCH 0316/2521] Added byte range on log pages --- .../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 490fe15fcf7..cac281eef63 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} From ebf5b8c7c5fe72fbe2d74a1bfa67cbdc9f9be0ea Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Wed, 10 Jul 2013 22:16:06 -0700 Subject: [PATCH 0317/2521] 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 ba24ab43b1a..1dd96a0a4a9 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: From 69ae7ea227e52c743652e5560ac7666187c0de6a Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 11 Jul 2013 18:30:18 +0530 Subject: [PATCH 0318/2521] Removed some unnecessary code and fixed dependencies --- bin/compute-classpath.sh | 2 +- pom.xml | 2 +- project/SparkBuild.scala | 12 ++++-------- 3 files changed, 6 insertions(+), 10 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 3a788802904..a0f35cb5eb1 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -3,7 +3,7 @@ # 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 +SCALA_VERSION=2.10 # Figure out where Spark is installed FWDIR="$(cd `dirname $0`/..; pwd)" diff --git a/pom.xml b/pom.xml index 5d624d14465..35eb9bf7a1d 100644 --- a/pom.xml +++ b/pom.xml @@ -245,7 +245,7 @@ com.github.scala-incubator.io - scala-io-file_2.9.2 + scala-io-file_2.10 0.4.1 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 987b78d3176..d35b39fcf36 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -161,7 +161,6 @@ object SparkBuild extends Build { "colt" % "colt" % "1.2.0", "org.apache.mesos" % "mesos" % "0.9.0-incubating", "org.apache.derby" % "derby" % "10.4.2.0" % "test", - "org.scala-lang" % "scala-actors" % "2.10.1", "org.scala-lang" % "jline" % "2.10.1", "org.scala-lang" % "scala-reflect" % "2.10.1" ) ++ ( @@ -169,19 +168,19 @@ 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-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) ) } else { Seq( - "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty), + "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty), "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty) ) } } else { - Seq("org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty) ) + Seq("org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty) ) }), unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / ( if (HADOOP_YARN && HADOOP_MAJOR_VERSION == "2") { @@ -222,9 +221,6 @@ object SparkBuild extends Build { def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", - resolvers ++= Seq( - "Akka Repository" at "http://repo.akka.io/releases/" - ), libraryDependencies ++= Seq( "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty), "com.github.sgroschupf" % "zkclient" % "0.1", From ca249eea50065a80cf01f62e3144a0c18cce5f18 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 11 Jul 2013 18:31:36 +0530 Subject: [PATCH 0319/2521] Removed an unnecessary test case --- .../repl/StandaloneClusterReplSuite.scala | 101 ------------------ 1 file changed, 101 deletions(-) delete mode 100644 repl/src/test/scala/spark/repl/StandaloneClusterReplSuite.scala diff --git a/repl/src/test/scala/spark/repl/StandaloneClusterReplSuite.scala b/repl/src/test/scala/spark/repl/StandaloneClusterReplSuite.scala deleted file mode 100644 index 0212d21fb85..00000000000 --- a/repl/src/test/scala/spark/repl/StandaloneClusterReplSuite.scala +++ /dev/null @@ -1,101 +0,0 @@ -package spark.repl - -import java.io.FileWriter - -import org.scalatest.FunSuite - -import com.google.common.io.Files - -class StandaloneClusterReplSuite extends FunSuite with ReplSuiteMixin { - val sparkUrl = "local-cluster[1,1,512]" - - test("simple collect") { - val output = runInterpreter(sparkUrl, """ - var x = 123 - val data = sc.parallelize(1 to 3).map(_ + x) - data.take(3) - """) - assertDoesNotContain("error:", output) - assertDoesNotContain("Exception", output) - assertContains("124", output) - assertContains("125", output) - assertContains("126", output) - } - - test("simple foreach with accumulator") { - val output = runInterpreter(sparkUrl, """ - val accum = sc.accumulator(0) - sc.parallelize(1 to 10).foreach(x => accum += x) - accum.value - """) - assertDoesNotContain("error:", output) - assertDoesNotContain("Exception", output) - assertContains("res1: Int = 55", output) - } - - test("external vars") { - val output = runInterpreter(sparkUrl, """ - var v = 7 - sc.parallelize(1 to 10).map(x => v).take(10).reduceLeft(_+_) - v = 10 - sc.parallelize(1 to 10).map(x => v).take(10).reduceLeft(_+_) - """) - assertDoesNotContain("error:", output) - assertDoesNotContain("Exception", output) - assertContains("res0: Int = 70", output) - assertContains("res1: Int = 100", output) - } - - test("external classes") { - val output = runInterpreter(sparkUrl, """ - class C { - def foo = 5 - } - sc.parallelize(1 to 10).map(x => (new C).foo).take(10).reduceLeft(_+_) - """) - assertDoesNotContain("error:", output) - assertDoesNotContain("Exception", output) - assertContains("res0: Int = 50", output) - } - - test("external functions") { - val output = runInterpreter(sparkUrl, """ - def double(x: Int) = x + x - sc.parallelize(1 to 10).map(x => double(x)).take(10).reduceLeft(_+_) - """) - assertDoesNotContain("error:", output) - assertDoesNotContain("Exception", output) - assertContains("res0: Int = 110", output) - } - - test("external functions that access vars") { - val output = runInterpreter(sparkUrl, """ - var v = 7 - def getV() = v - sc.parallelize(1 to 10).map(x => getV()).take(10).reduceLeft(_+_) - v = 10 - sc.parallelize(1 to 10).map(x => getV()).take(10).reduceLeft(_+_) - """) - assertDoesNotContain("error:", output) - assertDoesNotContain("Exception", output) - 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 - - val output = runInterpreter(sparkUrl, """ - var array = new Array[Int](5) - val broadcastArray = sc.broadcast(array) - sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).take(5) - array(0) = 5 - sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).take(5) - """) - assertDoesNotContain("error:", output) - assertDoesNotContain("Exception", output) - assertContains("res0: Array[Int] = Array(0, 0, 0, 0, 0)", output) - assertContains("res2: Array[Int] = Array(5, 0, 0, 0, 0)", output) - } -} From 0ecc33f0c80733ff88518157be543ec458a76bdb Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 11:25:58 -0700 Subject: [PATCH 0320/2521] 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 --- .../deploy/master/ui/ApplicationPage.scala | 4 +- .../scala/spark/deploy/worker/Worker.scala | 2 +- .../spark/deploy/worker/ui/IndexPage.scala | 9 +++-- .../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 27549e3b4a9..b2589abb89b 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 6ae1cef9402..f20ea42d7ff 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 313ecec0849..fc1ec31b4dd 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 cac281eef63..3ab0f2eded1 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() { From 044d4577ec564fe0f0bd9c1ad86f4749b50fc100 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 12:02:15 -0700 Subject: [PATCH 0321/2521] 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 3ab0f2eded1..923b835c3fe 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() { From e3a3fcf61b117d63db33ee28928dfd77cfd935b8 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 12:16:38 -0700 Subject: [PATCH 0322/2521] 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 923b835c3fe..a3cbe4f5d3c 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}
    From 311e63f2c7eb570c83184fa6698461ffd3b08971 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 11 Jul 2013 14:13:10 -0700 Subject: [PATCH 0323/2521] Removing mesos terminology --- ec2/deploy.generic/root/spark-ec2/ec2-variables.sh | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh index 72bf911a3fd..5ea0776d086 100644 --- a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh +++ b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh @@ -1,12 +1,11 @@ #!/bin/bash -# These variables are automatically filled in by the mesos-ec2 script. -export MESOS_MASTERS="{{master_list}}" -export MESOS_SLAVES="{{slave_list}}" -export MESOS_ZOO_LIST="{{zoo_list}}" -export MESOS_HDFS_DATA_DIRS="{{hdfs_data_dirs}}" -export MESOS_MAPRED_LOCAL_DIRS="{{mapred_local_dirs}}" -export MESOS_SPARK_LOCAL_DIRS="{{spark_local_dirs}}" +# These variables are automatically filled in by the spark-ec2 script. +export MASTERS="{{master_list}}" +export SLAVES="{{slave_list}}" +export HDFS_DATA_DIRS="{{hdfs_data_dirs}}" +export MAPRED_LOCAL_DIRS="{{mapred_local_dirs}}" +export SPARK_LOCAL_DIRS="{{spark_local_dirs}}" export MODULES="{{modules}}" export SPARK_VERSION="{{spark_version}}" export SHARK_VERSION="{{shark_version}}" From 11872888ca44d0c08c157973a8e35d344b0119e4 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 14:56:37 -0700 Subject: [PATCH 0324/2521] Created getByteRange function for logs and log pages, removed lastNBytes function --- core/src/main/scala/spark/Utils.scala | 30 ++++++++++--- .../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 1da9c9574e8..04041d11796 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 a3cbe4f5d3c..f8ac682dbf4 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() { From 15fd11d65735265f6fcc25419b43c8c104a4bd17 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 15:01:50 -0700 Subject: [PATCH 0325/2521] 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 f8ac682dbf4..676de231f0b 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() { From 5d5dbc39f603201a3596031542d52742cfb30139 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 15:22:45 -0700 Subject: [PATCH 0326/2521] getByteRange moved to WorkerWebUI, takes converted parameters, returns only start/end offset --- core/src/main/scala/spark/Utils.scala | 23 ---------- .../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 04041d11796..5a37532306a 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 676de231f0b..1fb59de1d84 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()) } From fdc226a14cfdb5f699627191f1682763b8571126 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 15:36:43 -0700 Subject: [PATCH 0327/2521] Clarified start and end byte variable names --- .../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 1fb59de1d84..2a82c352316 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() { From f1689185d612fbdcc3234b15eb85e1cc909a0fb1 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 11 Jul 2013 15:44:31 -0700 Subject: [PATCH 0328/2521] Adding ability to custom deploy only Spark --- ec2/spark_ec2.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index d8890f26f4e..dd75c49abe6 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -495,8 +495,15 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): cluster_url = "%s:7077" % active_master if "." in opts.spark_version: + # Pre-built deploy (spark_v, shark_v) = get_spark_shark_version(opts) + else if opts.shark_version is None: + # Spark-only custom deploy + spark_v = "%s|%s" % (opts.spark_git_repo, opts.spark_version) + shark_v = "" + modules = filter(lambda x: x != "shark", modules) else: + # Spark and Shark custom deploy spark_v = "%s|%s" % (opts.spark_git_repo, opts.spark_version) shark_v = "%s|%s" % (opts.shark_git_repo, opts.shark_version) From 13809d363c0a5b3212751561e0ecc09c601caf8a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 11 Jul 2013 15:45:35 -0700 Subject: [PATCH 0329/2521] Small fix --- 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 dd75c49abe6..425d397c3ef 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -497,7 +497,7 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): if "." in opts.spark_version: # Pre-built deploy (spark_v, shark_v) = get_spark_shark_version(opts) - else if opts.shark_version is None: + elif opts.shark_version is None: # Spark-only custom deploy spark_v = "%s|%s" % (opts.spark_git_repo, opts.spark_version) shark_v = "" From 31c18a2528bf0e89a96857bab14672617a2dd35d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 11 Jul 2013 16:18:16 -0700 Subject: [PATCH 0330/2521] Removing support for custom Shark version --- ec2/spark_ec2.py | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 425d397c3ef..0ce5ce867f2 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -71,11 +71,6 @@ def parse_args(): parser.add_option("--spark-git-repo", default="https://github.com/mesos/spark", help="Github repo from which to checkout supplied commit hash") - parser.add_option("--shark-version", - help="Git hash of shark version. Used only if spark hash is also given.") - parser.add_option("--shark-git-repo", - default="https://github.com/amplab/shark", - help="Github repo from which to checkout supplied commit hash") parser.add_option("--hadoop-major-version", default="2", help="Major version of Hadoop (default: 2)") @@ -495,17 +490,13 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): cluster_url = "%s:7077" % active_master if "." in opts.spark_version: - # Pre-built deploy + # Pre-built spark & shark deploy (spark_v, shark_v) = get_spark_shark_version(opts) - elif opts.shark_version is None: + else: # Spark-only custom deploy spark_v = "%s|%s" % (opts.spark_git_repo, opts.spark_version) shark_v = "" modules = filter(lambda x: x != "shark", modules) - else: - # Spark and Shark custom deploy - spark_v = "%s|%s" % (opts.spark_git_repo, opts.spark_version) - shark_v = "%s|%s" % (opts.shark_git_repo, opts.shark_version) template_vars = { "master_list": '\n'.join([i.public_dns_name for i in master_nodes]), From 9ed036ccdbb1c7aa3279998f3177c1be6b01e16b Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 16:33:53 -0700 Subject: [PATCH 0331/2521] Replaced logPageLength with byteLength to prevent buffer shrink bug --- .../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 2a82c352316..94d919f57af 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) From ece2388585a468dd57c973f254ba20c24299a464 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 16:35:56 -0700 Subject: [PATCH 0332/2521] 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 94d919f57af..1ab65d41e85 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)} From 8dccee16af14284eff0b9dbddcde6c8303abb474 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 11 Jul 2013 16:50:27 -0700 Subject: [PATCH 0333/2521] Bug fix --- 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 0ce5ce867f2..413fcf26953 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -236,7 +236,7 @@ def launch_cluster(conn, opts, cluster_name): die_on_error=False) if any(active_nodes): print >> stderr, ("ERROR: There are already instances running in " + - "group %s, %s or %s" % (master_group.name, slave_group.name)) + "group %s or %s" % (master_group.name, slave_group.name)) sys.exit(1) # Figure out Spark AMI From a32784109d11e061cdca2eced07ac01be2061056 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 16:57:55 -0700 Subject: [PATCH 0334/2521] 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 dbdc8e1057a..4dd6c448a97 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() } } From 6d054487bf4c9e9aa9033187f73e4e9d8366bc37 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 17:12:17 -0700 Subject: [PATCH 0335/2521] Replace default buffer value to 100 GB, changed buttons to use String notation, removed default buffer parameter in UI URLs --- .../spark/deploy/master/ui/ApplicationPage.scala | 4 ++-- .../scala/spark/deploy/worker/ui/IndexPage.scala | 4 ++-- .../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 b2589abb89b..8553377d8f5 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 fc1ec31b4dd..c515f2e2382 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 1ab65d41e85..4e7f86d77a9 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) From 5c67ca027834582bcd6cdd55baee5fd74e743a71 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 11 Jul 2013 17:31:59 -0700 Subject: [PATCH 0336/2521] Remove "Bytes" in lieu of String notation --- .../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 4e7f86d77a9..ccd55c1ce46 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 = From a2c915fba84a1deadac199994322e0f15d5750b4 Mon Sep 17 00:00:00 2001 From: seanm Date: Thu, 11 Jul 2013 18:55:00 -0700 Subject: [PATCH 0337/2521] 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 af52040fa68..4c1591ed5ae 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 fe17d1d5e71..aa3ee5f5eea 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") { From 6e6d94ffdfa885432408d6996bad7df2a641748a Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 12 Jul 2013 11:55:16 +0530 Subject: [PATCH 0338/2521] Added add jars functionality to new repl, which was dropped while merging with old. --- repl/src/main/scala/spark/repl/SparkILoop.scala | 11 +++++++++-- repl/src/main/scala/spark/repl/SparkILoopInit.scala | 2 +- repl/src/test/scala/spark/repl/ReplSuiteMixin.scala | 2 +- 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/spark/repl/SparkILoop.scala index 2b6e7b68bf3..cc7a63e166b 100644 --- a/repl/src/main/scala/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/spark/repl/SparkILoop.scala @@ -30,7 +30,7 @@ import scala.tools.util._ import scala.language.{implicitConversions, existentials} import scala.reflect.{ClassTag, classTag} import scala.tools.reflect.StdRuntimeTags._ -import scala.reflect.{ClassTag, classTag} + import java.lang.{Class => jClass} import scala.reflect.api.{Mirror, TypeCreator, Universe => ApiUniverse} @@ -641,6 +641,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, reset() } + def reset() { intp.reset() // unleashAndSetPhase() @@ -921,7 +922,8 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, if (prop != null) prop else "local" } } - sparkContext = new SparkContext(master, "Spark shell") + val jars = SparkILoop.getAddedJars.map(new java.io.File(_).getAbsolutePath) + sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars) echo("Created spark context..") sparkContext } @@ -948,6 +950,8 @@ object SparkILoop { implicit def loopToInterpreter(repl: SparkILoop): SparkIMain = repl.intp private def echo(msg: String) = Console println msg + def getAddedJars: Array[String] = Option(System.getenv("ADD_JARS")).map(_.split(',')).getOrElse(new Array[String](0)) + // Designed primarily for use by test code: take a String with a // bunch of code, and prints out a transcript of what it would look // like if you'd just typed it into the repl. @@ -975,9 +979,12 @@ object SparkILoop { } } val repl = new SparkILoop(input, output) + if (settings.classpath.isDefault) settings.classpath.value = sys.props("java.class.path") + getAddedJars.foreach(settings.classpath.append(_)) + repl process settings } } diff --git a/repl/src/main/scala/spark/repl/SparkILoopInit.scala b/repl/src/main/scala/spark/repl/SparkILoopInit.scala index 8b7da3d3c69..b89495d2f3b 100644 --- a/repl/src/main/scala/spark/repl/SparkILoopInit.scala +++ b/repl/src/main/scala/spark/repl/SparkILoopInit.scala @@ -119,7 +119,7 @@ trait SparkILoopInit { command(""" @transient val sc = spark.repl.Main.interp.createSparkContext(); """) - command("import spark.SparkContext._"); + command("import spark.SparkContext._") } echo("Spark context available as sc.") } diff --git a/repl/src/test/scala/spark/repl/ReplSuiteMixin.scala b/repl/src/test/scala/spark/repl/ReplSuiteMixin.scala index 3b5b6385adf..d88e44ad198 100644 --- a/repl/src/test/scala/spark/repl/ReplSuiteMixin.scala +++ b/repl/src/test/scala/spark/repl/ReplSuiteMixin.scala @@ -36,7 +36,7 @@ trait ReplSuiteMixin { interp.closeInterpreter() } // 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") return out.toString } From 2080e250060975a876a388eb785e7f2b3cf2c0cd Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Fri, 12 Jul 2013 14:25:18 +0800 Subject: [PATCH 0339/2521] Enhance job ui in spark ui system with adding pool information --- core/src/main/scala/spark/SparkContext.scala | 17 ++- .../scala/spark/scheduler/DAGScheduler.scala | 4 +- .../scala/spark/scheduler/JobLogger.scala | 2 +- .../scala/spark/scheduler/SparkListener.scala | 2 +- .../scala/spark/scheduler/TaskScheduler.scala | 7 + .../scheduler/cluster/ClusterScheduler.scala | 11 +- .../cluster/ClusterTaskSetManager.scala | 4 +- .../spark/scheduler/cluster/Schedulable.scala | 6 +- .../scheduler/cluster/SchedulingMode.scala | 4 +- .../scheduler/cluster/TaskSetManager.scala | 1 + .../scheduler/local/LocalScheduler.scala | 10 +- .../scheduler/local/LocalTaskSetManager.scala | 1 + .../main/scala/spark/ui/jobs/IndexPage.scala | 121 ++++----------- .../spark/ui/jobs/JobProgressListener.scala | 140 +++++++++++++++++ .../scala/spark/ui/jobs/JobProgressUI.scala | 112 +++----------- .../main/scala/spark/ui/jobs/PoolPage.scala | 38 +++++ .../main/scala/spark/ui/jobs/PoolTable.scala | 98 ++++++++++++ .../main/scala/spark/ui/jobs/StageTable.scala | 143 ++++++++++++++++++ .../spark/scheduler/DAGSchedulerSuite.scala | 6 + .../spark/scheduler/JobLoggerSuite.scala | 2 +- 20 files changed, 530 insertions(+), 199 deletions(-) create mode 100644 core/src/main/scala/spark/ui/jobs/JobProgressListener.scala create mode 100644 core/src/main/scala/spark/ui/jobs/PoolPage.scala create mode 100644 core/src/main/scala/spark/ui/jobs/PoolTable.scala create mode 100644 core/src/main/scala/spark/ui/jobs/StageTable.scala diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 43e6af2351a..b5225d56812 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -10,6 +10,7 @@ import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.generic.Growable import scala.collection.mutable.HashMap +import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.util.DynamicVariable import scala.collection.mutable.{ConcurrentMap, HashMap} @@ -43,13 +44,14 @@ 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.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler} +import spark.scheduler.{DAGScheduler, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler, ActiveJob} +import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler, Schedulable} 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} +import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -540,6 +542,17 @@ class SparkContext( env.blockManager.master.getStorageStatus } + def getPoolsInfo: ArrayBuffer[Schedulable] = { + taskScheduler.rootPool.schedulableQueue + } + + def getSchedulingMode: SchedulingMode = { + taskScheduler.schedulingMode + } + + def getPoolNameToPool: HashMap[String, Schedulable] = { + taskScheduler.rootPool.schedulableNameToSchedulable + } /** * 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/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 3d3b9ea0118..c865743e376 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -472,11 +472,11 @@ class DAGScheduler( } } if (tasks.size > 0) { - sparkListeners.foreach(_.onStageSubmitted(SparkListenerStageSubmitted(stage, tasks.size))) + val properties = idToActiveJob(stage.priority).properties + sparkListeners.foreach(_.onStageSubmitted(SparkListenerStageSubmitted(stage, tasks.size, properties))) 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, properties)) if (!stage.submissionTime.isDefined) { diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index 6a9d52f3561..8e5540873f9 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -45,7 +45,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { event match { case SparkListenerJobStart(job, properties) => processJobStartEvent(job, properties) - case SparkListenerStageSubmitted(stage, taskSize) => + case SparkListenerStageSubmitted(stage, taskSize, properties) => processStageSubmittedEvent(stage, taskSize) case StageCompleted(stageInfo) => processStageCompletedEvent(stageInfo) diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 8de3aa91a4d..94fdad9b985 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -8,7 +8,7 @@ import spark.executor.TaskMetrics sealed trait SparkListenerEvents -case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int) extends SparkListenerEvents +case class SparkListenerStageSubmitted(stage: Stage, taskSize: Int, properties: Properties = null) extends SparkListenerEvents case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents diff --git a/core/src/main/scala/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/spark/scheduler/TaskScheduler.scala index 7787b547620..5cdf846032d 100644 --- a/core/src/main/scala/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/spark/scheduler/TaskScheduler.scala @@ -1,5 +1,7 @@ package spark.scheduler +import spark.scheduler.cluster.Pool +import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * Low-level task scheduler interface, implemented by both ClusterScheduler and LocalScheduler. * These schedulers get sets of tasks submitted to them from the DAGScheduler for each stage, @@ -8,6 +10,11 @@ package spark.scheduler * the TaskSchedulerListener interface. */ private[spark] trait TaskScheduler { + + def rootPool: Pool + + def schedulingMode: SchedulingMode + def start(): Unit // Invoked after system has successfully initialized (typically in spark context). diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 3a0c29b27fe..1b23fd6cef5 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -12,6 +12,7 @@ import spark.scheduler._ import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicLong import java.util.{TimerTask, Timer} +import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * The main TaskScheduler implementation, for running tasks on a cluster. Clients should first call @@ -97,6 +98,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) var schedulableBuilder: SchedulableBuilder = null var rootPool: Pool = null + //default scheduler is FIFO + val schedulingMode: SchedulingMode = SchedulingMode.withName(System.getProperty("spark.cluster.schedulingmode", "FIFO")) override def setListener(listener: TaskSchedulerListener) { this.listener = listener @@ -104,15 +107,13 @@ private[spark] class ClusterScheduler(val sc: SparkContext) def initialize(context: SchedulerBackend) { backend = context - //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) + rootPool = new Pool("", schedulingMode, 0, 0) schedulableBuilder = { schedulingMode match { - case "FIFO" => + case SchedulingMode.FIFO => new FIFOSchedulableBuilder(rootPool) - case "FAIR" => + case SchedulingMode.FAIR => new FairSchedulableBuilder(rootPool) } } diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index fe6420a522c..7a6a6b78269 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -90,8 +90,8 @@ private[spark] class ClusterTaskSetManager( var priority = taskSet.priority var stageId = taskSet.stageId var name = "TaskSet_"+taskSet.stageId.toString - var parent:Schedulable = null - + var parent: Schedulable = null + var schedulableQueue :ArrayBuffer[Schedulable] = null // Last time when we launched a preferred task (for delay scheduling) var lastPreferredLaunchTime = System.currentTimeMillis diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala index 2dd9c0564fc..2e4f14c11f2 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala @@ -1,13 +1,17 @@ package spark.scheduler.cluster -import scala.collection.mutable.ArrayBuffer +import spark.scheduler.cluster.SchedulingMode.SchedulingMode +import scala.collection.mutable.ArrayBuffer /** * An interface for schedulable entities. * there are two type of Schedulable entities(Pools and TaskSetManagers) */ private[spark] trait Schedulable { var parent: Schedulable + //childrens + def schedulableQueue: ArrayBuffer[Schedulable] + def schedulingMode: SchedulingMode def weight: Int def minShare: Int def runningTasks: Int diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala index 6e0c6793e01..c5c7ee3b229 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala @@ -1,7 +1,7 @@ package spark.scheduler.cluster -object SchedulingMode extends Enumeration("FAIR","FIFO"){ +object SchedulingMode extends Enumeration("FAIR", "FIFO", "NONE"){ type SchedulingMode = Value - val FAIR,FIFO = Value + val FAIR,FIFO,NONE = Value } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index b4dd75d90f2..472e01b227e 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -6,6 +6,7 @@ import spark.TaskState.TaskState import java.nio.ByteBuffer private[spark] trait TaskSetManager extends Schedulable { + def schedulingMode = SchedulingMode.NONE def taskSet: TaskSet def slaveOffer(execId: String, hostPort: String, availableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index b000e328e67..19a48895e31 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -12,6 +12,7 @@ import spark.TaskState.TaskState import spark.executor.ExecutorURLClassLoader import spark.scheduler._ import spark.scheduler.cluster._ +import spark.scheduler.cluster.SchedulingMode.SchedulingMode import akka.actor._ /** @@ -63,6 +64,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: var schedulableBuilder: SchedulableBuilder = null var rootPool: Pool = null + val schedulingMode: SchedulingMode = SchedulingMode.withName(System.getProperty("spark.cluster.schedulingmode", "FIFO")) val activeTaskSets = new HashMap[String, TaskSetManager] val taskIdToTaskSetId = new HashMap[Long, String] val taskSetTaskIds = new HashMap[String, HashSet[Long]] @@ -70,15 +72,13 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: var localActor: ActorRef = null 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) + rootPool = new Pool("", schedulingMode, 0, 0) schedulableBuilder = { schedulingMode match { - case "FIFO" => + case SchedulingMode.FIFO => new FIFOSchedulableBuilder(rootPool) - case "FAIR" => + case SchedulingMode.FAIR => new FairSchedulableBuilder(rootPool) } } diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index f12fec41d50..8954f40ea90 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -14,6 +14,7 @@ import spark.scheduler.cluster._ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) extends TaskSetManager with Logging { var parent: Schedulable = null + var schedulableQueue :ArrayBuffer[Schedulable] = null var weight: Int = 1 var minShare: Int = 0 var runningTasks: Int = 0 diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 1e675ab2cb2..e765cecb011 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -6,107 +6,52 @@ import javax.servlet.http.HttpServletRequest import scala.Some import scala.xml.{NodeSeq, Node} +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet import spark.scheduler.Stage import spark.ui.UIUtils._ import spark.ui.Page._ import spark.storage.StorageLevel +import spark.scheduler.cluster.Schedulable +import spark.scheduler.cluster.SchedulingMode +import spark.scheduler.cluster.SchedulingMode.SchedulingMode -/** Page showing list of all ongoing and recently finished stages */ +/** Page showing list of all ongoing and recently finished stages and pools*/ private[spark] class IndexPage(parent: JobProgressUI) { def listener = parent.listener - val dateFmt = parent.dateFmt + + def stageTable: StageTable = parent.stageTable + + def poolTable: PoolTable = parent.poolTable def render(request: HttpServletRequest): Seq[Node] = { val activeStages = listener.activeStages.toSeq val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq - /** 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 ++ -

    Failed Stages

    ++ failedStageTable - - headerSparkPage(content, parent.sc, "Spark Stages", Jobs) - } - - def getElapsedTime(submitted: Option[Long], completed: Long): String = { - submitted match { - case Some(t) => parent.formatDuration(completed - t) - case _ => "Unknown" - } - } - - def makeProgressBar(completed: Int, total: Int): Seq[Node] = { - val width=130 - val height=15 - val completeWidth = (completed.toDouble / total) * width - - - - - - } - - - def stageRow(s: Stage): Seq[Node] = { - val submissionTime = s.submissionTime match { - case Some(t) => dateFmt.format(new Date(t)) - case None => "Unknown" - } - val (read, write) = (listener.hasShuffleRead(s.id), listener.hasShuffleWrite(s.id)) - val shuffleInfo = (read, write) match { - case (true, true) => "Read/Write" - case (true, false) => "Read" - case (false, true) => "Write" - case _ => "" - } - val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) - val totalTasks = s.numPartitions - - - {s.id} - {s.origin} - {submissionTime} - {getElapsedTime(s.submissionTime, - s.completionTime.getOrElse(System.currentTimeMillis()))} - {makeProgressBar(completedTasks, totalTasks)} - {completedTasks} / {totalTasks} - {listener.stageToTasksFailed.getOrElse(s.id, 0) match { - case f if f > 0 => "(%s failed)".format(f) - case _ => - }} - - {shuffleInfo} - {if (s.rdd.getStorageLevel != StorageLevel.NONE) { - - {Option(s.rdd.name).getOrElse(s.rdd.id)} - - }} - - + stageTable.setStagePoolInfo(parent.stagePoolInfo) + poolTable.setPoolSource(parent.stagePagePoolSource) + + val activeStageNodeSeq = stageTable.toNodeSeq(activeStages) + val completedStageNodeSeq = stageTable.toNodeSeq(completedStages) + val failedStageNodeSeq = stageTable.toNodeSeq(failedStages) + + val content =
    +
    +
      +
    • Active Stages Number: {activeStages.size}
    • +
    • Completed Stages Number: {completedStages.size}
    • +
    • Failed Stages Number: {failedStages.size}
    • +
    • Scheduling Mode: {parent.sc.getSchedulingMode}
    • +
    +
    +
    ++ +

    Pools

    ++ poolTable.toNodeSeq ++ +

    Active Stages : {activeStages.size}

    ++ activeStageNodeSeq++ +

    Completed Stages : {completedStages.size}

    ++ completedStageNodeSeq++ +

    Failed Stages : {failedStages.size}

    ++ failedStageNodeSeq + + headerSparkPage(content, parent.sc, "Spark Stages/Pools", Jobs) } } diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala new file mode 100644 index 00000000000..1244f9538b7 --- /dev/null +++ b/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala @@ -0,0 +1,140 @@ +package spark.ui.jobs + +import scala.Seq +import scala.collection.mutable.{HashSet, ListBuffer, HashMap, ArrayBuffer} + +import spark.{ExceptionFailure, SparkContext, Success, Utils} +import spark.scheduler._ +import spark.scheduler.cluster.TaskInfo +import spark.executor.TaskMetrics +import collection.mutable + +private[spark] class FairJobProgressListener(val sparkContext: SparkContext) + extends JobProgressListener(sparkContext) { + + val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.cluster.fair.pool" + val DEFAULT_POOL_NAME = "default" + + override val stageToPool = HashMap[Stage, String]() + override val poolToActiveStages = HashMap[String, HashSet[Stage]]() + + override def onStageCompleted(stageCompleted: StageCompleted) = { + super.onStageCompleted(stageCompleted) + val stage = stageCompleted.stageInfo.stage + poolToActiveStages(stageToPool(stage)) -= stage + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = { + super.onStageSubmitted(stageSubmitted) + val stage = stageSubmitted.stage + var poolName = DEFAULT_POOL_NAME + if (stageSubmitted.properties != null) { + poolName = stageSubmitted.properties.getProperty(FAIR_SCHEDULER_PROPERTIES, DEFAULT_POOL_NAME) + } + stageToPool(stage) = poolName + val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]()) + stages += stage + } + + override def onJobEnd(jobEnd: SparkListenerJobEnd) { + super.onJobEnd(jobEnd) + jobEnd match { + case end: SparkListenerJobEnd => + end.jobResult match { + case JobFailed(ex, Some(stage)) => + poolToActiveStages(stageToPool(stage)) -= stage + case _ => + } + case _ => + } + } +} + +private[spark] class JobProgressListener(val sc: SparkContext) extends SparkListener { + // How many stages to remember + val RETAINED_STAGES = System.getProperty("spark.ui.retained_stages", "1000").toInt + + def stageToPool: HashMap[Stage, String] = null + def poolToActiveStages: HashMap[String, HashSet[Stage]] =null + + 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, Option[TaskMetrics], Option[ExceptionFailure])]]() + + override def onJobStart(jobStart: SparkListenerJobStart) {} + + override def onStageCompleted(stageCompleted: StageCompleted) = { + val stage = stageCompleted.stageInfo.stage + activeStages -= stage + completedStages += stage + trimIfNecessary(completedStages) + } + + /** 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 => { + stageToTaskInfos.remove(s.id) + }) + stages.trimEnd(toRemove) + } + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = + activeStages += stageSubmitted.stage + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val sid = taskEnd.task.stageId + val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = + taskEnd.reason match { + case e: ExceptionFailure => + stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 + (Some(e), e.metrics) + case _ => + stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 + (None, Some(taskEnd.taskMetrics)) + } + val taskList = stageToTaskInfos.getOrElse( + sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) + taskList += ((taskEnd.taskInfo, metrics, failureInfo)) + stageToTaskInfos(sid) = taskList + } + + override def onJobEnd(jobEnd: SparkListenerJobEnd) { + jobEnd match { + case end: SparkListenerJobEnd => + end.jobResult match { + case JobFailed(ex, Some(stage)) => + activeStages -= stage + failedStages += stage + trimIfNecessary(failedStages) + case _ => + } + case _ => + } + } + + /** 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.flatMap(m => m.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.flatMap(m => m.shuffleWriteMetrics).isDefined + } + return false // No tasks have finished for this stage + } +} diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 84730cc091b..e610252242d 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -14,9 +14,9 @@ import scala.collection.mutable.{HashSet, ListBuffer, HashMap, ArrayBuffer} import spark.ui.JettyUtils._ import spark.{ExceptionFailure, SparkContext, Success, Utils} import spark.scheduler._ -import spark.scheduler.cluster.TaskInfo -import spark.executor.TaskMetrics import collection.mutable +import spark.scheduler.cluster.SchedulingMode +import spark.scheduler.cluster.SchedulingMode.SchedulingMode /** Web UI showing progress status of all jobs in the given SparkContext. */ private[spark] class JobProgressUI(val sc: SparkContext) { @@ -24,104 +24,38 @@ private[spark] class JobProgressUI(val sc: SparkContext) { def listener = _listener.get val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + private val indexPage = new IndexPage(this) private val stagePage = new StagePage(this) + private val poolPage = new PoolPage(this) + + var stageTable: StageTable = null + var stagePoolInfo: StagePoolInfo = null + var poolTable: PoolTable = null + var stagePagePoolSource: PoolSource = null def start() { - _listener = Some(new JobProgressListener) + sc.getSchedulingMode match { + case SchedulingMode.FIFO => + _listener = Some(new JobProgressListener(sc)) + stagePoolInfo = new FIFOStagePoolInfo() + stagePagePoolSource = new FIFOSource() + case SchedulingMode.FAIR => + _listener = Some(new FairJobProgressListener(sc)) + stagePoolInfo = new FairStagePoolInfo(listener) + stagePagePoolSource = new FairSource(sc) + } + sc.addSparkListener(listener) + stageTable = new StageTable(dateFmt, formatDuration, listener) + poolTable = new PoolTable(listener) } def formatDuration(ms: Long) = Utils.msDurationToString(ms) def getHandlers = Seq[(String, Handler)]( ("/stages/stage", (request: HttpServletRequest) => stagePage.render(request)), + ("/stages/pool", (request: HttpServletRequest) => poolPage.render(request)), ("/stages", (request: HttpServletRequest) => indexPage.render(request)) ) } - -private[spark] class JobProgressListener extends SparkListener { - // How many stages to remember - val RETAINED_STAGES = System.getProperty("spark.ui.retained_stages", "1000").toInt - - 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, Option[TaskMetrics], Option[ExceptionFailure])]]() - - override def onJobStart(jobStart: SparkListenerJobStart) {} - - override def onStageCompleted(stageCompleted: StageCompleted) = { - val stage = stageCompleted.stageInfo.stage - activeStages -= stage - completedStages += stage - trimIfNecessary(completedStages) - } - - /** 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 => { - stageToTaskInfos.remove(s.id) - }) - stages.trimEnd(toRemove) - } - } - - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = - activeStages += stageSubmitted.stage - - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val sid = taskEnd.task.stageId - val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = - taskEnd.reason match { - case e: ExceptionFailure => - stageToTasksFailed(sid) = stageToTasksFailed.getOrElse(sid, 0) + 1 - (Some(e), e.metrics) - case _ => - stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 - (None, Some(taskEnd.taskMetrics)) - } - val taskList = stageToTaskInfos.getOrElse( - sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) - taskList += ((taskEnd.taskInfo, metrics, failureInfo)) - stageToTaskInfos(sid) = taskList - } - - override def onJobEnd(jobEnd: SparkListenerJobEnd) { - jobEnd match { - case end: SparkListenerJobEnd => - end.jobResult match { - case JobFailed(ex, Some(stage)) => - activeStages -= stage - failedStages += stage - trimIfNecessary(failedStages) - case _ => - } - case _ => - } - } - - /** 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.flatMap(m => m.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.flatMap(m => m.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/PoolPage.scala b/core/src/main/scala/spark/ui/jobs/PoolPage.scala new file mode 100644 index 00000000000..00703887c3b --- /dev/null +++ b/core/src/main/scala/spark/ui/jobs/PoolPage.scala @@ -0,0 +1,38 @@ +package spark.ui.jobs + +import javax.servlet.http.HttpServletRequest + +import scala.xml.{NodeSeq, Node} +import scala.collection.mutable.HashSet + +import spark.scheduler.Stage +import spark.ui.UIUtils._ +import spark.ui.Page._ + +/** Page showing specific pool details*/ +private[spark] class PoolPage(parent: JobProgressUI) { + def listener = parent.listener + + def stageTable: StageTable = parent.stageTable + + def poolTable: PoolTable = parent.poolTable + + def render(request: HttpServletRequest): Seq[Node] = { + val poolName = request.getParameter("poolname") + val poolToActiveStages = listener.poolToActiveStages + val activeStages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[Stage]).toSeq + val stageToPool = listener.stageToPool + + val poolDetailPoolSource = new PoolDetailSource(parent.sc, poolName) + poolTable.setPoolSource(poolDetailPoolSource) + + stageTable.setStagePoolInfo(parent.stagePoolInfo) + + val activeStageNodeSeq = stageTable.toNodeSeq(activeStages) + + val content =

    Pool

    ++ poolTable.toNodeSeq ++ +

    Active Stages : {activeStages.size}

    ++ activeStageNodeSeq + + headerSparkPage(content, parent.sc, "Spark Pool Details", Jobs) + } +} diff --git a/core/src/main/scala/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/spark/ui/jobs/PoolTable.scala new file mode 100644 index 00000000000..bb8be4b26e3 --- /dev/null +++ b/core/src/main/scala/spark/ui/jobs/PoolTable.scala @@ -0,0 +1,98 @@ +package spark.ui.jobs + +import java.util.Date + +import javax.servlet.http.HttpServletRequest + +import scala.Some +import scala.xml.{NodeSeq, Node} +import scala.collection.mutable.HashMap +import scala.collection.mutable.HashSet + +import spark.SparkContext +import spark.scheduler.Stage +import spark.ui.UIUtils._ +import spark.ui.Page._ +import spark.storage.StorageLevel +import spark.scheduler.cluster.Schedulable + +/* + * Interface for get pools seq showing on Index or pool detail page + */ + +private[spark] trait PoolSource { + def getPools: Seq[Schedulable] +} + +/* + * Pool source for FIFO scheduler algorithm on Index page + */ +private[spark] class FIFOSource() extends PoolSource{ + def getPools: Seq[Schedulable] = { + Seq[Schedulable]() + } +} + +/* + * Pool source for Fair scheduler algorithm on Index page + */ +private[spark] class FairSource(sc: SparkContext) extends PoolSource{ + def getPools: Seq[Schedulable] = { + sc.getPoolsInfo.toSeq + } +} + +/* + * specific pool info for pool detail page + */ +private[spark] class PoolDetailSource(sc: SparkContext, poolName: String) extends PoolSource{ + def getPools: Seq[Schedulable] = { + val pools = HashSet[Schedulable]() + pools += sc.getPoolNameToPool(poolName) + pools.toSeq + } +} + +/** Table showing list of pools */ +private[spark] class PoolTable(listener: JobProgressListener) { + + var poolSource: PoolSource = null + var poolToActiveStages: HashMap[String, HashSet[Stage]] = listener.poolToActiveStages + + def toNodeSeq: Seq[Node] = { + poolTable(poolRow, poolSource.getPools) + } + + def setPoolSource(poolSource: PoolSource) { + this.poolSource = poolSource + } + + //pool tables + def poolTable(makeRow: (Schedulable, HashMap[String, HashSet[Stage]]) => Seq[Node], rows: Seq[Schedulable]): Seq[Node] = { + + + + + + + + + + + {rows.map(r => makeRow(r, poolToActiveStages))} + +
    Pool NameMinimum SharePool WeightActive StagesRunning TasksSchedulingMode
    + } + + def poolRow(p: Schedulable, poolToActiveStages: HashMap[String, HashSet[Stage]]): Seq[Node] = { + + {p.name} + {p.minShare} + {p.weight} + {poolToActiveStages.getOrElseUpdate(p.name, new HashSet[Stage]()).size} + {p.runningTasks} + {p.schedulingMode} + + } +} + diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/spark/ui/jobs/StageTable.scala new file mode 100644 index 00000000000..83e566c55b8 --- /dev/null +++ b/core/src/main/scala/spark/ui/jobs/StageTable.scala @@ -0,0 +1,143 @@ +package spark.ui.jobs + +import java.util.Date +import java.text.SimpleDateFormat + +import javax.servlet.http.HttpServletRequest + +import scala.Some +import scala.xml.{NodeSeq, Node} +import scala.collection.mutable.HashMap + +import spark.scheduler.Stage +import spark.ui.UIUtils._ +import spark.ui.Page._ +import spark.storage.StorageLevel + +/* + * Interface to get stage's pool name + */ +private[spark] trait StagePoolInfo { + def getStagePoolName(s: Stage): String + + def hasHerf: Boolean +} + +/* + * For FIFO scheduler algorithm, just show "N/A" and its link status is false + */ +private[spark] class FIFOStagePoolInfo extends StagePoolInfo { + def getStagePoolName(s: Stage): String = "N/A" + + def hasHerf: Boolean = false +} + +/* + * For Fair scheduler algorithm, show its pool name and pool detail link status is true + */ +private[spark] class FairStagePoolInfo(listener: JobProgressListener) extends StagePoolInfo { + def getStagePoolName(s: Stage): String = { + listener.stageToPool(s) + } + + def hasHerf: Boolean = true +} + +/** Page showing list of all ongoing and recently finished stages */ +private[spark] class StageTable(val dateFmt: SimpleDateFormat, val formatDuration: Long => String, val listener: JobProgressListener) { + + var stagePoolInfo: StagePoolInfo = null + + def toNodeSeq(stages: Seq[Stage]): Seq[Node] = { + stageTable(stageRow, stages) + } + + def setStagePoolInfo(stagePoolInfo: StagePoolInfo) { + this.stagePoolInfo = stagePoolInfo + } + + /** 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 IdPool NameOriginSubmittedDurationTasks: Complete/TotalShuffle ActivityStored RDD
    + } + + def getElapsedTime(submitted: Option[Long], completed: Long): String = { + submitted match { + case Some(t) => formatDuration(completed - t) + case _ => "Unknown" + } + } + + def makeProgressBar(completed: Int, total: Int): Seq[Node] = { + val width=130 + val height=15 + val completeWidth = (completed.toDouble / total) * width + + + + + + } + + + def stageRow(s: Stage): Seq[Node] = { + val submissionTime = s.submissionTime match { + case Some(t) => dateFmt.format(new Date(t)) + case None => "Unknown" + } + val (read, write) = (listener.hasShuffleRead(s.id), listener.hasShuffleWrite(s.id)) + val shuffleInfo = (read, write) match { + case (true, true) => "Read/Write" + case (true, false) => "Read" + case (false, true) => "Write" + case _ => "" + } + val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) + val totalTasks = s.numPartitions + + val poolName = stagePoolInfo.getStagePoolName(s) + + + {s.id} + {if (stagePoolInfo.hasHerf) { + {poolName} + } else { + {poolName} + }} + {s.origin} + {submissionTime} + {getElapsedTime(s.submissionTime, + s.completionTime.getOrElse(System.currentTimeMillis()))} + {makeProgressBar(completedTasks, totalTasks)} + {completedTasks} / {totalTasks} + {listener.stageToTasksFailed.getOrElse(s.id, 0) match { + case f if f > 0 => "(%s failed)".format(f) + case _ => + }} + + {shuffleInfo} + {if (s.rdd.getStorageLevel != StorageLevel.NONE) { + + {Option(s.rdd.name).getOrElse(s.rdd.id)} + + }} + + + } +} diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index 30e6fef9505..da72bfbf89f 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -22,6 +22,10 @@ import spark.TaskEndReason import spark.{FetchFailed, Success} +import spark.scheduler.cluster.Pool +import spark.scheduler.cluster.SchedulingMode +import spark.scheduler.cluster.SchedulingMode.SchedulingMode + /** * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler * rather than spawning an event loop thread as happens in the real code. They use EasyMock @@ -39,6 +43,8 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont /** Set of TaskSets the DAGScheduler has requested executed. */ val taskSets = scala.collection.mutable.Buffer[TaskSet]() val taskScheduler = new TaskScheduler() { + override def rootPool: Pool = null + override def schedulingMode: SchedulingMode = SchedulingMode.NONE override def start() = {} override def stop() = {} override def submitTasks(taskSet: TaskSet) = { diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala index 699901f1a13..328e7e7529d 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(SparkListenerStageSubmitted(rootStage, 4)) + joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStage, 4, null)) joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getName) parentRdd.setName("MyRDD") joblogger.getRddNameTest(parentRdd) should be ("MyRDD") From a1662326e9b1486361eba2f2caf903875fbba597 Mon Sep 17 00:00:00 2001 From: seanm Date: Fri, 12 Jul 2013 08:38:19 -0700 Subject: [PATCH 0340/2521] 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 4c1591ed5ae..8aa77266bc0 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 94b95af714d..27f40ecdfd4 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 From cd7259b4b8d8abbff6db963fd8f84d4bd0b3737b Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Fri, 12 Jul 2013 11:51:14 -0700 Subject: [PATCH 0341/2521] 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 f5788dc4677..8cd1b0cd660 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`. From 531a7e55745ad76d17d41f80dbdea0072acce079 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Wed, 10 Jul 2013 17:59:43 -0700 Subject: [PATCH 0342/2521] Pass executor env vars (e.g. SPARK_CLASSPATH) to compute-classpath. --- core/src/main/scala/spark/Utils.scala | 12 +++++++++--- .../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 64547bbdcd2..d2bf151cbf6 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 d7f58b2cb1b..5d3d54c65e7 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 } From 73984b96a8450674b472676eaa855cc2df68a754 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 12 Jul 2013 14:26:56 -0700 Subject: [PATCH 0343/2521] 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 d83a0307fac..1e1260f6060 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" From 6855338e1400638188358a7d7926eb86f668c160 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 12 Jul 2013 19:24:16 -0700 Subject: [PATCH 0344/2521] 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. --- .../main/scala/spark/storage/StorageUtils.scala | 11 +++++++++-- .../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 950c0cdf352..3e7fa287e54 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 0cb1e47ea55..428db6fa956 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}
    )} + } From 08150f19abcf08f2a18305080b08736fb8a33a12 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 12 Jul 2013 19:32:35 -0700 Subject: [PATCH 0345/2521] 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 3e7fa287e54..aca16f533aa 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 } From c4d5b01e44c5f289d67670185a14ff4ccd9537db Mon Sep 17 00:00:00 2001 From: seanm Date: Sat, 13 Jul 2013 14:56:23 -0600 Subject: [PATCH 0346/2521] 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 641d3797497..0a4c176a29a 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, From 1889f9f450d3e28864a5ac7b5f9d3831cf7b17bf Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 13 Jul 2013 14:43:21 -0700 Subject: [PATCH 0347/2521] Increase PermGen size --- sbt/sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbt/sbt b/sbt/sbt index 850c58e1e97..523fbb346b3 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 "$@" From ddb97f0fdf16442afaa9cab656376267e4044510 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 13 Jul 2013 16:05:21 -0700 Subject: [PATCH 0348/2521] 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 ++ .../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 c853b44b765..ed8f91842cc 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 b3bdc2c4902..874e5ba8ecf 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 7b79290d1bd..36d9c472451 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 00000000000..8a3b3304056 --- /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) + } +} From e271fde10b342216e33f4f45af73c5d103215cf2 Mon Sep 17 00:00:00 2001 From: root Date: Sun, 14 Jul 2013 06:24:29 +0000 Subject: [PATCH 0349/2521] Fixed a delay scheduling bug in the YARN branch, found by Patrick --- .../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 fe6420a522c..327d6797ae2 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 From a44a7b123862202cf97f2de7a96aeaf29a93002a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 14 Jul 2013 07:23:09 +0000 Subject: [PATCH 0350/2521] 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 c6a3f97872b..a36186bf8a7 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) From b91a218cea5a7ab4037675667922fc06bfec6fbf Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 14 Jul 2013 07:31:33 +0000 Subject: [PATCH 0351/2521] Cosmetic fixes to web UI --- .../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 c6de2bafa33..7545ecf8685 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 1e675ab2cb2..7907ab3bc70 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))} From c5c38d1987137a1dc5eb66dd1065735a542ef9b5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 14 Jul 2013 07:59:50 +0000 Subject: [PATCH 0352/2521] Some optimizations to loading phase of ALS --- .../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 6c9fb2359cb..dec3701ec0d 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(',') From 931e4c96ef56302cc384086d1faa0f4ca3258e8b Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 14 Jul 2013 08:03:13 +0000 Subject: [PATCH 0353/2521] 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 dec3701ec0d..21eb21276ec 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 } From 00556a94c98577d1536d0c2b4606a9933069a584 Mon Sep 17 00:00:00 2001 From: BlackNiuza Date: Sun, 14 Jul 2013 17:04:53 +0800 Subject: [PATCH 0354/2521] add spaces before curly braces and after for if conditions --- .../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 68bb36d3166..6a0617cc063 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) } } From 4883586838b960c405af208262aed3ec4b078613 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 14 Jul 2013 10:37:26 -0700 Subject: [PATCH 0355/2521] Responding to Matei's review --- .../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 8a3b3304056..6b8b9f05bbd 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}
        From ed7fd501cf7ece730cbdee6c152b917cf6bfb16a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 15 Jul 2013 00:30:10 +0000 Subject: [PATCH 0356/2521] Make number of blocks in ALS configurable and lower the default --- .../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 21eb21276ec..2abaf2f2dd8 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(" ") } From d47c16f78d5cb935bd4022c9bed8376691371682 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 15 Jul 2013 01:55:54 +0000 Subject: [PATCH 0357/2521] 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 d723ab7b1ec..c7dbcc6fbca 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 5a805109593..5c06897cae4 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 From 4698a0d6886905ef21cbd52e108d0dcab3df12df Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 15 Jul 2013 02:54:11 +0000 Subject: [PATCH 0358/2521] Shuffle ratings in a more efficient way at start of ALS --- .../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 2abaf2f2dd8..4c18cbdc6bc 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 => From a1e56a43b3bc7ed66cbf5d5cee9d5bc12b029f0d Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 15 Jul 2013 11:28:18 +0530 Subject: [PATCH 0359/2521] Fixed compilation issues as Map is by default immutable.Map in scala-2.10 --- core/src/main/scala/spark/Utils.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 9817272b0e2..e40c5c5c8dc 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -12,10 +12,12 @@ import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.collection.JavaConversions._ +import scala.collection.Map import scala.io.Source import scala.reflect.ClassTag import scala.Some + import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder From 27250914290da28bf2b95253419ba94f57dd482c Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 15 Jul 2013 11:29:45 +0530 Subject: [PATCH 0360/2521] Fixed previously left out merge conflict in scripts --- run | 4 ---- 1 file changed, 4 deletions(-) diff --git a/run b/run index 46d9c61ba17..f32e719d161 100755 --- a/run +++ b/run @@ -131,9 +131,5 @@ else # The JVM doesn't read JAVA_OPTS by default so we need to pass it in EXTRA_ARGS="$JAVA_OPTS" fi -<<<<<<< HEAD -exec "$RUNNER" -cp "$CLASSPATH" $EXTRA_ARGS "$@" -======= exec "$RUNNER" -cp "$CLASSPATH" $EXTRA_ARGS "$@" ->>>>>>> 4698a0d6886905ef21cbd52e108d0dcab3df12df From 9d7781c4e1c6b2390aafe7069a1d3b1036a9a962 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 15 Jul 2013 12:03:48 +0530 Subject: [PATCH 0361/2521] Adding commons io as dependency --- project/SparkBuild.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d7cd337fa48..ffe9a92e94a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -110,7 +110,9 @@ object SparkBuild extends Build { "org.scalatest" %% "scalatest" % "1.9.1" % "test", "org.scalacheck" %% "scalacheck" % "1.10.0" % "test", "com.novocode" % "junit-interface" % "0.8" % "test", - "org.easymock" % "easymock" % "3.1" % "test" + "org.easymock" % "easymock" % "3.1" % "test", + "commons-io" % "commons-io" % "2.4" % "test" + ), parallelExecution := false, From b59152a7c3593d31f5ecef971046b8bcfa1f2b5d Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 15 Jul 2013 12:09:17 +0530 Subject: [PATCH 0362/2521] Changed to master version of the test, messed up during merge. --- core/src/test/scala/spark/ui/UISuite.scala | 48 ---------------------- 1 file changed, 48 deletions(-) diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index 60bb0240a56..fc0c1607202 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -4,12 +4,6 @@ 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} -import spark.Utils -import com.google.common.io.Files -import java.io.{FileOutputStream, File} -import com.google.common.base.Charsets class UISuite extends FunSuite { test("jetty port increases under contention") { @@ -32,46 +26,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") - - } } From b2aaa1199e7ecd8e1b2a9ddd8356b6393edafe6b Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 11:44:42 -0700 Subject: [PATCH 0363/2521] 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 36d9c472451..fa46e2487df 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} From 0d78b6d9cd11fc12c546f25fa857ba8b285c062d Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 13:42:12 -0700 Subject: [PATCH 0364/2521] 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 4dd6c448a97..87a7791fbd0 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) From 39557112501901da7f9b4be6159a5a0be5511b42 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 15:47:21 -0700 Subject: [PATCH 0365/2521] 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 7545ecf8685..434f600e8ec 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} From fbf5aa761e40649d0488a8673d488882c8bdae48 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 15:50:03 -0700 Subject: [PATCH 0366/2521] 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 87a7791fbd0..4dd6c448a97 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) From 6dc7c9bfb17cefdf162e86c8b52a4ffc8b59efaf Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 15 Jul 2013 16:33:50 -0700 Subject: [PATCH 0367/2521] 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 434f600e8ec..5e3c5e064fd 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} From a96b4ef761aa80310b194176f41a088c5bf6274a Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 15 Jul 2013 19:13:17 -0600 Subject: [PATCH 0368/2521] 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 feb13d52f9d..ef3d2529d08 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 From 90b0142985ca2c6c76ded1a5c073774308c1a727 Mon Sep 17 00:00:00 2001 From: seanm Date: Mon, 15 Jul 2013 19:13:39 -0600 Subject: [PATCH 0369/2521] adding files generated from make-distribution.sh to .gitignore --- .gitignore | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.gitignore b/.gitignore index ae39c52b113..00fbff6a2ca 100644 --- a/.gitignore +++ b/.gitignore @@ -38,3 +38,5 @@ dependency-reduced-pom.xml .ensime_lucene checkpoint derby.log +dist/ +spark-*-bin.tar.gz From 69316603d6bf11ecf1ea3dab63df178bad835e2d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 15 Jul 2013 22:50:11 -0700 Subject: [PATCH 0370/2521] 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 3d3b9ea0118..8173ef709dc 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 aa3ee5f5eea..7f7d4c82115 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) + } + } } From d733527bb4dad14b276b4f56b1ff5c3ee1cb7f75 Mon Sep 17 00:00:00 2001 From: Henry Saputra Date: Mon, 15 Jul 2013 23:45:57 -0700 Subject: [PATCH 0371/2521] Test commit karma for Spark git. --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index ba24ab43b1a..7cbe5a16b11 100644 --- a/README.md +++ b/README.md @@ -71,3 +71,4 @@ project's open source license. Whether or not you state this explicitly, by submitting any copyrighted material via pull request, email, or other means you agree to license the material under the project's open source license and warrant that you have the legal authority to do so. + From 63addd93a8337f5181c0e7d64c86393cb519c661 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Jul 2013 14:09:52 +0530 Subject: [PATCH 0372/2521] Fixed warning ClassManifest -> ClassTag --- core/src/main/scala/spark/PairRDDFunctions.scala | 2 +- core/src/main/scala/spark/RDD.scala | 6 +++--- .../scala/spark/api/java/function/FlatMapFunction2.scala | 4 +++- core/src/main/scala/spark/rdd/EmptyRDD.scala | 4 ++-- core/src/main/scala/spark/rdd/JdbcRDD.scala | 3 ++- core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala | 9 +++++---- .../spark/streaming/dstream/KafkaInputDStream.scala | 2 +- 7 files changed, 17 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 0095b868a83..5cf162f23eb 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -536,7 +536,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag]( * supplied codec. */ def saveAsHadoopFile[F <: OutputFormat[K, V]]( - path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassManifest[F]) { + path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassTag[F]) { saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]], codec) } diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 3a454df10d7..2ebfaadc464 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -500,18 +500,18 @@ abstract class RDD[T: ClassTag]( * *same number of partitions*, but does *not* require them to have the same number * of elements in each partition. */ - def zipPartitions[B: ClassManifest, V: ClassManifest]( + def zipPartitions[B: ClassTag, V: ClassTag]( f: (Iterator[T], Iterator[B]) => Iterator[V], rdd2: RDD[B]): RDD[V] = new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2) - def zipPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest]( + def zipPartitions[B: ClassTag, C: ClassTag, V: ClassTag]( f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V], rdd2: RDD[B], rdd3: RDD[C]): RDD[V] = new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3) - def zipPartitions[B: ClassManifest, C: ClassManifest, D: ClassManifest, V: ClassManifest]( + def zipPartitions[B: ClassTag, C: ClassTag, D: ClassTag, V: ClassTag]( f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], rdd2: RDD[B], rdd3: RDD[C], 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 6044043add3..4562884eb3a 100644 --- a/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala +++ b/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala @@ -1,5 +1,7 @@ package spark.api.java.function +import scala.reflect.ClassTag + /** * A function that takes two inputs and returns zero or more output records. */ @@ -7,5 +9,5 @@ abstract class FlatMapFunction2[A, B, C] extends Function2[A, B, java.lang.Itera @throws(classOf[Exception]) def call(a: A, b:B) : java.lang.Iterable[C] - def elementType() : ClassManifest[C] = ClassManifest.Any.asInstanceOf[ClassManifest[C]] + def elementType() : ClassTag[C] = ClassTag.Any.asInstanceOf[ClassTag[C]] } diff --git a/core/src/main/scala/spark/rdd/EmptyRDD.scala b/core/src/main/scala/spark/rdd/EmptyRDD.scala index e4dd3a7fa7d..3b9899238c7 100644 --- a/core/src/main/scala/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/spark/rdd/EmptyRDD.scala @@ -1,12 +1,12 @@ package spark.rdd import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} - +import scala.reflect.ClassTag /** * An RDD that is empty, i.e. has no element in it. */ -class EmptyRDD[T: ClassManifest](sc: SparkContext) extends RDD[T](sc, Nil) { +class EmptyRDD[T: ClassTag](sc: SparkContext) extends RDD[T](sc, Nil) { override def getPartitions: Array[Partition] = Array.empty diff --git a/core/src/main/scala/spark/rdd/JdbcRDD.scala b/core/src/main/scala/spark/rdd/JdbcRDD.scala index a50f4077373..f570dd6d8b5 100644 --- a/core/src/main/scala/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/spark/rdd/JdbcRDD.scala @@ -4,6 +4,7 @@ import java.sql.{Connection, ResultSet} import spark.{Logging, Partition, RDD, SparkContext, TaskContext} import spark.util.NextIterator +import scala.reflect.ClassTag private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { override def index = idx @@ -28,7 +29,7 @@ private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) e * 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]( +class JdbcRDD[T: ClassTag]( sc: SparkContext, getConnection: () => Connection, sql: String, diff --git a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala index b234428ab2b..d3c23708856 100644 --- a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala @@ -2,6 +2,7 @@ package spark.rdd import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} +import scala.reflect.ClassTag private[spark] class ZippedPartitionsPartition( idx: Int, @@ -20,7 +21,7 @@ private[spark] class ZippedPartitionsPartition( } } -abstract class ZippedPartitionsBaseRDD[V: ClassManifest]( +abstract class ZippedPartitionsBaseRDD[V: ClassTag]( sc: SparkContext, var rdds: Seq[RDD[_]]) extends RDD[V](sc, rdds.map(x => new OneToOneDependency(x))) { @@ -67,7 +68,7 @@ abstract class ZippedPartitionsBaseRDD[V: ClassManifest]( } } -class ZippedPartitionsRDD2[A: ClassManifest, B: ClassManifest, V: ClassManifest]( +class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag]( sc: SparkContext, f: (Iterator[A], Iterator[B]) => Iterator[V], var rdd1: RDD[A], @@ -87,7 +88,7 @@ class ZippedPartitionsRDD2[A: ClassManifest, B: ClassManifest, V: ClassManifest] } class ZippedPartitionsRDD3 - [A: ClassManifest, B: ClassManifest, C: ClassManifest, V: ClassManifest]( + [A: ClassTag, B: ClassTag, C: ClassTag, V: ClassTag]( sc: SparkContext, f: (Iterator[A], Iterator[B], Iterator[C]) => Iterator[V], var rdd1: RDD[A], @@ -111,7 +112,7 @@ class ZippedPartitionsRDD3 } class ZippedPartitionsRDD4 - [A: ClassManifest, B: ClassManifest, C: ClassManifest, D:ClassManifest, V: ClassManifest]( + [A: ClassTag, B: ClassTag, C: ClassTag, D:ClassTag, V: ClassTag]( sc: SparkContext, f: (Iterator[A], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], var rdd1: RDD[A], diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index e0f6351ef75..ba1312cbe8a 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -95,7 +95,7 @@ class KafkaReceiver[T: ClassTag, D <: Decoder[_]: Manifest]( } // Handles Kafka Messages - private class MessageHandler[T: ClassManifest](stream: KafkaStream[T]) extends Runnable { + private class MessageHandler[T: ClassTag](stream: KafkaStream[T]) extends Runnable { def run() { logInfo("Starting MessageHandler.") for (msgAndMetadata <- stream) { From ff14f38f3dce314eb1afb876e79cbab6b1c252b8 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Jul 2013 14:34:56 +0530 Subject: [PATCH 0373/2521] Fixed warning Throwables --- .../src/main/scala/spark/api/python/PythonWorkerFactory.scala | 4 ++-- .../scala/spark/scheduler/cluster/ClusterTaskSetManager.scala | 2 +- .../scala/spark/streaming/dstream/KafkaInputDStream.scala | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala index 85d1dfeac8b..7163f01b245 100644 --- a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala @@ -28,7 +28,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String startDaemon() new Socket(daemonHost, daemonPort) } - case e => throw e + case e : Throwable => throw e } } } @@ -88,7 +88,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } }.start() } catch { - case e => { + case e :Throwable => { stopDaemon() throw e } diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 327d6797ae2..56374be35a3 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -524,7 +524,7 @@ private[spark] class ClusterTaskSetManager( case cnf: ClassNotFoundException => val loader = Thread.currentThread().getContextClassLoader throw new SparkException("ClassNotFound with classloader: " + loader, cnf) - case ex => throw ex + case ex : Throwable => throw ex } // Mark finished and stop if we've finished all the tasks finished(index) = true diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index ba1312cbe8a..8b23642e4a1 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -118,7 +118,7 @@ class KafkaReceiver[T: ClassTag, D <: Decoder[_]: Manifest]( zk.deleteRecursive(dir) zk.close() } catch { - case _ => // swallow + case _ : Throwable => // swallow } } } From 55da6e9504c533b6eaced215c9d4a6199062f799 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Jul 2013 14:36:36 +0530 Subject: [PATCH 0374/2521] Fixed warning erasure -> runtimeClass --- core/src/main/scala/spark/PairRDDFunctions.scala | 2 +- .../main/scala/spark/streaming/dstream/KafkaInputDStream.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 5cf162f23eb..ee09c2085b8 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -537,7 +537,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag]( */ def saveAsHadoopFile[F <: OutputFormat[K, V]]( path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassTag[F]) { - saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]], codec) + saveAsHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]], codec) } /** diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala index 8b23642e4a1..47274f41a56 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala @@ -85,7 +85,7 @@ class KafkaReceiver[T: ClassTag, D <: Decoder[_]: Manifest]( } // Create Threads for each Topic/Message Stream we are listening - val decoder = manifest[D].erasure.newInstance.asInstanceOf[Decoder[T]] + val decoder = manifest[D].runtimeClass.newInstance.asInstanceOf[Decoder[T]] val topicMessageStreams = consumerConnector.createMessageStreams(topics, decoder) // Start the messages handler for each partition From 50f3cd8890514f878a10b6b18cd4e38448d5d475 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Jul 2013 14:39:46 +0530 Subject: [PATCH 0375/2521] Fixed warning enumerations --- .../spark/scheduler/cluster/ClusterTaskSetManager.scala | 6 +++--- .../main/scala/spark/scheduler/cluster/SchedulingMode.scala | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 56374be35a3..9d618cf5313 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -13,13 +13,13 @@ 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 { +private[spark] object TaskLocality extends Enumeration with Logging { + + type TaskLocality = 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. diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala index 6e0c6793e01..1c1f17ebf26 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala @@ -1,6 +1,6 @@ package spark.scheduler.cluster -object SchedulingMode extends Enumeration("FAIR","FIFO"){ +object SchedulingMode extends Enumeration{ type SchedulingMode = Value val FAIR,FIFO = Value From f89cc7ae3c475b3ffdadbe1daccc26265f0bca2a Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Jul 2013 14:59:24 +0530 Subject: [PATCH 0376/2521] Fixed warning for type erasure --- core/src/main/scala/spark/api/python/PythonRDD.scala | 2 +- 1 file changed, 1 insertion(+), 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 3d1e45cb2c7..d15c75289dd 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -195,7 +195,7 @@ private[spark] object PythonRDD { val arr = elem.asInstanceOf[Array[Byte]] dOut.writeInt(arr.length) dOut.write(arr) - } else if (elem.isInstanceOf[scala.Tuple2[Array[Byte], Array[Byte]]]) { + } else if (elem.isInstanceOf[scala.Tuple2[_, _]]) { val t = elem.asInstanceOf[scala.Tuple2[Array[Byte], Array[Byte]]] val length = t._1.length + t._2.length - 3 - 3 + 4 // stripPickle() removes 3 bytes dOut.writeInt(length) From 119c98c1becf9f0038d6bb946545cd65006bd367 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Jul 2013 15:01:33 +0530 Subject: [PATCH 0377/2521] code formatting, The warning related to scope exit and enter is not worth fixing as it only affects debugging scopes and nothing else. --- .../streaming/util/MasterFailureTest.scala | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala index b3daa5a91b8..ca654c45c96 100644 --- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala @@ -363,20 +363,20 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) val tempHadoopFile = new Path(testDir, ".tmp_" + (i+1).toString) FileUtils.writeStringToFile(localFile, input(i).toString + "\n") var tries = 0 - var done = false - while (!done && tries < maxTries) { - tries += 1 - try { - // fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile) - fs.copyFromLocalFile(new Path(localFile.toString), tempHadoopFile) - fs.rename(tempHadoopFile, hadoopFile) - done = true - } catch { - case ioe: IOException => { - fs = testDir.getFileSystem(new Configuration()) - logWarning("Attempt " + tries + " at generating file " + hadoopFile + " failed.", ioe) - } - } + var done = false + while (!done && tries < maxTries) { + tries += 1 + try { + // fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile) + fs.copyFromLocalFile(new Path(localFile.toString), tempHadoopFile) + fs.rename(tempHadoopFile, hadoopFile) + done = true + } catch { + case ioe: IOException => { + fs = testDir.getFileSystem(new Configuration()) + logWarning("Attempt " + tries + " at generating file " + hadoopFile + " failed.", ioe) + } + } } if (!done) logError("Could not generate file " + hadoopFile) From 2748e73eb9bb3f08b116c58ea404a6e76f9cbff9 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Jul 2013 16:08:46 +0530 Subject: [PATCH 0378/2521] 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 178a056de83..2510a614e8c 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 From f347cc3f659d4414a21de26feadcbe23a130e622 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 10:53:30 -0700 Subject: [PATCH 0379/2521] 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 9c24ca430da..c7855a7bd34 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))) } From 5c388808a81ebfb1fc23511882c18f9ae76ec509 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 11:01:14 -0700 Subject: [PATCH 0380/2521] SPARK-814: Result stages should be named after action --- .../scala/spark/scheduler/DAGScheduler.scala | 22 ++++++++++++------- .../main/scala/spark/scheduler/Stage.scala | 5 +++-- .../main/scala/spark/ui/jobs/IndexPage.scala | 2 +- .../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 3d3b9ea0118..30a648f50b4 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 539cf8233bf..4937eb3b882 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 7907ab3bc70..2df5f0192bf 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 699901f1a13..2474d744b80 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) From 00a14deb6d9b6d89e0ddbe1540636622b1dc4d16 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 11:52:20 -0700 Subject: [PATCH 0381/2521] 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 63dd80b5fea..6fa1f1d4f10 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} From af3c9d50424602f3e5af1055e83e9badef0a1632 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 17:21:33 -0700 Subject: [PATCH 0382/2521] 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 ++ .../spark/bagel/examples/PageRankUtils.scala | 17 ++ .../bagel/examples/WikipediaPageRank.scala | 17 ++ .../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 ++ .../hadoop/mapred/HadoopMapRedUtil.scala | 17 ++ .../mapreduce/HadoopMapReduceUtil.scala | 17 ++ .../scala/spark/deploy/SparkHadoopUtil.scala | 17 ++ .../hadoop/mapred/HadoopMapRedUtil.scala | 17 ++ .../mapreduce/HadoopMapReduceUtil.scala | 17 ++ .../scala/spark/deploy/SparkHadoopUtil.scala | 17 ++ .../spark/deploy/yarn/ApplicationMaster.scala | 17 ++ .../yarn/ApplicationMasterArguments.scala | 17 ++ .../scala/spark/deploy/yarn/Client.scala | 17 ++ .../spark/deploy/yarn/ClientArguments.scala | 17 ++ .../spark/deploy/yarn/WorkerRunnable.scala | 17 ++ .../deploy/yarn/YarnAllocationHandler.scala | 17 ++ .../cluster/YarnClusterScheduler.scala | 17 ++ .../hadoop/mapred/HadoopMapRedUtil.scala | 17 ++ .../mapreduce/HadoopMapReduceUtil.scala | 17 ++ .../scala/spark/deploy/SparkHadoopUtil.scala | 17 ++ .../java/spark/network/netty/FileClient.java | 17 ++ .../netty/FileClientChannelInitializer.java | 17 ++ .../network/netty/FileClientHandler.java | 17 ++ .../java/spark/network/netty/FileServer.java | 17 ++ .../netty/FileServerChannelInitializer.java | 17 ++ .../network/netty/FileServerHandler.java | 17 ++ .../spark/network/netty/PathResolver.java | 17 ++ core/src/main/scala/spark/Accumulators.scala | 17 ++ core/src/main/scala/spark/Aggregator.scala | 17 ++ .../spark/BlockStoreShuffleFetcher.scala | 17 ++ core/src/main/scala/spark/Cache.scala | 17 ++ core/src/main/scala/spark/CacheManager.scala | 17 ++ .../src/main/scala/spark/ClosureCleaner.scala | 17 ++ core/src/main/scala/spark/Dependency.scala | 17 ++ .../main/scala/spark/DoubleRDDFunctions.scala | 17 ++ .../scala/spark/FetchFailedException.scala | 17 ++ core/src/main/scala/spark/HadoopWriter.scala | 17 ++ .../src/main/scala/spark/HttpFileServer.scala | 19 +- core/src/main/scala/spark/HttpServer.scala | 17 ++ .../src/main/scala/spark/JavaSerializer.scala | 17 ++ .../src/main/scala/spark/KryoSerializer.scala | 17 ++ core/src/main/scala/spark/Logging.scala | 17 ++ .../main/scala/spark/MapOutputTracker.scala | 17 ++ .../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 ++ .../main/scala/spark/RDDCheckpointData.scala | 17 ++ .../spark/SequenceFileRDDFunctions.scala | 17 ++ .../scala/spark/SerializableWritable.scala | 17 ++ .../src/main/scala/spark/ShuffleFetcher.scala | 17 ++ core/src/main/scala/spark/SizeEstimator.scala | 17 ++ .../main/scala/spark/SoftReferenceCache.scala | 17 ++ core/src/main/scala/spark/SparkContext.scala | 17 ++ core/src/main/scala/spark/SparkEnv.scala | 17 ++ .../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 ++ .../scala/spark/api/java/JavaDoubleRDD.scala | 17 ++ .../scala/spark/api/java/JavaPairRDD.scala | 17 ++ .../main/scala/spark/api/java/JavaRDD.scala | 17 ++ .../scala/spark/api/java/JavaRDDLike.scala | 17 ++ .../spark/api/java/JavaSparkContext.scala | 17 ++ .../JavaSparkContextVarargsWorkaround.java | 17 ++ .../scala/spark/api/java/StorageLevels.java | 17 ++ .../java/function/DoubleFlatMapFunction.java | 17 ++ .../api/java/function/DoubleFunction.java | 17 ++ .../api/java/function/FlatMapFunction.scala | 17 ++ .../api/java/function/FlatMapFunction2.scala | 17 ++ .../spark/api/java/function/Function.java | 17 ++ .../spark/api/java/function/Function2.java | 17 ++ .../java/function/PairFlatMapFunction.java | 17 ++ .../spark/api/java/function/PairFunction.java | 17 ++ .../api/java/function/VoidFunction.scala | 19 +- .../api/java/function/WrappedFunction1.scala | 17 ++ .../api/java/function/WrappedFunction2.scala | 17 ++ .../spark/api/python/PythonPartitioner.scala | 17 ++ .../scala/spark/api/python/PythonRDD.scala | 17 ++ .../api/python/PythonWorkerFactory.scala | 17 ++ .../spark/broadcast/BitTorrentBroadcast.scala | 17 ++ .../scala/spark/broadcast/Broadcast.scala | 17 ++ .../spark/broadcast/BroadcastFactory.scala | 17 ++ .../scala/spark/broadcast/HttpBroadcast.scala | 17 ++ .../scala/spark/broadcast/MultiTracker.scala | 17 ++ .../scala/spark/broadcast/SourceInfo.scala | 17 ++ .../scala/spark/broadcast/TreeBroadcast.scala | 17 ++ .../spark/deploy/ApplicationDescription.scala | 17 ++ .../src/main/scala/spark/deploy/Command.scala | 17 ++ .../scala/spark/deploy/DeployMessage.scala | 17 ++ .../scala/spark/deploy/ExecutorState.scala | 17 ++ .../scala/spark/deploy/JsonProtocol.scala | 19 +- .../spark/deploy/LocalSparkCluster.scala | 17 ++ core/src/main/scala/spark/deploy/WebUI.scala | 17 ++ .../scala/spark/deploy/client/Client.scala | 17 ++ .../spark/deploy/client/ClientListener.scala | 17 ++ .../spark/deploy/client/TestClient.scala | 17 ++ .../spark/deploy/client/TestExecutor.scala | 17 ++ .../spark/deploy/master/ApplicationInfo.scala | 17 ++ .../deploy/master/ApplicationState.scala | 17 ++ .../spark/deploy/master/ExecutorInfo.scala | 17 ++ .../scala/spark/deploy/master/Master.scala | 17 ++ .../spark/deploy/master/MasterArguments.scala | 17 ++ .../spark/deploy/master/WorkerInfo.scala | 17 ++ .../spark/deploy/master/WorkerState.scala | 17 ++ .../deploy/master/ui/ApplicationPage.scala | 17 ++ .../spark/deploy/master/ui/IndexPage.scala | 17 ++ .../spark/deploy/master/ui/MasterWebUI.scala | 19 +- .../spark/deploy/worker/ExecutorRunner.scala | 17 ++ .../scala/spark/deploy/worker/Worker.scala | 17 ++ .../spark/deploy/worker/WorkerArguments.scala | 17 ++ .../spark/deploy/worker/ui/IndexPage.scala | 17 ++ .../spark/deploy/worker/ui/WorkerWebUI.scala | 17 ++ .../main/scala/spark/executor/Executor.scala | 17 ++ .../spark/executor/ExecutorBackend.scala | 17 ++ .../spark/executor/ExecutorExitCode.scala | 17 ++ .../executor/ExecutorURLClassLoader.scala | 17 ++ .../spark/executor/MesosExecutorBackend.scala | 17 ++ .../executor/StandaloneExecutorBackend.scala | 17 ++ .../scala/spark/executor/TaskMetrics.scala | 17 ++ .../scala/spark/network/BufferMessage.scala | 19 +- .../main/scala/spark/network/Connection.scala | 17 ++ .../spark/network/ConnectionManager.scala | 17 ++ .../spark/network/ConnectionManagerId.scala | 17 ++ .../spark/network/ConnectionManagerTest.scala | 17 ++ .../main/scala/spark/network/Message.scala | 17 ++ .../scala/spark/network/MessageChunk.scala | 17 ++ .../spark/network/MessageChunkHeader.scala | 17 ++ .../scala/spark/network/ReceiverTest.scala | 17 ++ .../main/scala/spark/network/SenderTest.scala | 17 ++ .../spark/network/netty/FileHeader.scala | 17 ++ .../spark/network/netty/ShuffleCopier.scala | 17 ++ .../spark/network/netty/ShuffleSender.scala | 17 ++ core/src/main/scala/spark/package.scala | 17 ++ .../partial/ApproximateActionListener.scala | 17 ++ .../spark/partial/ApproximateEvaluator.scala | 17 ++ .../scala/spark/partial/BoundedDouble.scala | 17 ++ .../scala/spark/partial/CountEvaluator.scala | 17 ++ .../spark/partial/GroupedCountEvaluator.scala | 17 ++ .../spark/partial/GroupedMeanEvaluator.scala | 17 ++ .../spark/partial/GroupedSumEvaluator.scala | 17 ++ .../scala/spark/partial/MeanEvaluator.scala | 17 ++ .../scala/spark/partial/PartialResult.scala | 17 ++ .../scala/spark/partial/StudentTCacher.scala | 17 ++ .../scala/spark/partial/SumEvaluator.scala | 17 ++ core/src/main/scala/spark/rdd/BlockRDD.scala | 17 ++ .../main/scala/spark/rdd/CartesianRDD.scala | 17 ++ .../main/scala/spark/rdd/CheckpointRDD.scala | 17 ++ .../main/scala/spark/rdd/CoGroupedRDD.scala | 17 ++ .../main/scala/spark/rdd/CoalescedRDD.scala | 17 ++ core/src/main/scala/spark/rdd/EmptyRDD.scala | 17 ++ .../main/scala/spark/rdd/FilteredRDD.scala | 17 ++ .../main/scala/spark/rdd/FlatMappedRDD.scala | 17 ++ .../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 ++ .../scala/spark/rdd/MapPartitionsRDD.scala | 17 ++ .../spark/rdd/MapPartitionsWithIndexRDD.scala | 17 ++ core/src/main/scala/spark/rdd/MappedRDD.scala | 17 ++ .../main/scala/spark/rdd/NewHadoopRDD.scala | 17 ++ .../spark/rdd/ParallelCollectionRDD.scala | 17 ++ .../scala/spark/rdd/PartitionPruningRDD.scala | 17 ++ core/src/main/scala/spark/rdd/PipedRDD.scala | 17 ++ .../src/main/scala/spark/rdd/SampledRDD.scala | 17 ++ .../main/scala/spark/rdd/ShuffledRDD.scala | 17 ++ .../main/scala/spark/rdd/SubtractedRDD.scala | 17 ++ core/src/main/scala/spark/rdd/UnionRDD.scala | 17 ++ .../scala/spark/rdd/ZippedPartitionsRDD.scala | 17 ++ core/src/main/scala/spark/rdd/ZippedRDD.scala | 17 ++ .../scala/spark/scheduler/ActiveJob.scala | 17 ++ .../scala/spark/scheduler/DAGScheduler.scala | 17 ++ .../spark/scheduler/DAGSchedulerEvent.scala | 17 ++ .../spark/scheduler/InputFormatInfo.scala | 17 ++ .../scala/spark/scheduler/JobListener.scala | 17 ++ .../scala/spark/scheduler/JobLogger.scala | 17 ++ .../scala/spark/scheduler/JobResult.scala | 17 ++ .../scala/spark/scheduler/JobWaiter.scala | 17 ++ .../scala/spark/scheduler/MapStatus.scala | 17 ++ .../scala/spark/scheduler/ResultTask.scala | 17 ++ .../spark/scheduler/ShuffleMapTask.scala | 17 ++ .../scala/spark/scheduler/SparkListener.scala | 17 ++ .../scala/spark/scheduler/SplitInfo.scala | 17 ++ .../main/scala/spark/scheduler/Stage.scala | 17 ++ .../scala/spark/scheduler/StageInfo.scala | 19 +- .../src/main/scala/spark/scheduler/Task.scala | 17 ++ .../scala/spark/scheduler/TaskResult.scala | 17 ++ .../scala/spark/scheduler/TaskScheduler.scala | 17 ++ .../scheduler/TaskSchedulerListener.scala | 17 ++ .../main/scala/spark/scheduler/TaskSet.scala | 17 ++ .../scheduler/cluster/ClusterScheduler.scala | 17 ++ .../cluster/ClusterTaskSetManager.scala | 17 ++ .../cluster/ExecutorLossReason.scala | 17 ++ .../scala/spark/scheduler/cluster/Pool.scala | 17 ++ .../spark/scheduler/cluster/Schedulable.scala | 17 ++ .../cluster/SchedulableBuilder.scala | 17 ++ .../scheduler/cluster/SchedulerBackend.scala | 17 ++ .../cluster/SchedulingAlgorithm.scala | 17 ++ .../scheduler/cluster/SchedulingMode.scala | 17 ++ .../cluster/SparkDeploySchedulerBackend.scala | 17 ++ .../cluster/StandaloneClusterMessage.scala | 17 ++ .../cluster/StandaloneSchedulerBackend.scala | 17 ++ .../scheduler/cluster/TaskDescription.scala | 17 ++ .../spark/scheduler/cluster/TaskInfo.scala | 17 ++ .../scheduler/cluster/TaskSetManager.scala | 17 ++ .../spark/scheduler/cluster/WorkerOffer.scala | 17 ++ .../scheduler/local/LocalScheduler.scala | 17 ++ .../scheduler/local/LocalTaskSetManager.scala | 17 ++ .../mesos/CoarseMesosSchedulerBackend.scala | 17 ++ .../mesos/MesosSchedulerBackend.scala | 17 ++ .../scala/spark/serializer/Serializer.scala | 17 ++ .../spark/serializer/SerializerManager.scala | 17 ++ .../scala/spark/storage/BlockException.scala | 17 ++ .../spark/storage/BlockFetchTracker.scala | 17 ++ .../spark/storage/BlockFetcherIterator.scala | 17 ++ .../scala/spark/storage/BlockManager.scala | 17 ++ .../scala/spark/storage/BlockManagerId.scala | 17 ++ .../spark/storage/BlockManagerMaster.scala | 17 ++ .../storage/BlockManagerMasterActor.scala | 17 ++ .../spark/storage/BlockManagerMessages.scala | 17 ++ .../storage/BlockManagerSlaveActor.scala | 17 ++ .../spark/storage/BlockManagerWorker.scala | 17 ++ .../scala/spark/storage/BlockMessage.scala | 17 ++ .../spark/storage/BlockMessageArray.scala | 17 ++ .../spark/storage/BlockObjectWriter.scala | 17 ++ .../main/scala/spark/storage/BlockStore.scala | 17 ++ .../main/scala/spark/storage/DiskStore.scala | 17 ++ .../scala/spark/storage/MemoryStore.scala | 17 ++ .../main/scala/spark/storage/PutResult.scala | 17 ++ .../spark/storage/ShuffleBlockManager.scala | 17 ++ .../scala/spark/storage/StorageLevel.scala | 17 ++ .../scala/spark/storage/StorageUtils.scala | 17 ++ .../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 +- .../scala/spark/ui/UIWorkloadGenerator.scala | 17 ++ .../scala/spark/ui/env/EnvironmentUI.scala | 17 ++ .../main/scala/spark/ui/jobs/IndexPage.scala | 17 ++ .../scala/spark/ui/jobs/JobProgressUI.scala | 19 +- .../main/scala/spark/ui/jobs/StagePage.scala | 17 ++ .../spark/ui/storage/BlockManagerUI.scala | 17 ++ .../scala/spark/ui/storage/IndexPage.scala | 17 ++ .../main/scala/spark/ui/storage/RDDPage.scala | 17 ++ .../src/main/scala/spark/util/AkkaUtils.scala | 17 ++ .../spark/util/BoundedPriorityQueue.scala | 17 ++ .../spark/util/ByteBufferInputStream.scala | 17 ++ .../scala/spark/util/CompletionIterator.scala | 19 +- .../main/scala/spark/util/Distribution.scala | 19 +- .../main/scala/spark/util/IdGenerator.scala | 17 ++ core/src/main/scala/spark/util/IntParam.scala | 17 ++ .../main/scala/spark/util/MemoryParam.scala | 17 ++ .../scala/spark/util/MetadataCleaner.scala | 17 ++ .../main/scala/spark/util/NextIterator.scala | 19 +- .../spark/util/RateLimitedOutputStream.scala | 17 ++ .../scala/spark/util/SerializableBuffer.scala | 17 ++ .../main/scala/spark/util/StatCounter.scala | 17 ++ .../scala/spark/util/TimeStampedHashMap.scala | 17 ++ .../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 +- .../test/scala/spark/AccumulatorSuite.scala | 17 ++ .../src/test/scala/spark/BroadcastSuite.scala | 17 ++ .../test/scala/spark/CheckpointSuite.scala | 17 ++ .../scala/spark/ClosureCleanerSuite.scala | 17 ++ .../test/scala/spark/DistributedSuite.scala | 17 ++ core/src/test/scala/spark/DriverSuite.scala | 17 ++ core/src/test/scala/spark/FailureSuite.scala | 17 ++ .../test/scala/spark/FileServerSuite.scala | 17 ++ core/src/test/scala/spark/FileSuite.scala | 17 ++ core/src/test/scala/spark/JavaAPISuite.java | 17 ++ .../scala/spark/KryoSerializerSuite.scala | 17 ++ .../test/scala/spark/LocalSparkContext.scala | 17 ++ .../scala/spark/MapOutputTrackerSuite.scala | 17 ++ .../scala/spark/PairRDDFunctionsSuite.scala | 17 ++ .../test/scala/spark/PartitioningSuite.scala | 17 ++ core/src/test/scala/spark/PipedRDDSuite.scala | 17 ++ core/src/test/scala/spark/RDDSuite.scala | 17 ++ .../test/scala/spark/SharedSparkContext.scala | 17 ++ .../test/scala/spark/ShuffleNettySuite.scala | 17 ++ core/src/test/scala/spark/ShuffleSuite.scala | 17 ++ .../test/scala/spark/SizeEstimatorSuite.scala | 17 ++ core/src/test/scala/spark/SortingSuite.scala | 17 ++ .../src/test/scala/spark/ThreadingSuite.scala | 17 ++ .../src/test/scala/spark/UnpersistSuite.scala | 17 ++ core/src/test/scala/spark/UtilsSuite.scala | 17 ++ .../scala/spark/ZippedPartitionsSuite.scala | 17 ++ .../test/scala/spark/rdd/JdbcRDDSuite.scala | 17 ++ .../rdd/ParallelCollectionSplitSuite.scala | 17 ++ .../scheduler/ClusterSchedulerSuite.scala | 17 ++ .../spark/scheduler/DAGSchedulerSuite.scala | 17 ++ .../spark/scheduler/JobLoggerSuite.scala | 17 ++ .../spark/scheduler/LocalSchedulerSuite.scala | 17 ++ .../spark/scheduler/SparkListenerSuite.scala | 17 ++ .../spark/scheduler/TaskContextSuite.scala | 17 ++ .../spark/storage/BlockManagerSuite.scala | 17 ++ core/src/test/scala/spark/ui/UISuite.scala | 17 ++ .../scala/spark/util/DistributionSuite.scala | 17 ++ .../scala/spark/util/NextIteratorSuite.scala | 17 ++ .../util/RateLimitedOutputStreamSuite.scala | 17 ++ docs/_plugins/copy_api_dirs.rb | 17 ++ ec2/spark-ec2 | 2 + ec2/spark_ec2.py | 2 + examples/pom.xml | 17 ++ .../main/java/spark/examples/JavaHdfsLR.java | 17 ++ .../main/java/spark/examples/JavaKMeans.java | 17 ++ .../java/spark/examples/JavaLogQuery.java | 17 ++ .../main/java/spark/examples/JavaSparkPi.java | 17 ++ .../src/main/java/spark/examples/JavaTC.java | 17 ++ .../java/spark/examples/JavaWordCount.java | 17 ++ .../examples/JavaFlumeEventCount.java | 17 ++ .../examples/JavaNetworkWordCount.java | 17 ++ .../streaming/examples/JavaQueueStream.java | 17 ++ .../scala/spark/examples/BroadcastTest.scala | 17 ++ .../scala/spark/examples/CassandraTest.scala | 17 ++ .../examples/ExceptionHandlingTest.scala | 17 ++ .../scala/spark/examples/GroupByTest.scala | 17 ++ .../main/scala/spark/examples/HBaseTest.scala | 19 +- .../main/scala/spark/examples/HdfsTest.scala | 17 ++ .../main/scala/spark/examples/LocalALS.scala | 17 ++ .../scala/spark/examples/LocalFileLR.scala | 17 ++ .../scala/spark/examples/LocalKMeans.scala | 17 ++ .../main/scala/spark/examples/LocalLR.scala | 17 ++ .../main/scala/spark/examples/LocalPi.scala | 17 ++ .../main/scala/spark/examples/LogQuery.scala | 17 ++ .../spark/examples/MultiBroadcastTest.scala | 17 ++ .../examples/SimpleSkewedGroupByTest.scala | 17 ++ .../spark/examples/SkewedGroupByTest.scala | 17 ++ .../main/scala/spark/examples/SparkALS.scala | 17 ++ .../scala/spark/examples/SparkHdfsLR.scala | 17 ++ .../scala/spark/examples/SparkKMeans.scala | 17 ++ .../main/scala/spark/examples/SparkLR.scala | 17 ++ .../main/scala/spark/examples/SparkPi.scala | 17 ++ .../main/scala/spark/examples/SparkTC.scala | 17 ++ .../streaming/examples/ActorWordCount.scala | 17 ++ .../streaming/examples/FlumeEventCount.scala | 17 ++ .../streaming/examples/HdfsWordCount.scala | 17 ++ .../streaming/examples/KafkaWordCount.scala | 17 ++ .../streaming/examples/NetworkWordCount.scala | 17 ++ .../streaming/examples/QueueStream.scala | 17 ++ .../streaming/examples/RawNetworkGrep.scala | 17 ++ .../examples/StatefulNetworkWordCount.scala | 17 ++ .../examples/TwitterAlgebirdCMS.scala | 17 ++ .../examples/TwitterAlgebirdHLL.scala | 17 ++ .../examples/TwitterPopularTags.scala | 17 ++ .../streaming/examples/ZeroMQWordCount.scala | 17 ++ .../clickstream/PageViewGenerator.scala | 17 ++ .../examples/clickstream/PageViewStream.scala | 17 ++ make-distribution.sh | 19 ++ .../scala/spark/mllib/clustering/KMeans.scala | 17 ++ .../spark/mllib/clustering/KMeansModel.scala | 17 ++ .../spark/mllib/clustering/LocalKMeans.scala | 17 ++ .../spark/mllib/optimization/Gradient.scala | 17 ++ .../mllib/optimization/GradientDescent.scala | 17 ++ .../spark/mllib/optimization/Updater.scala | 17 ++ .../spark/mllib/recommendation/ALS.scala | 17 ++ .../MatrixFactorizationModel.scala | 17 ++ .../mllib/regression/LogisticRegression.scala | 17 ++ .../LogisticRegressionGenerator.scala | 17 ++ .../spark/mllib/regression/Regression.scala | 17 ++ .../mllib/regression/RidgeRegression.scala | 17 ++ .../regression/RidgeRegressionGenerator.scala | 17 ++ .../main/scala/spark/mllib/util/MLUtils.scala | 17 ++ mllib/src/test/resources/log4j.properties | 19 +- .../spark/mllib/clustering/KMeansSuite.scala | 17 ++ .../spark/mllib/recommendation/ALSSuite.scala | 17 ++ .../regression/LogisticRegressionSuite.scala | 17 ++ .../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 ++ .../spark/repl/ExecutorClassLoader.scala | 17 ++ repl/src/main/scala/spark/repl/Main.scala | 17 ++ repl/src/test/resources/log4j.properties | 19 +- .../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 ++ .../scala/spark/streaming/Checkpoint.scala | 17 ++ .../main/scala/spark/streaming/DStream.scala | 17 ++ .../streaming/DStreamCheckpointData.scala | 17 ++ .../scala/spark/streaming/DStreamGraph.scala | 17 ++ .../main/scala/spark/streaming/Duration.scala | 17 ++ .../main/scala/spark/streaming/Interval.scala | 17 ++ .../src/main/scala/spark/streaming/Job.scala | 17 ++ .../scala/spark/streaming/JobManager.scala | 17 ++ .../spark/streaming/NetworkInputTracker.scala | 17 ++ .../streaming/PairDStreamFunctions.scala | 17 ++ .../scala/spark/streaming/Scheduler.scala | 17 ++ .../spark/streaming/StreamingContext.scala | 17 ++ .../src/main/scala/spark/streaming/Time.scala | 19 +- .../streaming/api/java/JavaDStream.scala | 19 +- .../streaming/api/java/JavaDStreamLike.scala | 19 +- .../streaming/api/java/JavaPairDStream.scala | 17 ++ .../api/java/JavaStreamingContext.scala | 17 ++ .../streaming/dstream/CoGroupedDStream.scala | 17 ++ .../dstream/ConstantInputDStream.scala | 19 +- .../streaming/dstream/FileInputDStream.scala | 17 ++ .../streaming/dstream/FilteredDStream.scala | 17 ++ .../dstream/FlatMapValuedDStream.scala | 17 ++ .../streaming/dstream/FlatMappedDStream.scala | 17 ++ .../streaming/dstream/FlumeInputDStream.scala | 17 ++ .../streaming/dstream/ForEachDStream.scala | 17 ++ .../streaming/dstream/GlommedDStream.scala | 17 ++ .../streaming/dstream/InputDStream.scala | 17 ++ .../streaming/dstream/KafkaInputDStream.scala | 17 ++ .../dstream/MapPartitionedDStream.scala | 17 ++ .../streaming/dstream/MapValuedDStream.scala | 17 ++ .../streaming/dstream/MappedDStream.scala | 17 ++ .../dstream/NetworkInputDStream.scala | 17 ++ .../dstream/PluggableInputDStream.scala | 17 ++ .../streaming/dstream/QueueInputDStream.scala | 17 ++ .../streaming/dstream/RawInputDStream.scala | 17 ++ .../dstream/ReducedWindowedDStream.scala | 17 ++ .../streaming/dstream/ShuffledDStream.scala | 17 ++ .../dstream/SocketInputDStream.scala | 17 ++ .../streaming/dstream/StateDStream.scala | 17 ++ .../dstream/TransformedDStream.scala | 17 ++ .../dstream/TwitterInputDStream.scala | 17 ++ .../streaming/dstream/UnionDStream.scala | 17 ++ .../streaming/dstream/WindowedDStream.scala | 17 ++ .../streaming/receivers/ActorReceiver.scala | 17 ++ .../streaming/receivers/ZeroMQReceiver.scala | 17 ++ .../scala/spark/streaming/util/Clock.scala | 17 ++ .../streaming/util/MasterFailureTest.scala | 17 ++ .../spark/streaming/util/RawTextHelper.scala | 17 ++ .../spark/streaming/util/RawTextSender.scala | 17 ++ .../spark/streaming/util/RecurringTimer.scala | 17 ++ .../java/spark/streaming/JavaAPISuite.java | 17 ++ .../java/spark/streaming/JavaTestUtils.scala | 19 +- streaming/src/test/resources/log4j.properties | 19 +- .../streaming/BasicOperationsSuite.scala | 17 ++ .../spark/streaming/CheckpointSuite.scala | 17 ++ .../scala/spark/streaming/FailureSuite.scala | 17 ++ .../spark/streaming/InputStreamsSuite.scala | 17 ++ .../scala/spark/streaming/TestSuiteBase.scala | 17 ++ .../streaming/WindowOperationsSuite.scala | 17 ++ 488 files changed, 8448 insertions(+), 59 deletions(-) create mode 100644 NOTICE diff --git a/LICENSE b/LICENSE index d17afa1fc63..d6456956733 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 00000000000..7cbb114b2ae --- /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 b83a0ef6c0f..60bbc49e6c5 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 5ecdd7d0045..80c8d53d2bb 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 b97d786ed4d..de65e27fe0e 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 bc32663e0fd..a0c5ac9c183 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 9d9d80d809d..3c54a85f425 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 83d05cab2f8..5cdcf35b23a 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 a09c9780686..ef2d57fbd0a 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 030581e058c..c8fb5ca4736 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 d4b65588660..cd2c7b7b0d4 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 8ee3ec481fe..a5b88ca785a 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 0619097e4dc..354eb905a1c 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 b9891ad2f6e..0182f1ab240 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 83a3e1f3dc1..2288fb19d7b 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 1082c09eb10..d6db16882d0 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 4e05224190e..dad7c3df765 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 d352f6f6311..a043ac0095b 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 172ee5891d1..31a610bf9df 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 fbfc594472f..8e056f23d4e 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 dbb2da9a9cd..6329b2fbd88 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 f286f2cf9c5..25386b2796e 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 264d421d14a..b1002e0cace 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 f1c86de4cc9..df55be12545 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 875c0a220bd..0f972b7a0b7 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 8bc6fb6dea1..1a7cdf47881 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 301a57fffa2..6122fdced0d 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 6a0617cc063..1b06169739c 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 1b00208511b..8de44b1f661 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 514c17f241e..8bcbfc27351 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 07e7edea363..67aff03781c 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 cc6f3344a18..f458f2f6a1e 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 61dd72a6518..b0af8baf08c 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 ed732d36bfc..307d96111c8 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 a0652d7fc78..4b3d84670c2 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 7fdbe322fdf..aa3b1ed3a58 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 f1c86de4cc9..df55be12545 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 a4bb4bc701a..0625a6d5029 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 af25baf6411..05ad4b61d72 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 9fc9449827c..e8cd9801f63 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 dd3a557ae59..9f009a61d5d 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 8f1f5c65cd7..50c57a81a35 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 a78eddb1b58..176ba8da491 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 302411672cf..f446c55b19b 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 57c6df35bed..6ff92ce833b 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 df8ce9c054f..136b4da61e5 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 3239f4c3854..8f6953b1f52 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 20d677a8541..b0c83ce59d1 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 f7a2b7e8027..81314805a93 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 d5e7132ff96..8b39241095b 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 2af44aa3834..d17e70a4faf 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 178d31a73b9..93ef0977023 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 40b0193f199..a2dae6cae9a 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 5e8396edb92..b1fe0075a35 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 00901d95e26..a13a7a28590 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 4e0507c080a..c9dffbc6311 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 b04a27d0736..04c5f44e6be 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 c7dbcc6fbca..ee37da79483 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 0fc8c314630..79b0362830d 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 fde597ffd1a..2c417e31dba 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 8b313c645f8..6b0cc2fbf15 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 e384308ef6e..2a4edcec984 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 6f8cd17c882..660af70d524 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 8aa77266bc0..ca7cdd622ae 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 57e0405fb41..b615f820eb4 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 2911f9036eb..9f30b7f22f2 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 8306fbf5700..0236611ef9d 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 9513a001263..dcced035e74 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 f8a4c4e4891..6cc57566d79 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 3dd0a4b1f95..f41a379582a 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 43e6af2351a..46b9935cb7a 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 ec59b4f48fd..f2bdc11bdb2 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 aa7a16d7dd1..b7045eea63e 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 566aec622c0..f9b3f7965ea 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 dd0609026ac..b79f4ca8130 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 bb75ec208cb..3ad665da344 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 78eb33a628b..9df7d8277b8 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 a36186bf8a7..e6a96a5ec12 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 16692c04403..392556f261e 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 76051597b65..ccc511dc5ff 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 626b4994542..c0bf2cf5686 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 27f40ecdfd4..21b5abf053f 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 5f18b1e15bd..fe182e7ab60 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 97344e73da0..42b1de01b10 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 5e5845ac3a6..f385636e832 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 3a8192be3a4..8bc88d757fd 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 c6ef76d088f..1aa1e5dae03 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 e027cdacd3a..9eb0cfe3f9a 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 6044043add3..dda98710c2d 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 dae8295f213..2a2ea0aacf0 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 69bf12c8c9d..952d31ece49 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 b3cc4df6aa4..4aad602da3a 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 9fc6df4b888..ccfe64ecf14 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 b0096cf2bf0..f6fc0b0f7db 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 923f5cdf4f7..1758a38c4e2 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 2c6e9b15718..b093567d2c1 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 d618c098c2b..31a719fbff3 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 31d8ea89d43..af10822dbd2 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 85d1dfeac8b..078ad45ce85 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 adcb2d2415e..6f7d3853798 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 415bde5d677..aba56a60cad 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 5c6184c3c71..d33d95c7d95 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 7e30b8f7d21..c5658769504 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 3fd77af73f1..7855d44e9b0 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 c79bb93c388..b17ae63b5c4 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 c55c4761172..ea1e9a12c1b 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 02193c7008d..a8b22fbef87 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 577101e3c33..bad629e9652 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 51274acb1ed..e1f8aff6f51 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 5dc0c54552a..08c9a3b7252 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 335aaeb769b..64f89623e14 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 939f26b6f48..6b8e9f27af7 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 844c4142c7a..8ea7792ef4e 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 4af44f9c164..edefa0292d0 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 e8c4083f9df..064024455ee 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 f195082808a..4f4daa141a4 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 0e46db2272a..8a22b6b89fa 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 785c16e2bef..15ff9197382 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 15016b388d2..94f0ad8baec 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 48e6055fb57..99b60f7d092 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 4dd6c448a97..e5a7a87e2ea 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 3d28ecabb4d..d0ec3d5ea07 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 0c08c5f4172..4135cfeb284 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 0bf35014c80..3e50b7748d6 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 8553377d8f5..32264af3934 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 5e3c5e064fd..b05197c1b97 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 82bc6961e26..04b32c79688 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 5d3d54c65e7..8b51ff1c3a4 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 f20ea42d7ff..0bd88ea2537 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 2b96611ee33..9fcd3260ca9 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 c515f2e2382..7548a26c2ef 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 ccd55c1ce46..61d4cd6d99f 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 8360547a745..2e811518828 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 e97e509700b..33a6f8a824e 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 fd76029cb34..64b9fb88f89 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 5beb4d049ec..09d12fb65b9 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 10f3531df0d..4961c42fadd 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 ebe2ac68d8b..f4003da7329 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 1dc13754f90..31516278398 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 7b0e489a6c0..e566aeac137 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 6e28f677a31..b66c00b58c7 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 624a0948567..6c4e7dc03eb 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 b554e84251c..9d5c518293d 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 533e4610f3a..9e3827aaf56 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 d4f03610ebf..a25457ea35f 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 aaf9204d0e6..784db5ab62d 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 3693d509d68..18d0cbcc146 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 a174d5f4037..2bbc736f405 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 a4ff69e4d20..542c54c36b0 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 aed4254234e..bf46d32aa3a 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 8d5194a7379..b01f6369f65 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 d6fa4b1e807..cdf88b03a08 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 389ec4da3e5..b244bfbf069 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 de2dce161a0..691d939150a 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 75713b2eaaa..5eae144dfbc 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 463c33d6e23..8bdbe6c0123 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 daf2c5170cf..6aa92094eb4 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 01fbb8a11b5..ebe2e5a1e3a 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 c622df52205..2dadbbd5fb4 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 20fa55cff2a..ae2b63f7cb7 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 762c85400d8..5ddcad70759 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 200ed4ea1e9..922a9f9bc65 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 443abba5cdd..f3bb987d46c 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 58fb60f4418..4083abef03d 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 719d4bf03e3..0ebb722d736 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 38600b8be4e..150e5bca29d 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 377b1bdbe00..6794e0e2018 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 8966f9f86e3..c540cd36eb9 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 6d862c0c289..2b5bf18541c 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 e4dd3a7fa7d..d7d4db5d309 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 c84ec39d21f..783508cfd17 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 8ebc7789251..ed75eac3ff6 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 e16c7ba8819..1573f8a289a 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 07c103503c8..d0fdeb741e4 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 a50f4077373..59132437d20 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 d283c5b2bb8..af8f0a112f2 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 afb7504ba12..3b4e9518fd0 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 af07311b6d0..8b411dd85d9 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 901d01ef308..17fe805fd43 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 07585a88ceb..16ba0c26f8e 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 41ff62dd228..191cfde5651 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 c0baf43d43e..2cefdc78b01 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 243673f1518..574c9b141de 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 c7d1926b83d..0137f80953e 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 8a9efc5da21..0402b9f2501 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 2c52a67e226..2776826f187 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 b234428ab2b..6a4fa13ad61 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 f728e93d245..b1c43b3195a 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 105eaecb22e..71cc94edb68 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 64ed91f5a05..29e879aa425 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 acad915f13f..506c87f65b5 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 17d0ea4f80f..65f8c3200e4 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 f46b9d551d9..af108b8fec3 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 6a9d52f3561..85b5ddd4a81 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 a0fdf391e68..a61b335152e 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 6ff2e294345..69cd161c1f0 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 203abb917be..2f6a68ee85b 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 83166bce224..361b1e6b91e 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 95647389c3a..1c25605f75c 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 8de3aa91a4d..4fb1c5d42d3 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 6abfb7a1f71..4e3661ec5da 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 4937eb3b882..5428daeb947 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 8d83ff10c42..c4026f995a4 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 a6462c6968b..50768d43e02 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 6de0aa7adf8..dc0621ea7ba 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 7787b547620..51883080069 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 b75d3736cf5..245e7ccb52f 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 e4b5fcaedb6..dc3550dd0b1 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 3a0c29b27fe..7c10074dc76 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 327d6797ae2..3d065206752 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 8bf838209f3..8825f2dd242 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 941ba7a3f18..83708f07e1a 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 2dd9c0564fc..f557b142c4e 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 18cc15c2a5e..95554023c02 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 8844057a5c3..4431744ec33 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 f33310a34a2..69e0ac2a6bd 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 6e0c6793e01..4b3e3e50e13 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 170ede0f442..55d6c0a47ef 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 33352948446..ac9e5ef94d3 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 004592a5404..03a64e01921 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 b41e951be99..761fdf69198 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 718f26bfbd7..a1ebd48b013 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 b4dd75d90f2..07c3ddcc7e6 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 c47824315cc..06d1203f70b 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 b000e328e67..1f73cb99a78 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 f12fec41d50..e662ad67098 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 f4a2994b6d6..7bc60405440 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 e83368b98dd..75b8268b55b 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 2ad73b711de..dc94d42bb66 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 60b2aac797a..b7b24705a2a 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 f275d476df9..8ebfaf3cbf6 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 0718156b1ba..265e554ad88 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 bec876213e9..1965c5bc19b 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 9b39d3aadfd..e4ffa57ad20 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 1e557d61484..b36a6176c08 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 58888b1ebb5..3186f7c85b1 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 0d4384ba1f1..244000d9527 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 0010726c8d3..01de4ccb8f4 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 b264d1deb59..45cffad810e 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 3057ade2339..39064bce92c 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 30d7500e01e..ab72dbb62ba 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 ee0c5ff9a2d..b0229d6124d 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 42e2b07d5ca..01ed6e8c1fa 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 8188d3595ed..c8db0022b02 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 da859eebcb9..3495d653bd2 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 eba5ee507ff..b5a86b85a77 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 76f236057b2..3a0974fe15f 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 44638e0c2d8..8a7a6f9ed3d 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 cc0c354e7e9..f52650988c9 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 aca16f533aa..2aeed4ea3cf 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 5c406e68cb2..b3ab1ff4b4d 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 bc6f9c10d5a..ca6088ad93a 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 ed8f91842cc..a31e750d06b 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 874e5ba8ecf..9396f220634 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 fa46e2487df..b1d11954dd5 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 8bbc6ce88ea..a80e2d70026 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 6b8b9f05bbd..5ae7935ed48 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 2df5f0192bf..f31af3cda6e 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 84730cc091b..44dcf82d113 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 51b82b6a8c2..292966f23a0 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 3d67888e2cb..49ed069c759 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 d284134391f..4e0360d19ae 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 428db6fa956..003be54ad8e 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 18fc9673bf2..9233277bdb3 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 4bc5db8bb79..0575497f5d8 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 d7e67497fec..47a28e2f76e 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 81391837805..210450892b0 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 24738b43078..5d4d7a6c50a 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 b6e309fe1ae..34222805595 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 04276467471..daf0d58fa25 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 3726738842f..298562323ae 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 dafa9067121..92909e09590 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 48b5018dddb..22163ece8dd 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 e3f00ea8c73..00f782bbe76 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 09d588fe1ce..7e6842628a6 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 2b980340b73..76358d4151b 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 92dfaa6e6f3..cc7909194ad 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 5f1cc937524..41e3fd8cba1 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 835822edb23..ed49386f185 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 6e573b18836..996ffb18640 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 d05cf3dec1e..f6fef03689a 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 ac8ae7d308d..0af175f3168 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 362a31fb0d9..785721ece86 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 28a7b21b92b..a84c89e3c95 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 b2d0dd4627e..7d2831e19c9 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 0024ede828a..e11efe459c8 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 31c3dd75fb2..ed16b9d8efd 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 8c1445a4656..6c847b8fef8 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 c7855a7bd34..242ae971f85 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 e61ff7793d2..1e2c257c4b2 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 d306124fca4..aaf03e683bf 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 327e2ff848a..c3323dcbb33 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 bd184222ed3..ddc212d290d 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 6e585e1c3a5..ce6cec0451e 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 682d2745bff..b102eaf4e65 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 99e433e3bd6..b1e0b2b4d06 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 d263bb00e9c..35c04710a39 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 7f7d4c82115..cbddf4e523d 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 1da79f9824d..70c24515be1 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 bfaffa953e3..6bad6c1d13d 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 950218fa280..3a56c26befe 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 b5c8525f914..1ef812dfbdd 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 f7bf207c68c..b933c4aab83 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 ff315b66935..f2acd0bd3c1 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 94776e75720..93977d16f4c 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 1e1260f6060..31c3b25c502 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 96cb295f45b..5e6d7b09d8d 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 6afb0fa9bc7..dc8ca941c10 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 d27a2538e44..d1276d541f7 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 8e1ad27e14f..8f81d0b6ee5 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 30e6fef9505..f802b66cf13 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 2474d744b80..0f855c38da4 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 8bd813fd149..14bb58731b7 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 48aa67c543c..392d67d67bf 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 647bcaf860a..95a6eee2fcf 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 b9d5f9668e6..b719d65342d 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 fc0c1607202..56c1fed6ad0 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 cc6249b1dda..6578b55e824 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 ed5b36da73f..fdbd43d941b 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 794063fb6d3..4c0044202f0 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 c10ae595de0..45ef4bba82f 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 2714f19ba3f..454057aa0d2 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 7affe6fffc9..2ec3c007fba 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 1976765c3dd..7a8d08fadec 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 8b0a9b6808e..9485e0cfa96 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 626034eb0d4..2d34776177c 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 6b22e7120c9..d22684d9804 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 a15a967de85..d5f42fbb38e 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 b319bdab44e..559d7f9e533 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 9d4c7a252df..1af370c1c38 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 e24c6ddaa79..096a9ae2197 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 3e57580fd4e..c54d3f3d599 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 15b82c8da15..1f4a9915428 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 ba59be16872..911490cb6c1 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 0fe1833e83f..104bfd52047 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 21a90f2e5ad..67ddaec8d20 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 a6603653f15..5cee4136158 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 6e910154d42..4dd6c243acf 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 dd61c467f7f..23258336e23 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 2de810e062b..7a449a9d720 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 f958ef9f72f..c1f8d32aa87 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 4849f216fb2..0a0bc6f476b 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 cd73f553d6e..ab99bf1fbe4 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 9457472f2d5..ccd69695df4 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 6497596d35f..e815ececf7a 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 a0aaf609186..d0b1cf06e59 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 461b84a2c66..d197bbaf7c4 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 435675f9de4..4641b824448 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 8fb3b0fb2ad..ba0dfd8f9b5 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 3d080a02577..ef6e09a8e84 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 4161c59fead..38ed3b149af 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 2f41aeb376f..52a0d69744a 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 f598d2ff9c7..00560ac9d16 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 911ae8f168d..bf988a953bb 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 3b847fe6034..f97174aeaea 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 39c76fd98a1..3ab4fc2c37b 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 9389f8a38dc..f5baec242da 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 9202e65e092..4929703ba2f 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 704540c2bf5..150fb5eb9c3 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 f450e210401..da36c8c23c6 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 175281e0956..7fb680bcc31 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 51c3c9f9b4b..33ab324732c 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 528778ed722..8770abd57e0 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 896e9fd8af1..cba5c986bea 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 65f0b6d3525..682b99f75e5 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 74d0d338b75..e264fae6097 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 4c6e08bc744..375d5c9d220 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 e226a4a73a1..a24342bebf9 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 feb13d52f9d..0116215163b 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 b0e141ff329..d875d6de50a 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 4fd0646160e..b8f80e80cd7 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 e12b3be2512..89fe7d7e853 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 90b0999a5ec..2fb0c8136f8 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 eff853f379b..e1b73bc25e9 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 ea80bfcbfd4..b864fd4634d 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 4c18cbdc6bc..7da96397a62 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 fb812a6dbeb..38637b3dd13 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 e4db7bb9b73..bb294c22570 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 6e7c023bac7..8094d224054 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 f79974c1915..645204ddf3e 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 5f813df402a..7c7f912b43c 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 b83f505d8e4..c2260ae2869 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 08a031dded1..b5e564df6d4 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 390c92763cb..a112e0b5069 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 cb096f39a97..bebade9afbc 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 2ada9ae76b5..f98590b8d96 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 04d3400cb42..bc9bfd054fe 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 df41dbbdff8..3c588c61620 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 6fa1f1d4f10..4b48072c6e0 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 178a056de83..7a4d4c4575b 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 9b860e23c51..08e17131f64 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 d662e90287e..37a355462e7 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 45102cd9fec..d5d5aa5454b 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 010f80097f1..f2b2eee64cd 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 72cf9f88c6e..c670556f2b8 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 f13698a86fd..54d227d0d33 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 127cba029bd..33c026e824b 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 7f85a1008e9..40be3b50007 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 857160624b5..41c846ba79b 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 3e9d7d36da8..d367f91967f 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 def810dd461..dfdaba274f4 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 657fe6f9897..2f741cb3451 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 78c9457b84d..eb18ec08c91 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 001b7a28b66..89bcbcfe068 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 2329e536cc4..e503fb7621e 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 a9fec17a9da..c6a6b24c5a6 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 5a95144983a..fecacd12416 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 54ff1bf8e7c..cc8cd9e3c41 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 1e34d473650..dfd841b10a0 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 379bbfd4c2c..75d692beeb0 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 a3a9ff5dcb2..1ee947d4145 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 7a7280313ed..7c4e722cc15 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 c54c9e97a00..0e5100e00be 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 47b9cccdfed..d3c1c234137 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 219c66eb0ba..8b258543db0 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 92a2020b48d..7d8da03254b 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 0e9aa863b55..274bc585db9 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 58809ab646a..d824d62fd10 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 cfb1a390e6e..a6d33e69d21 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 f46e6d8be47..80ae6055580 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 6b5bc01a51d..72ee76d7221 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 cc5605f8a9e..c91764e6179 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 a9c4df180ff..dc5e50931e3 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 523fbb346b3..8c9a42824fe 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 ce3ae701745..56a16d77f21 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 b66c374ca86..feccbf5cc2a 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 a8e72143fbc..5371fc540a3 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 34697d52d73..ec65eabb741 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 2fb5bbdeb5c..7e6b06d772f 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 450e48d66e2..1e4c1e3742b 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 9be7926a4a8..684d3abb564 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 6b0fade7c64..399ca1c63d8 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 3b331956f59..c09a332d44c 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 c2135195d89..12a14e233d9 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 6a8b81760e3..b30cd969e9f 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 67bd8388bc8..ceb3f92b65b 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 7696c4a592b..a31230689f9 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 b159d26c02b..d4cf2e568ca 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 8d0a83d4399..47bf07bee13 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 1c4b22a8981..252cc2a3032 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 36b841af8fa..ffd656227dd 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 f14decf08ba..ad5eab9dd2a 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 4d93f0a5f72..7dcb1d713d9 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 548809a3596..3ab5c1fddeb 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 30240cad988..ccd15563b07 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 ed7b789d981..b7720ad0ea1 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 4ef4bb7de10..99553d295d5 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 41c3af46948..095137092a9 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 41b9bd9461d..de0536125de 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 e993164f994..9d8c5c3175b 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 cabd34f5f2e..78d7117f0f7 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 a69af605890..d13bebb10f6 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 c9644b3a83a..4906f503c29 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 ee69ea51779..7df537eb560 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 b589cbd4d50..06fda6fe8e4 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 3c5d43a6095..4dbdec459d0 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 55d2957be49..6ee588af154 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 848afecfad5..af41a1b9acd 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 6055aa6a059..8d8a6161c67 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 20818a0cab1..3fda84a38a1 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 122a529bb76..344b41c4d0f 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 3c2a81947b9..33f7cd063f3 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 da224ad6f71..b269061b737 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 1b2fa567795..236f74f575e 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 343b6915e79..96260501ab7 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 1f9548bfb85..83b57b27f77 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 1408af0afa5..5877b10e0e1 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 db62955036e..4b46613d5e1 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 99660d9dee0..e7fbc5bbcfa 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 ff7a58be452..f09a8b9f907 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 00bad5da34c..3eaa9a7e7f4 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 cbf0c88108b..fd24d61730d 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 036c95a860c..2d9937eab81 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 5533c3cf1ef..22d554e7e49 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 974651f9f6c..d9ac722df5c 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 426a9b6f71d..8ce5d8daf52 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 03749d4a944..bf04120293e 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 bd0b0e74c1d..5cc6ad9dee3 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 8e10276deb9..7ecc44236d4 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 4cf10582a95..3b93790baa8 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 64a7e7cbf9a..f9d25db8da6 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 59c445e63f7..063529a9cbc 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 565089a853d..67e3e0cd30c 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 607dea77ec2..8c639648f00 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 a5fa7ab92dd..7fc649fe270 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 b024fc9dccc..1c5419b16da 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 ad6aa79d102..cb34b5a7cc4 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 80d827706f6..894b765fc62 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._ From 4ff494de20c36151dc29a60825d67e094d14acd4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 16 Jul 2013 17:26:48 -0700 Subject: [PATCH 0383/2521] 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 44826f339c1..835d1af794d 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 bd48b43da37..4c539649a25 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. From 039087e1e3916799799b2e544288857545dddaec Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 17 Jul 2013 11:46:00 +0530 Subject: [PATCH 0384/2521] Fixed formatting As per review comments on #709 --- .../src/main/scala/spark/scheduler/cluster/SchedulingMode.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala index 1c1f17ebf26..3ff3cd5fe45 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala @@ -1,6 +1,6 @@ package spark.scheduler.cluster -object SchedulingMode extends Enumeration{ +object SchedulingMode extends Enumeration { type SchedulingMode = Value val FAIR,FIFO = Value From a1d2c3434657f0d901a2f8cfddf62065b5ad4385 Mon Sep 17 00:00:00 2001 From: ctn Date: Wed, 17 Jul 2013 11:27:11 -0700 Subject: [PATCH 0385/2521] [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 8c9a42824fe..d64645e32da 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 "$@" From 84fa20c2a135f54745ddde9abb4f5e60af8856d1 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 14:04:05 -0700 Subject: [PATCH 0386/2521] Allow initial weight vectors in LogisticRegression. Also move LogisticGradient to the LogisticRegression file and fix the unit tests log path. --- .../spark/mllib/optimization/Gradient.scala | 18 ------ .../mllib/optimization/GradientDescent.scala | 5 +- .../mllib/regression/LogisticRegression.scala | 61 ++++++++++++++++++- mllib/src/test/resources/log4j.properties | 2 +- .../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 2fb0c8136f8..d5338360c83 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 e1b73bc25e9..2ac08083572 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 bb294c22570..dab15aa386c 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 a112e0b5069..4265ba6e5de 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 bc9bfd054fe..2ff248d256d 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]") + } } From 64b88e039a66fffdc1d0f0eb1caa9328d76bbb06 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 14:11:44 -0700 Subject: [PATCH 0387/2521] Move ML lib data generator files to util/ --- .../{regression => util}/LogisticRegressionGenerator.scala | 2 +- .../mllib/{regression => util}/RidgeRegressionGenerator.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) rename mllib/src/main/scala/spark/mllib/{regression => util}/LogisticRegressionGenerator.scala (98%) rename mllib/src/main/scala/spark/mllib/{regression => util}/RidgeRegressionGenerator.scala (98%) diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala similarity index 98% rename from mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala rename to mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala index 8094d224054..4c580b44daa 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.regression +package spark.mllib.util import scala.util.Random diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala similarity index 98% rename from mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala rename to mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala index c2260ae2869..a5bb2e762f3 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.regression +package spark.mllib.util import scala.util.Random From 3bf989713654129ad35a80309d1b354ca5ddd06c Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 14:20:24 -0700 Subject: [PATCH 0388/2521] Rename loss -> stochasticLoss and add a note to explain why we have multiple train methods. --- .../spark/mllib/optimization/GradientDescent.scala | 9 +++++---- .../spark/mllib/regression/LogisticRegression.scala | 10 ++++++---- .../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 2ac08083572..77f5a7ae246 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 dab15aa386c..664baf33a3a 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 7c7f912b43c..f724edd732f 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 { From 45f3c855181539306d5610c5aa265f24b431c142 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 16:03:29 -0700 Subject: [PATCH 0389/2521] Change weights to be Array[Double] in LR model. Also ensure weights are initialized to a column vector. --- .../mllib/optimization/GradientDescent.scala | 5 ++--- .../mllib/regression/LogisticRegression.scala | 17 +++++++++++------ .../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 77f5a7ae246..2c5038757b6 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 664baf33a3a..ab865af0c68 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 2ff248d256d..47191d9a5a0 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]") } From 217667174e267adba5469cf26b3e4418e3d1cc90 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 17 Jul 2013 16:08:34 -0700 Subject: [PATCH 0390/2521] Return Array[Double] from SGD instead of DoubleMatrix --- .../scala/spark/mllib/optimization/GradientDescent.scala | 4 ++-- .../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 2c5038757b6..4c996c0903e 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 ab865af0c68..711e205c39d 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) From 31b3c1df54fb2d3cb0aac9acbc0a13c274c64cd2 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 17 Jul 2013 17:43:15 -0700 Subject: [PATCH 0391/2521] Small improvement in python script --- 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 413fcf26953..d1db535e8cf 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -158,7 +158,7 @@ def is_active(instance): def get_spark_shark_version(opts): spark_shark_map = {"0.7.2": "0.7.0"} version = opts.spark_version.replace("v", "") - if version not in ["0.7.2"]: + if version not in spark_shark_map: print >> stderr, "Don't know about Spark version: %s" % version sys.exit(1) return (version, spark_shark_map[version]) From 88a0823c58304fa1b290667d95a482105a8a5f52 Mon Sep 17 00:00:00 2001 From: Ubuntu Date: Thu, 18 Jul 2013 00:51:18 +0000 Subject: [PATCH 0392/2521] 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 4c539649a25..e6fdc5306e5 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 9875de62bdf..fbdbc7d19d9 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 50ecf834045..ede6c78428a 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 166a884c889..685ed8be8c3 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 0e5100e00be..8b5d8300f2e 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 d3c1c234137..bcfae22677f 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 8b258543db0..ec7e33e1e3d 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 72ee76d7221..f49aa927969 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 8c9a42824fe..b6f91f5085a 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 5371fc540a3..31a41381242 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 From 3aad45265324537e51cab79db3f09d9310339f77 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 19 Jul 2013 02:29:46 +0800 Subject: [PATCH 0393/2521] 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 e9819b877d0..59edda50973 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( From 7ab1170503ab46ffd838fb98590dd3d66748dd38 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 18 Jul 2013 11:55:19 -0700 Subject: [PATCH 0394/2521] Refactor data generators to have a function that can be used in tests. --- .../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 4c580b44daa..8d659cd97c6 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 a5bb2e762f3..6861913dc7e 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() From 2c9ea56db4a1f4324f1dc8fadaf2491d83eda8ca Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 18 Jul 2013 11:57:14 -0700 Subject: [PATCH 0395/2521] Rename classes to be called DataGenerator --- ...Generator.scala => LogisticRegressionDataGenerator.scala} | 0 ...ionGenerator.scala => RidgeRegressionDataGenerator.scala} | 5 ++--- 2 files changed, 2 insertions(+), 3 deletions(-) rename mllib/src/main/scala/spark/mllib/util/{LogisticRegressionGenerator.scala => LogisticRegressionDataGenerator.scala} (100%) rename mllib/src/main/scala/spark/mllib/util/{RidgeRegressionGenerator.scala => RidgeRegressionDataGenerator.scala} (96%) diff --git a/mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala similarity index 100% rename from mllib/src/main/scala/spark/mllib/util/LogisticRegressionGenerator.scala rename to mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala diff --git a/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionDataGenerator.scala similarity index 96% rename from mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala rename to mllib/src/main/scala/spark/mllib/util/RidgeRegressionDataGenerator.scala index 6861913dc7e..c5b8a29942c 100644 --- a/mllib/src/main/scala/spark/mllib/util/RidgeRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionDataGenerator.scala @@ -23,8 +23,7 @@ import org.jblas.DoubleMatrix import spark.{RDD, SparkContext} - -object RidgeRegressionGenerator { +object RidgeRegressionDataGenerator { /** * Generate an RDD containing test data used for RidgeRegression. This function generates @@ -82,7 +81,7 @@ object RidgeRegressionGenerator { val parts: Int = if (args.length > 4) args(4).toInt else 2 val eps = 10 - val sc = new SparkContext(sparkMaster, "RidgeRegressionGenerator") + val sc = new SparkContext(sparkMaster, "RidgeRegressionDataGenerator") val data = generateRidgeRDD(sc, nexamples, nfeatures, eps, parts) MLUtils.saveLabeledData(data, outputPath) From a613628c5078cf41feb973d0ee8a06eb69615bcf Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 19 Jul 2013 16:59:12 +0800 Subject: [PATCH 0396/2521] 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 46b9935cb7a..957c541ecf6 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)) + } } } From aa6f83289b87f38481dbae60ad91d2ac78ccea46 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 19 Jul 2013 22:25:28 +0800 Subject: [PATCH 0397/2521] A better fix for giving local jars unde Yarn mode. --- .../scala/spark/deploy/SparkHadoopUtil.scala | 3 +++ .../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 df55be12545..9f040faac3b 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 df55be12545..9f040faac3b 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 957c541ecf6..c01e315e357 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)) } } From 4530e8a9bfe35b6d562876b1fb66e534ff5c286d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 20 Jul 2013 00:04:25 +0800 Subject: [PATCH 0398/2521] 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 9f040faac3b..617954cb987 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 9f040faac3b..617954cb987 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 } } From d1738d72ba2ff28c38e03beb3e17f03d3dd77e1a Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 20 Jul 2013 00:37:24 +0800 Subject: [PATCH 0399/2521] 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 59edda50973..2f2cbf646ae 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 { From cfce9a6a365a8a4b156e99f412157704cab592b9 Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Thu, 18 Jul 2013 17:43:27 -0700 Subject: [PATCH 0400/2521] Regression: default webui-port can't be set via command line "--webui-port" anymore --- core/src/main/scala/spark/deploy/master/Master.scala | 2 +- .../main/scala/spark/deploy/master/MasterArguments.scala | 5 ++++- .../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 e5a7a87e2ea..eddcafd84d1 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 d0ec3d5ea07..0ae01607673 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 04b32c79688..dabc2d8dc76 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" } From 81bb5dc64007121df52cbafdf5f025c97f145953 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 19 Jul 2013 14:08:30 -0700 Subject: [PATCH 0401/2521] Creates Executors tab for application with RDD block and memory/disk used, solves SPARK-808 --- .../main/scala/spark/executor/Executor.scala | 1 + .../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 ++ .../scala/spark/ui/exec/ExecutorsUI.scala | 79 +++++++++++++++++++ .../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 2e811518828..82e5f5a0155 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 31516278398..e14dc57501c 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 a31e750d06b..03034a45207 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 9396f220634..1534705bc5a 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 b1d11954dd5..e33c80282a5 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 00000000000..de0e571553c --- /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 4e0360d19ae..f76192eba84 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) } From 865dc63bac1cb72ecb84038c0a7cd708cc9e19d7 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 19 Jul 2013 15:57:01 -0700 Subject: [PATCH 0402/2521] Changed table format for executors --- .../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 de0e571553c..fa516a4a340 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 From f4d514810e6fd9f42868ebb9a89390c62c3b42e1 Mon Sep 17 00:00:00 2001 From: Konstantin Boudnik Date: Tue, 2 Jul 2013 17:03:24 -0700 Subject: [PATCH 0403/2521] 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 00000000000..6ee2a536d7a --- /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 00000000000..1382539f245 --- /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 00000000000..dd05f35f1fb --- /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 4b48072c6e0..6d9437b69f3 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 31a41381242..62fc18550df 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: From 0337d88321f3681009de548ce10ba7e0ca8f1a58 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Sun, 21 Jul 2013 18:07:19 -0700 Subject: [PATCH 0404/2521] 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 c01e315e357..1b46665d2ce 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 cbddf4e523d..ff2dcd72d85 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") { From 8901f379c98d19d3145f0cb2ae678d1b0060a311 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 22 Jul 2013 09:58:03 -0700 Subject: [PATCH 0405/2521] 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 fa516a4a340..5fb75c60fd3 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( From f649dabb4a3a57cb25a852808297fb718cbfedd4 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Jul 2013 13:14:33 -0700 Subject: [PATCH 0406/2521] Fix bug: DoubleRDDFunctions.sampleStdev() computed non-sample stdev(). Update JavaDoubleRDD to add new methods and docs. Fixes SPARK-825. --- .../main/scala/spark/DoubleRDDFunctions.scala | 8 ++++- .../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 93ef0977023..104168e61cc 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 392556f261e..8ce7df62133 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 aaf03e683bf..5e2bf2d231d 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -326,7 +326,9 @@ public Boolean call(Double x) { 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); From 85c4d7bf3bf0969f58ebcda6ca68719972ff0c46 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 22 Jul 2013 14:35:47 -0700 Subject: [PATCH 0407/2521] Shows number of complete/total/failed tasks (bug: failed tasks assigned to null executor) --- core/src/main/scala/spark/ui/SparkUI.scala | 1 + .../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 1534705bc5a..7599f82a944 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 5fb75c60fd3..a981c680d22 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 From b4b230e6067d9a8290b79fe56bbfdea3fb12fdbb Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Mon, 22 Jul 2013 14:42:47 -0700 Subject: [PATCH 0408/2521] Fixing for LocalScheduler with test, that much works .. --- .../scala/spark/scheduler/TaskResult.scala | 27 +++++++++++++++++-- .../scheduler/local/LocalScheduler.scala | 6 +++-- .../scala/spark/KryoSerializerSuite.scala | 15 +++++++++++ 3 files changed, 44 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/TaskResult.scala b/core/src/main/scala/spark/scheduler/TaskResult.scala index 6de0aa7adf8..0b459ea6006 100644 --- a/core/src/main/scala/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/spark/scheduler/TaskResult.scala @@ -4,6 +4,8 @@ import java.io._ import scala.collection.mutable.Map import spark.executor.TaskMetrics +import spark.SparkEnv +import java.nio.ByteBuffer // Task result. Also contains updates to accumulator variables. // TODO: Use of distributed cache to return result is a hack to get around @@ -13,7 +15,19 @@ class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var metrics: def this() = this(null.asInstanceOf[T], null, null) override def writeExternal(out: ObjectOutput) { - out.writeObject(value) + + val objectSer = SparkEnv.get.serializer.newInstance() + val bb = objectSer.serialize(value) + + out.writeInt( bb.remaining()) + if (bb.hasArray) { + out.write(bb.array(), bb.arrayOffset() + bb.position(), bb.remaining()) + } else { + val bbval = new Array[Byte](bb.remaining()) + bb.get(bbval) + out.write(bbval) + } + out.writeInt(accumUpdates.size) for ((key, value) <- accumUpdates) { out.writeLong(key) @@ -23,7 +37,16 @@ class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var metrics: } override def readExternal(in: ObjectInput) { - value = in.readObject().asInstanceOf[T] + + //this doesn't work since SparkEnv.get == null + // in this context + val objectSer = SparkEnv.get.serializer.newInstance() + + val blen = in.readInt() + val byteVal = new Array[Byte](blen) + in.readFully(byteVal) + value = objectSer.deserialize(ByteBuffer.wrap(byteVal)) + val numUpdates = in.readInt if (numUpdates == 0) { accumUpdates = null diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 93d4318b299..0b800ec7409 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -145,6 +145,7 @@ 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() + val objectSer = SparkEnv.get.serializer.newInstance() try { Accumulators.clear() Thread.currentThread().setContextClassLoader(classLoader) @@ -165,9 +166,9 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: // 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) + val serResult = objectSer.serialize(result) deserializedTask.metrics.get.resultSize = serResult.limit() - val resultToReturn = ser.deserialize[Any](serResult) + val resultToReturn = objectSer.deserialize[Any](serResult) val accumUpdates = ser.deserialize[collection.mutable.Map[Long, Any]]( ser.serialize(Accumulators.values)) logInfo("Finished " + taskId) @@ -218,6 +219,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: val taskSetId = taskIdToTaskSetId(taskId) val taskSetManager = activeTaskSets(taskSetId) taskSetTaskIds(taskSetId) -= taskId + SparkEnv.set(env) taskSetManager.statusUpdate(taskId, state, serializedData) } } diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/spark/KryoSerializerSuite.scala index 327e2ff848a..a2f91648b68 100644 --- a/core/src/test/scala/spark/KryoSerializerSuite.scala +++ b/core/src/test/scala/spark/KryoSerializerSuite.scala @@ -7,6 +7,7 @@ import org.scalatest.FunSuite import com.esotericsoftware.kryo._ import SparkContext._ +import spark.test.{ClassWithoutNoArgConstructor, MyRegistrator} class KryoSerializerSuite extends FunSuite { test("basic types") { @@ -109,6 +110,20 @@ class KryoSerializerSuite extends FunSuite { System.clearProperty("spark.kryo.registrator") } + + test("kryo-collect") { + System.setProperty("spark.serializer", "spark.KryoSerializer") + System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName) + + val sc = new SparkContext("local", "kryoTest") + val control = 1 :: 2 :: Nil + val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_)).collect().map(_.x) + assert(control == result.toSeq) + + System.clearProperty("spark.kryo.registrator") + System.clearProperty("spark.serializer") + } + } package test { From 8ca0c319441c7beddead4dc5b4bd2a1c3e2141e8 Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Mon, 22 Jul 2013 14:48:46 -0700 Subject: [PATCH 0409/2521] removing non-pertinent comment --- core/src/main/scala/spark/scheduler/TaskResult.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/TaskResult.scala b/core/src/main/scala/spark/scheduler/TaskResult.scala index 0b459ea6006..e14d41249b3 100644 --- a/core/src/main/scala/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/spark/scheduler/TaskResult.scala @@ -38,8 +38,6 @@ class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var metrics: override def readExternal(in: ObjectInput) { - //this doesn't work since SparkEnv.get == null - // in this context val objectSer = SparkEnv.get.serializer.newInstance() val blen = in.readInt() From 2eea974795dfa2bb79e66496454f36cb499065b0 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 22 Jul 2013 15:15:54 -0700 Subject: [PATCH 0410/2521] 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 82e5f5a0155..2e811518828 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 e14dc57501c..31516278398 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 a981c680d22..40bee325b2a 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( From 8e38e7723230c1ca9564ddb79087fd1c4326b351 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 22 Jul 2013 16:05:32 -0700 Subject: [PATCH 0411/2521] 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 3a56c26befe..752e4b85e67 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") } } From c83680434bfd6c241ed2e66f2e8704d3b93f752f Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 17 Jul 2013 22:59:00 -0700 Subject: [PATCH 0412/2521] 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 +++ .../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 835d1af794d..eb836b0ffd1 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 e6fdc5306e5..eb270a54281 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 4b48072c6e0..eb7bd7e9df9 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 2f2cbf646ae..f3f67b57c89 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 00000000000..3dfba5808c9 --- /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 00000000000..89ef7236f47 --- /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() + } +} From e17e1b388eaddea39a7272bae201fd3d0060c821 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 20 Jul 2013 22:04:30 -0700 Subject: [PATCH 0413/2521] Remove annotation code that broke build. --- .../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 89ef7236f47..3a55f508128 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 From 872c97ad829ba20e866c4e45054e7d2d05b02042 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 22 Jul 2013 16:54:37 -0700 Subject: [PATCH 0414/2521] Split task columns, memory columns sort by numeric value --- .../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 40bee325b2a..20ea54d6a62 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 ) } From 2c2bfbe294c0082520c80a01562a2dbeeba63b7a Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 23 Jul 2013 01:36:44 -0700 Subject: [PATCH 0415/2521] 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 1b46665d2ce..0fb7dfa8101 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 cc7909194ad..07772a0afbe 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` */ From 4830e225624091fa836012651420cf2b5b97dcca Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 23 Jul 2013 09:50:13 -0700 Subject: [PATCH 0416/2521] 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 0fb7dfa8101..24ba6056466 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 From efd6418c1b99c1ecc2b0a4c72e6430eea4d86260 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 23 Jul 2013 10:40:41 -0700 Subject: [PATCH 0417/2521] Move getPersistentRDDs testing to a new Suite --- core/src/test/scala/spark/RDDSuite.scala | 6 -- .../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 ff2dcd72d85..cbddf4e523d 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 00000000000..6d50bf5e1b3 --- /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 From 87a9dd898ff51fd110799edae087d59f6b714211 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 12:13:27 -0700 Subject: [PATCH 0418/2521] Made RegressionModel serializable and added unit tests to make sure predict methods would work. --- .../mllib/optimization/GradientDescent.scala | 6 ++-- .../spark/mllib/optimization/Updater.scala | 6 ++-- .../mllib/regression/LogisticRegression.scala | 6 +++- .../spark/mllib/regression/Regression.scala | 2 +- .../mllib/regression/RidgeRegression.scala | 5 ++- .../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 4c996c0903e..185a2a24f63 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 b864fd4634d..18cb5f3a95c 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 711e205c39d..4b225460170 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 645204ddf3e..b845ba1a890 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 f724edd732f..6ba141e8fb3 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 47191d9a5a0..6a8098b59d3 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) } } From 2210e8ccf8d77f65442a344c4eae39e000fba927 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 12:52:15 -0700 Subject: [PATCH 0419/2521] Use a different validation dataset for Logistic Regression prediction testing. --- .../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 6a8098b59d3..0a99b78cf85 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) } } From ac60d063810c6eb4f7f5d8eac7237229cf73b9c1 Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Tue, 23 Jul 2013 13:13:19 -0700 Subject: [PATCH 0420/2521] Re-working in terms of changes to TaskSetManager. Verified with Standalone and Local mode. --- .../spark/scheduler/cluster/ClusterTaskSetManager.scala | 5 ++++- .../main/scala/spark/scheduler/local/LocalScheduler.scala | 1 - .../scala/spark/scheduler/local/LocalTaskSetManager.scala | 4 +++- 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index d72b0bfc9f5..813835d91a8 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -74,7 +74,9 @@ private[spark] class ClusterTaskSetManager( val SPECULATION_MULTIPLIER = System.getProperty("spark.speculation.multiplier", "1.5").toDouble // Serializer for closures and tasks. - val ser = SparkEnv.get.closureSerializer.newInstance() + val env = SparkEnv.get + val ser = env.closureSerializer.newInstance() + val tasks = taskSet.tasks val numTasks = tasks.length @@ -488,6 +490,7 @@ private[spark] class ClusterTaskSetManager( } def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { + SparkEnv.set(env) state match { case TaskState.FINISHED => taskFinished(tid, state, serializedData) diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 0b800ec7409..83b8dbc4dfc 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -219,7 +219,6 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: val taskSetId = taskIdToTaskSetId(taskId) val taskSetManager = activeTaskSets(taskSetId) taskSetTaskIds(taskSetId) -= taskId - SparkEnv.set(env) taskSetManager.statusUpdate(taskId, state, serializedData) } } diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index 70b69bb26fe..49d3bdc0af5 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -26,7 +26,8 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas val taskInfos = new HashMap[Long, TaskInfo] val numTasks = taskSet.tasks.size var numFinished = 0 - val ser = SparkEnv.get.closureSerializer.newInstance() + val env = SparkEnv.get + val ser = env.closureSerializer.newInstance() val copiesRunning = new Array[Int](numTasks) val finished = new Array[Boolean](numTasks) val numFailures = new Array[Int](numTasks) @@ -120,6 +121,7 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas } def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { + SparkEnv.set(env) state match { case TaskState.FINISHED => taskEnded(tid, state, serializedData) From 310e73d566331258fe27cf73624dbff85f94a847 Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Tue, 23 Jul 2013 13:23:25 -0700 Subject: [PATCH 0421/2521] style --- .../scala/spark/scheduler/cluster/ClusterTaskSetManager.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 813835d91a8..ec643dec92f 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -77,7 +77,6 @@ private[spark] class ClusterTaskSetManager( val env = SparkEnv.get val ser = env.closureSerializer.newInstance() - val tasks = taskSet.tasks val numTasks = tasks.length val copiesRunning = new Array[Int](numTasks) From 0200801a55b580c7504687e3476b7a71c7699001 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 23 Jul 2013 13:35:43 -0700 Subject: [PATCH 0422/2521] Tracks task start events and shows number of active tasks on Executor UI --- .../scala/spark/scheduler/DAGScheduler.scala | 8 ++++++++ .../spark/scheduler/DAGSchedulerEvent.scala | 2 ++ .../main/scala/spark/scheduler/JobLogger.scala | 16 +++++++++++++++- .../scala/spark/scheduler/SparkListener.scala | 9 ++++++++- .../spark/scheduler/TaskSchedulerListener.scala | 3 +++ .../cluster/ClusterTaskSetManager.scala | 6 ++++++ .../scheduler/local/LocalTaskSetManager.scala | 5 +++++ .../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 29e879aa425..b02bf8f4bf4 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 506c87f65b5..3b4ee6287aa 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 85b5ddd4a81..f7565b8c57e 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 4fb1c5d42d3..4eb7e4e6a5f 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 245e7ccb52f..2cdeb1c8c0c 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 3d065206752..14e87af653b 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 e662ad67098..b5004519903 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 20ea54d6a62..9ac33326c0b 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 => From 9f2dbb2a7ca9c1cf878cf96df7122b05d95e799b Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 23 Jul 2013 15:10:09 -0700 Subject: [PATCH 0423/2521] 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 9ac33326c0b..bb2b0034867 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 => From 72bac09c4246aba6a3c7d72d23c75fc2c4db8625 Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Tue, 23 Jul 2013 15:19:07 -0700 Subject: [PATCH 0424/2521] Leaking spark context in the test --- core/src/test/scala/spark/KryoSerializerSuite.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/spark/KryoSerializerSuite.scala index 33364391640..83df0af1ae1 100644 --- a/core/src/test/scala/spark/KryoSerializerSuite.scala +++ b/core/src/test/scala/spark/KryoSerializerSuite.scala @@ -133,9 +133,13 @@ class KryoSerializerSuite extends FunSuite { System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName) val sc = new SparkContext("local", "kryoTest") - val control = 1 :: 2 :: Nil - val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_)).collect().map(_.x) - assert(control == result.toSeq) + try { + val control = 1 :: 2 :: Nil + val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_)).collect().map(_.x) + assert(control == result.toSeq) + } finally { + sc.stop() + } System.clearProperty("spark.kryo.registrator") System.clearProperty("spark.serializer") From 101b8cc78a1fb49332265d7a7e4bf4ec6a22dd42 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 15:28:20 -0700 Subject: [PATCH 0425/2521] SPARK-829: scheduler shouldn't hang if a task contains unserializable objects in its closure. --- .../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 29e879aa425..5fcd807aff2 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 6c847b8fef8..c3c52f9118b 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. } From 5ed38b4d1d7154235d5b72c35d3a8e63bac3a2de Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 15:28:59 -0700 Subject: [PATCH 0426/2521] Scheduler code style cleanup. --- .../cluster/ClusterTaskSetManager.scala | 154 +++++++++++------- .../scheduler/cluster/TaskSetManager.scala | 22 ++- .../scheduler/local/LocalScheduler.scala | 22 ++- .../scheduler/local/LocalTaskSetManager.scala | 82 +++++----- .../scheduler/ClusterSchedulerSuite.scala | 2 +- .../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 3d065206752..d934293b70e 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 07c3ddcc7e6..7978a5df746 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 1f73cb99a78..edd83d4cb49 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 e662ad67098..bbce9eda648 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 8f81d0b6ee5..05afcd65676 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 f802b66cf13..a8b88d7936e 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 From f2422d4f29abb80b9bc76c4596d1cc31d9e6d590 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 15:28:20 -0700 Subject: [PATCH 0427/2521] SPARK-829: scheduler shouldn't hang if a task contains unserializable objects in its closure. --- .../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 29e879aa425..5fcd807aff2 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 6c847b8fef8..c3c52f9118b 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. } From 383684daaa62bebc177b4c74573ce509f154f74b Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 23 Jul 2013 15:33:27 -0700 Subject: [PATCH 0428/2521] 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 bb2b0034867..606e1eb2fca 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 => From abc78cd3318fb7bc69d10fd5422d20b299a8d7d8 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Tue, 23 Jul 2013 15:47:16 -0700 Subject: [PATCH 0429/2521] 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 b02bf8f4bf4..7bf50de660e 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 606e1eb2fca..69fb3060746 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 => From 6a31b7191d5a8203563a2a4e600210c67439abf5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 23 Jul 2013 16:20:24 -0700 Subject: [PATCH 0430/2521] 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 69fb3060746..bad5c442ab0 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 +} From 85ab8114bc1367a0f4f32d4b8635c41aa547bc72 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 20:25:58 -0700 Subject: [PATCH 0431/2521] Moved non-serializable closure catching exception from submitStage to submitMissingTasks --- .../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 5fcd807aff2..fde998494f3 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 c3c52f9118b..5b133cdd6e8 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() } From d33b8a2a0ffd6d085cbd8de22863a1f35c106270 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 20:28:39 -0700 Subject: [PATCH 0432/2521] Added comments on task closure serialization. --- .../scala/spark/scheduler/cluster/ClusterTaskSetManager.scala | 2 ++ .../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 d934293b70e..f64818876b9 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 bbce9eda648..a9b49cad0e3 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") From 3dae1df66f34d3716c697f85cfe4aedeee428688 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 23 Jul 2013 20:25:58 -0700 Subject: [PATCH 0433/2521] Moved non-serializable closure catching exception from submitStage to submitMissingTasks --- .../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 5fcd807aff2..fde998494f3 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 c3c52f9118b..5b133cdd6e8 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() } From 503acd3a379a3686d343fdf072fc231b8fba78f9 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 09:47:27 +0800 Subject: [PATCH 0434/2521] Build metrics system framwork --- conf/metrics.properties | 13 ++++ .../metrics/AbstractInstrumentation.scala | 68 +++++++++++++++++++ .../scala/spark/metrics/MetricsConfig.scala | 55 +++++++++++++++ .../spark/metrics/sink/ConsoleSink.scala | 43 ++++++++++++ .../scala/spark/metrics/sink/CsvSink.scala | 53 +++++++++++++++ .../scala/spark/metrics/sink/JmxSink.scala | 17 +++++ .../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 00000000000..78749cf3815 --- /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 00000000000..0fed6084888 --- /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 00000000000..0fec1988ea3 --- /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 00000000000..5426af8c4c2 --- /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 00000000000..3a80c369014 --- /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 00000000000..56e56777001 --- /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 00000000000..65ebcb4eac7 --- /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 From 9dec8c73e6f0c3b6b55a11ff92cc9bff18dadd24 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 09:48:41 +0800 Subject: [PATCH 0435/2521] Add Master and Worker instrumentation support --- .../scala/spark/deploy/master/Master.scala | 9 ++- .../deploy/master/MasterInstrumentation.scala | 44 +++++++++++++++ .../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 eddcafd84d1..3a7c4e5a521 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 00000000000..13088189a4b --- /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 0bd88ea2537..b64bdb8d28c 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 00000000000..04c43ce33b8 --- /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 From c3daad3f65630eb4ed536d06c0d467cde57a8142 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 12:00:19 +0800 Subject: [PATCH 0436/2521] Update metric source support for instrumentation --- conf/metrics.properties | 4 ++- .../deploy/master/MasterInstrumentation.scala | 7 +++-- .../deploy/worker/WorkerInstrumentation.scala | 5 +++- .../metrics/AbstractInstrumentation.scala | 29 +++++++++++++++++-- .../scala/spark/metrics/MetricsConfig.scala | 12 ++++---- .../scala/spark/metrics/sink/CsvSink.scala | 4 +-- .../main/scala/spark/metrics/sink/Sink.scala | 1 + .../spark/metrics/source/JvmSource.scala | 17 +++++++++++ .../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 78749cf3815..0bbb6b5229e 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 13088189a4b..c295e725d71 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 04c43ce33b8..2f725300b5c 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 0fed6084888..9cae1e0220a 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 0fec1988ea3..be4f670918a 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 3a80c369014..1d663f6cffd 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 65ebcb4eac7..26052b7231b 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 00000000000..8f6bf48843a --- /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 00000000000..35cfe0c8ff1 --- /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 From 03f9871116801abbdd7b4c7892c8d6affb1c4d9e Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 27 Jun 2013 18:29:54 +0800 Subject: [PATCH 0437/2521] MetricsSystem refactor --- .../scala/spark/deploy/master/Master.scala | 17 ++- .../deploy/master/MasterInstrumentation.scala | 64 ++++----- .../scala/spark/deploy/worker/Worker.scala | 12 +- .../deploy/worker/WorkerInstrumentation.scala | 131 +++++++++++------- ...trumentation.scala => MetricsSystem.scala} | 54 +++++--- .../spark/metrics/sink/ConsoleSink.scala | 10 +- .../scala/spark/metrics/sink/CsvSink.scala | 10 +- .../scala/spark/metrics/sink/JmxSink.scala | 4 +- .../main/scala/spark/metrics/sink/Sink.scala | 4 +- .../spark/metrics/source/JvmSource.scala | 30 ++-- .../scala/spark/metrics/source/Source.scala | 6 +- 11 files changed, 193 insertions(+), 149 deletions(-) rename core/src/main/scala/spark/metrics/{AbstractInstrumentation.scala => MetricsSystem.scala} (64%) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 3a7c4e5a521..e44f5e31680 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 c295e725d71..5ea9a90319f 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 b64bdb8d28c..eaa1c1806f4 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 2f725300b5c..37fd1548599 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/MetricsSystem.scala similarity index 64% rename from core/src/main/scala/spark/metrics/AbstractInstrumentation.scala rename to core/src/main/scala/spark/metrics/MetricsSystem.scala index 9cae1e0220a..ea1bc490b54 100644 --- a/core/src/main/scala/spark/metrics/AbstractInstrumentation.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -2,7 +2,7 @@ package spark.metrics import scala.collection.mutable -import com.codahale.metrics.{JmxReporter, MetricRegistry} +import com.codahale.metrics.{JmxReporter, MetricSet, MetricRegistry} import java.util.Properties import java.util.concurrent.TimeUnit @@ -11,70 +11,76 @@ import spark.Logging import spark.metrics.sink._ import spark.metrics.source._ -private [spark] trait AbstractInstrumentation extends Logging { +private[spark] class MetricsSystem private (val instance: String) 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 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, AbstractInstrumentation.SOURCE_REGEX) + val sourceConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) - // Register all the sources + // Register all the sources related to instance sourceConfigs.foreach { kv => val classPath = kv._2.getProperty("class") try { - val source = Class.forName(classPath).getConstructor(classOf[MetricRegistry]) - .newInstance(registryHandler) + 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) } } - sources.foreach(_.registerSource) } def registerSinks() { val instConfig = metricsConfig.getInstance(instance) - val sinkConfigs = MetricsConfig.subProperties(instConfig, AbstractInstrumentation.SINK_REGEX) + val sinkConfigs = MetricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) // Register JMX sink as a default sink - sinks += new JmxSink(registryHandler) + sinks += new JmxSink(MetricsSystem.registry) // Register other sinks according to conf sinkConfigs.foreach { kv => - val classPath = if (AbstractInstrumentation.DEFAULT_SINKS.contains(kv._1)) { - AbstractInstrumentation.DEFAULT_SINKS(kv._1) + 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, registryHandler) + .newInstance(kv._2, MetricsSystem.registry) sinks += sink.asInstanceOf[Sink] } catch { case e: Exception => logError("sink class " + classPath + " cannot be instantialized", e) } } - sinks.foreach(_.registerSink) - } - - def unregisterSinks() { - sinks.foreach(_.unregisterSink) + sinks.foreach(_.start) } } -object AbstractInstrumentation { +private[spark] object MetricsSystem { + val registry = new MetricRegistry() + val DEFAULT_SINKS = Map( "console" -> "spark.metrics.sink.ConsoleSink", "csv" -> "spark.metrics.sink.CsvSink") @@ -88,4 +94,6 @@ object AbstractInstrumentation { "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 5426af8c4c2..b49a211fb3f 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 1d663f6cffd..3f572c8e051 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 56e56777001..e223dc26e94 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 26052b7231b..9fef894fde1 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 8f6bf48843a..7a7c1b6ffb6 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 35cfe0c8ff1..edd59de46a9 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 From 4d6dd67fa1f8f031f1ef46a442cec733fa3b1a7a Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Fri, 28 Jun 2013 02:26:55 +0800 Subject: [PATCH 0438/2521] 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 +++++-------------- .../scala/spark/metrics/MetricsSystem.scala | 23 ++--- .../spark/metrics/sink/ConsoleSink.scala | 11 +-- .../scala/spark/metrics/sink/CsvSink.scala | 9 +- .../scala/spark/metrics/sink/JmxSink.scala | 5 +- .../spark/metrics/source/JvmSource.scala | 22 ++--- .../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 5ea9a90319f..46c90b94d22 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 37fd1548599..5ce29cf04c8 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 ea1bc490b54..a23ccd26927 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 b49a211fb3f..9cd17556fa1 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 3f572c8e051..62e51be0dc3 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 e223dc26e94..98b55f7b7f5 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 7a7c1b6ffb6..13270dae3c6 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 edd59de46a9..17cbe2f85a0 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 +} From 7fb574bf666661fdf8a786de779f85efe2f15f0c Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 10:14:30 +0800 Subject: [PATCH 0439/2521] Code clean and remarshal --- .../scala/spark/deploy/master/Master.scala | 9 +++++---- .../deploy/master/MasterInstrumentation.scala | 11 ++--------- .../deploy/worker/WorkerInstrumentation.scala | 18 ++++++------------ .../scala/spark/metrics/MetricsSystem.scala | 5 ++--- .../scala/spark/metrics/sink/ConsoleSink.scala | 1 - .../scala/spark/metrics/sink/CsvSink.scala | 1 - .../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 e44f5e31680..cc0b2d42953 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 46c90b94d22..61a561c9557 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 5ce29cf04c8..94c20a98c18 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 a23ccd26927..5bfdc00eaff 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 9cd17556fa1..e2e4197d1dc 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 62e51be0dc3..c2d645331c2 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 9fef894fde1..3ffdcbdaba4 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 From 576528f0f916cc0434972cbbc9321e4dcec45628 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 10:25:48 +0800 Subject: [PATCH 0440/2521] 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 f3f67b57c89..825319d3de7 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) { From 871bc1687eaeb59df24b4778c5992a5f7f105cc8 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 14:46:24 +0800 Subject: [PATCH 0441/2521] Add Executor instrumentation --- .../executor/ExecutorInstrumentation.scala | 35 +++++++++++++++++++ .../spark/executor/MesosExecutorBackend.scala | 10 +++++- .../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 00000000000..80aadb66b0e --- /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 4961c42fadd..8b6ab0c3919 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 f4003da7329..6ef74cd2ff2 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)) } From 5ce5dc9fcd7acf5c58dd3d456a629b01d57514e4 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 28 Jun 2013 14:48:21 +0800 Subject: [PATCH 0442/2521] Add default properties to deal with no configure file situation --- .../scala/spark/metrics/MetricsConfig.scala | 28 ++++++++++++------- .../scala/spark/metrics/MetricsSystem.scala | 9 ++---- .../spark/metrics/sink/ConsoleSink.scala | 6 ++-- .../scala/spark/metrics/sink/CsvSink.scala | 6 ++-- .../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 be4f670918a..7405192058e 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 5bfdc00eaff..6e448cb2a57 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 e2e4197d1dc..d7b7a9e5015 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 c2d645331c2..e6c5bffd3c4 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 98b55f7b7f5..f097a631c0c 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() From e080588f7396d9612ea5d909e59f2364c139103a Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 1 Jul 2013 17:04:06 +0800 Subject: [PATCH 0443/2521] Add metrics system unit test --- .../spark/metrics/MetricsConfigSuite.scala | 92 +++++++++++++++++++ .../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 00000000000..0c7142c4182 --- /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 00000000000..5e8f8fcf804 --- /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 From e9ac88754d4c5d58aedd4de8768787300b15eada Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 1 Jul 2013 17:05:18 +0800 Subject: [PATCH 0444/2521] Remove twice add Source bug and code clean --- .../src/main/scala/spark/metrics/MetricsSystem.scala | 10 ++++------ .../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 6e448cb2a57..bf4487e0fc1 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 13270dae3c6..8f0870c1a0e 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) } From 7d2eada451686824bd467641bf1763e82011f2a6 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Mon, 1 Jul 2013 21:37:21 +0800 Subject: [PATCH 0445/2521] 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 ++++ .../scala/spark/scheduler/DAGScheduler.scala | 3 +- .../spark/scheduler/DAGSchedulerSource.scala | 31 ++++++++++++++++ .../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 24ba6056466..a6128a9f302 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 f2bdc11bdb2..d34dafecc53 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 9b45fc29388..781e49bdec3 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 00000000000..57aa74512ca --- /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 00000000000..c0ce9259c8e --- /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 + } + }) +} From 9cea0c28184c86625f8281eea2af77eff15acb73 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Mon, 1 Jul 2013 22:22:29 +0800 Subject: [PATCH 0446/2521] Refactor metricsSystem unit test, add resource files. --- .../resources/test_metrics_config.properties | 6 ++++ .../resources/test_metrics_system.properties | 7 +++++ .../spark/metrics/MetricsConfigSuite.scala | 30 ++---------------- .../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 00000000000..20119400031 --- /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 00000000000..06afbc66250 --- /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 0c7142c4182..f4c83cb6443 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 5e8f8fcf804..967be6ec470 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 + } +} From 5f8802c1fb106cc04c30b6aca0a6ce98fa5c0e15 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Tue, 2 Jul 2013 10:41:43 +0800 Subject: [PATCH 0447/2521] 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 a6128a9f302..1255d0c72e6 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 d34dafecc53..125dc55bd8f 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) } } From 1daff54b2ed92d0bcee7030d7d3ab5c274f80d2f Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 2 Jul 2013 11:28:32 +0800 Subject: [PATCH 0448/2521] 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 +++++- .../spark/executor/ExecutorInstrumentation.scala | 10 +++++----- .../scala/spark/executor/MesosExecutorBackend.scala | 10 +--------- .../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 1255d0c72e6..f1d9d5e4429 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 125dc55bd8f..7b3dc693486 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 2e811518828..7179ed84a8c 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 80aadb66b0e..ebbcbee7423 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 8b6ab0c3919..4961c42fadd 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 6ef74cd2ff2..f4003da7329 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)) } From a79f6077f0b3bc0110f6e8e15de3068f296f3b81 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 2 Jul 2013 11:29:26 +0800 Subject: [PATCH 0449/2521] 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 0bbb6b5229e..2ea3aaf3458 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 6329b2fbd88..237d9887127 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 7b3dc693486..204049d9271 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 eb7bd7e9df9..3fbd93c7dd5 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 From 5730193e0c8639b654f489c5956e31451d81b7db Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 2 Jul 2013 13:48:09 +0800 Subject: [PATCH 0450/2521] 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 +++++----- .../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 f1d9d5e4429..1e59a4d47d1 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 204049d9271..4a1d341f5d2 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 f097a631c0c..6a40885b781 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 8f0870c1a0e..79f505079c8 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 17cbe2f85a0..5607e2c40a7 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 781e49bdec3..9b45fc29388 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 825319d3de7..d4d70afdd56 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" ) ++ ( From ed1a3bc206c01974eedd0b1fb1deec183376b5c6 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Wed, 3 Jul 2013 20:11:10 +0800 Subject: [PATCH 0451/2521] continue to refactor code style and functions --- core/src/main/scala/spark/SparkContext.scala | 6 +- .../scala/spark/deploy/master/Master.scala | 6 +- .../deploy/master/MasterInstrumentation.scala | 19 ++-- .../deploy/worker/WorkerInstrumentation.scala | 35 ++++---- .../main/scala/spark/executor/Executor.scala | 8 +- .../executor/ExecutorInstrumentation.scala | 35 ++++---- .../scala/spark/metrics/MetricsConfig.scala | 88 +++++++++---------- .../scala/spark/metrics/MetricsSystem.scala | 40 ++++----- .../spark/metrics/sink/ConsoleSink.scala | 30 +++---- .../scala/spark/metrics/sink/CsvSink.scala | 40 ++++----- .../spark/scheduler/DAGSchedulerSource.scala | 23 +++-- .../spark/storage/BlockManagerSource.scala | 14 +-- .../spark/metrics/MetricsConfigSuite.scala | 32 ++++--- .../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 1e59a4d47d1..77cb0ee0cd6 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 cc0b2d42953..5f67366eb60 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 61a561c9557..4c3708cc4cc 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 94c20a98c18..c76c0b4711c 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 7179ed84a8c..4ea05dec1ca 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 ebbcbee7423..ad406f41b45 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 7405192058e..b1f6a1e596e 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 bf4487e0fc1..54f6c6e4dac 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 d7b7a9e5015..c67c0ee9123 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 e6c5bffd3c4..a8ca819e875 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 57aa74512ca..38158b8a2b9 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 c0ce9259c8e..f9648271024 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 f4c83cb6443..bb1be4f4fc3 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 967be6ec470..f29bb9db674 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) - } + } } From 5b4a2f2094258920613f50c0d337e1e92c08d012 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Wed, 3 Jul 2013 20:31:53 +0800 Subject: [PATCH 0452/2521] 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 2ea3aaf3458..00000000000 --- 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 00000000000..b4204e1deb7 --- /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 + From 05637de8423da85c5934cdfa8f07254133a58474 Mon Sep 17 00:00:00 2001 From: Andrew xia Date: Wed, 3 Jul 2013 21:16:39 +0800 Subject: [PATCH 0453/2521] Change class xxxInstrumentation to class xxxSource --- core/src/main/scala/spark/deploy/master/Master.scala | 4 ++-- .../{MasterInstrumentation.scala => MasterSource.scala} | 2 +- core/src/main/scala/spark/deploy/worker/Worker.scala | 4 ++-- .../{WorkerInstrumentation.scala => WorkerSource.scala} | 2 +- core/src/main/scala/spark/executor/Executor.scala | 4 ++-- .../{ExecutorInstrumentation.scala => ExecutorSource.scala} | 2 +- core/src/test/scala/spark/metrics/MetricsSystemSuite.scala | 2 +- 7 files changed, 10 insertions(+), 10 deletions(-) rename core/src/main/scala/spark/deploy/master/{MasterInstrumentation.scala => MasterSource.scala} (90%) rename core/src/main/scala/spark/deploy/worker/{WorkerInstrumentation.scala => WorkerSource.scala} (93%) rename core/src/main/scala/spark/executor/{ExecutorInstrumentation.scala => ExecutorSource.scala} (94%) diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 5f67366eb60..1d592206c00 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/MasterSource.scala similarity index 90% rename from core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala rename to core/src/main/scala/spark/deploy/master/MasterSource.scala index 4c3708cc4cc..f94e5b2c342 100644 --- a/core/src/main/scala/spark/deploy/master/MasterInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/master/MasterSource.scala @@ -4,7 +4,7 @@ import com.codahale.metrics.{Gauge,MetricRegistry} import spark.metrics.source.Source -private[spark] class MasterInstrumentation(val master: Master) extends Source { +private[spark] class MasterSource(val master: Master) extends Source { val metricRegistry = new MetricRegistry() val sourceName = "master" diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index eaa1c1806f4..5c0f77fd756 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/WorkerSource.scala similarity index 93% rename from core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala rename to core/src/main/scala/spark/deploy/worker/WorkerSource.scala index c76c0b4711c..539eac71bdc 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerInstrumentation.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerSource.scala @@ -4,7 +4,7 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source -private[spark] class WorkerInstrumentation(val worker: Worker) extends Source { +private[spark] class WorkerSource(val worker: Worker) extends Source { val sourceName = "worker" val metricRegistry = new MetricRegistry() diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 4ea05dec1ca..8a74a8d853a 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/ExecutorSource.scala similarity index 94% rename from core/src/main/scala/spark/executor/ExecutorInstrumentation.scala rename to core/src/main/scala/spark/executor/ExecutorSource.scala index ad406f41b45..d8b531cb58d 100644 --- a/core/src/main/scala/spark/executor/ExecutorInstrumentation.scala +++ b/core/src/main/scala/spark/executor/ExecutorSource.scala @@ -4,7 +4,7 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import spark.metrics.source.Source -class ExecutorInstrumentation(val executor: Executor) extends Source { +class ExecutorSource(val executor: Executor) extends Source { val metricRegistry = new MetricRegistry() val sourceName = "executor" diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala index f29bb9db674..462c28e8949 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) } From 8d1ef7f2dfc453137b8dbbb72a7f1ca93b57bb15 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 4 Jul 2013 10:11:30 +0800 Subject: [PATCH 0454/2521] Code style changes --- conf/metrics.properties.template | 2 ++ .../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 b4204e1deb7..c7e24aa36cb 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 f94e5b2c342..65c22320d60 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 b1f6a1e596e..5066b7ac22c 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 c67c0ee9123..437f24a5759 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" From 31ec72b243459e0d1d9c89033328e1d79f3aa76f Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 16 Jul 2013 10:57:19 +0800 Subject: [PATCH 0455/2521] Code refactor according to comments --- conf/metrics.properties.template | 73 +++++++++++++++- .../scala/spark/deploy/master/Master.scala | 11 +-- .../spark/deploy/master/MasterSource.scala | 2 +- .../scala/spark/deploy/worker/Worker.scala | 11 +-- .../spark/deploy/worker/WorkerSource.scala | 12 +-- .../scala/spark/executor/ExecutorSource.scala | 8 +- .../scala/spark/metrics/MetricsConfig.scala | 50 +++++++---- .../scala/spark/metrics/MetricsSystem.scala | 83 ++++++++++++++----- .../spark/metrics/sink/ConsoleSink.scala | 12 +-- .../scala/spark/metrics/sink/CsvSink.scala | 12 +-- .../spark/scheduler/DAGSchedulerSource.scala | 10 +-- .../spark/storage/BlockManagerSource.scala | 31 +++++-- .../resources/test_metrics_config.properties | 6 +- .../resources/test_metrics_system.properties | 4 +- .../spark/metrics/MetricsConfigSuite.scala | 38 ++++----- .../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 c7e24aa36cb..0486ca4c792 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 1d592206c00..9692af52953 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 65c22320d60..b8cfa6a7736 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 5c0f77fd756..8fa0d12b828 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 539eac71bdc..39cb8e56901 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 d8b531cb58d..94116edfcf8 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 5066b7ac22c..ed505b0aa73 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 54f6c6e4dac..2f87577ff33 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 437f24a5759..eaaac5d153a 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 a8ca819e875..aa5bff0d34d 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 38158b8a2b9..87d27cc70d7 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 f9648271024..4faa715c947 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 20119400031..2b31ddf2ebc 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 06afbc66250..d5479f02980 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 bb1be4f4fc3..87cd2ffad25 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 462c28e8949..c1899964172 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) } } From 93c6015f82d4d27a6f09686b1e849be1cbbd0615 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 24 Jul 2013 10:53:02 -0700 Subject: [PATCH 0456/2521] Shows task status and running tasks on Stage Page: fixes SPARK-804 and 811 --- .../spark/scheduler/cluster/TaskInfo.scala | 12 ++++++++++++ .../scala/spark/ui/exec/ExecutorsUI.scala | 5 +++++ .../scala/spark/ui/jobs/JobProgressUI.scala | 16 ++++++++++++++++ .../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 a1ebd48b013..f840100eca4 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 bad5c442ab0..db1c9029551 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 44dcf82d113..6e332415dbc 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 292966f23a0..b77b29045ea 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} From bd3931c87415679fef8b91390d50188c3da98430 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 24 Jul 2013 11:27:17 -0700 Subject: [PATCH 0457/2521] Changed ifs with returns to if/else --- .../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 f840100eca4..afcfe96a090 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 = { From 4280e1768d09e43b55f26556e14a8eb70d9b9fad Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 24 Jul 2013 12:48:48 -0700 Subject: [PATCH 0458/2521] 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 afcfe96a090..c693b722ac3 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" } From 57009eef9099aa3b35b9262d9637af3dc94266a1 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Wed, 24 Jul 2013 13:43:09 -0700 Subject: [PATCH 0459/2521] 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 b77b29045ea..654f3477235 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) { From 1d101928069c31aec039bb8b69c9bf4c204eead4 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Mon, 22 Jul 2013 13:11:35 -0700 Subject: [PATCH 0460/2521] 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 eb270a54281..e4ce1ca8481 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 f49aa927969..4cffda47089 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 From eef678703eed96544224209b1555618968b2eb3f Mon Sep 17 00:00:00 2001 From: Xinghao Date: Wed, 24 Jul 2013 15:32:50 -0700 Subject: [PATCH 0461/2521] Adding SVM and Lasso, moving LogisticRegression to classification from regression Also, add regularization parameter to SGD --- .../mllib/classification/Classification.scala | 21 +++ .../LogisticRegression.scala | 28 +-- .../LogisticRegressionGenerator.scala | 2 +- .../spark/mllib/classification/SVM.scala | 170 ++++++++++++++++++ .../mllib/classification/SVMGenerator.scala | 45 +++++ .../spark/mllib/optimization/Gradient.scala | 28 +++ .../mllib/optimization/GradientDescent.scala | 10 +- .../spark/mllib/optimization/Updater.scala | 31 +++- .../scala/spark/mllib/regression/Lasso.scala | 167 +++++++++++++++++ .../mllib/regression/LassoGenerator.scala | 44 +++++ .../LogisticRegressionSuite.scala | 2 +- .../spark/mllib/classification/SVMSuite.scala | 61 +++++++ .../spark/mllib/regression/LassoSuite.scala | 51 ++++++ 13 files changed, 642 insertions(+), 18 deletions(-) create mode 100644 mllib/src/main/scala/spark/mllib/classification/Classification.scala rename mllib/src/main/scala/spark/mllib/{regression => classification}/LogisticRegression.scala (84%) rename mllib/src/main/scala/spark/mllib/{regression => classification}/LogisticRegressionGenerator.scala (96%) create mode 100644 mllib/src/main/scala/spark/mllib/classification/SVM.scala create mode 100644 mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala create mode 100644 mllib/src/main/scala/spark/mllib/regression/Lasso.scala create mode 100644 mllib/src/main/scala/spark/mllib/regression/LassoGenerator.scala rename mllib/src/test/scala/spark/mllib/{regression => classification}/LogisticRegressionSuite.scala (97%) create mode 100644 mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala create mode 100644 mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala diff --git a/mllib/src/main/scala/spark/mllib/classification/Classification.scala b/mllib/src/main/scala/spark/mllib/classification/Classification.scala new file mode 100644 index 00000000000..7f1eb210792 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/classification/Classification.scala @@ -0,0 +1,21 @@ +package spark.mllib.classification + +import spark.RDD + +trait ClassificationModel { + /** + * 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/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala similarity index 84% rename from mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala rename to mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index e4db7bb9b73..f39c1ec52eb 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -1,4 +1,4 @@ -package spark.mllib.regression +package spark.mllib.classification import spark.{Logging, RDD, SparkContext} import spark.mllib.optimization._ @@ -13,7 +13,7 @@ import org.jblas.DoubleMatrix class LogisticRegressionModel( val weights: DoubleMatrix, val intercept: Double, - val losses: Array[Double]) extends RegressionModel { + val losses: Array[Double]) extends ClassificationModel { override def predict(testData: spark.RDD[Array[Double]]) = { testData.map { x => @@ -29,14 +29,14 @@ class LogisticRegressionModel( } } -class LogisticRegression private (var stepSize: Double, var miniBatchFraction: Double, +class LogisticRegression private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { /** * Construct a LogisticRegression object with default parameters */ - def this() = this(1.0, 1.0, 100) + def this() = this(1.0, 1.0, 1.0, 100) /** * Set the step size per-iteration of SGD. Default 1.0. @@ -69,7 +69,7 @@ class LogisticRegression private (var stepSize: Double, var miniBatchFraction: D } val (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, miniBatchFraction) + data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, regParam, miniBatchFraction) val weightsScaled = weights.getRange(1, weights.length) val intercept = weights.get(0) @@ -96,16 +96,18 @@ object LogisticRegression { * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. * @param miniBatchFraction Fraction of data to be used per iteration. */ def train( input: RDD[(Double, Array[Double])], numIterations: Int, stepSize: Double, + regParam: Double, miniBatchFraction: Double) : LogisticRegressionModel = { - new LogisticRegression(stepSize, miniBatchFraction, numIterations).train(input) + new LogisticRegression(stepSize, regParam, miniBatchFraction, numIterations).train(input) } /** @@ -115,16 +117,18 @@ object LogisticRegression { * * @param input RDD of (label, array of features) pairs. * @param stepSize Step size to be used for each iteration of Gradient Descent. + * @param regParam Regularization parameter. * @param numIterations Number of iterations of gradient descent to run. * @return a LogisticRegressionModel which has the weights and offset from training. */ def train( input: RDD[(Double, Array[Double])], numIterations: Int, - stepSize: Double) + stepSize: Double, + regParam: Double) : LogisticRegressionModel = { - train(input, numIterations, stepSize, 1.0) + train(input, numIterations, stepSize, regParam, 1.0) } /** @@ -141,17 +145,17 @@ object LogisticRegression { numIterations: Int) : LogisticRegressionModel = { - train(input, numIterations, 1.0, 1.0) + train(input, numIterations, 1.0, 1.0, 1.0) } def main(args: Array[String]) { - if (args.length != 4) { - println("Usage: LogisticRegression ") + if (args.length != 5) { + println("Usage: LogisticRegression ") System.exit(1) } val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = LogisticRegression.train(data, args(3).toInt, args(2).toDouble) + val model = LogisticRegression.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala similarity index 96% rename from mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala rename to mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala index 6e7c023bac7..cde1148adf4 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LogisticRegressionGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala @@ -1,4 +1,4 @@ -package spark.mllib.regression +package spark.mllib.classification import scala.util.Random diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala new file mode 100644 index 00000000000..aceb903f1d6 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -0,0 +1,170 @@ +package spark.mllib.classification + +import scala.math.signum +import spark.{Logging, RDD, SparkContext} +import spark.mllib.optimization._ +import spark.mllib.util.MLUtils + +import org.jblas.DoubleMatrix + +/** + * SVM using Stochastic Gradient Descent. + */ +class SVMModel( + val weights: DoubleMatrix, + val intercept: Double, + val losses: Array[Double]) extends ClassificationModel { + + override def predict(testData: spark.RDD[Array[Double]]) = { + testData.map { x => { + println("Predicting " + x) + signum(new DoubleMatrix(1, x.length, x:_*).dot(this.weights) + this.intercept) + } + } + } + + override def predict(testData: Array[Double]): Double = { + val dataMat = new DoubleMatrix(1, testData.length, testData:_*) + signum(dataMat.dot(this.weights) + this.intercept) + } +} + +class SVM private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, + var numIters: Int) + extends Logging { + + /** + * Construct a SVM object with default parameters + */ + def this() = this(1.0, 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 the regularization parameter. Default 1.0. + */ + def setRegParam(param: Double) = { + this.regParam = param + 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])]): SVMModel = { + // 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 HingeGradient(), new SquaredL2Updater(), stepSize, numIters, regParam, miniBatchFraction) + + val weightsScaled = weights.getRange(1, weights.length) + val intercept = weights.get(0) + + val model = new SVMModel(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 SVM. + */ +object SVM { + + /** + * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double) + : SVMModel = + { + new SVM(stepSize, regParam, miniBatchFraction, numIterations).train(input) + } + + /** + * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. We use the entire data set to update + * the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param stepSize Step size to be used for each iteration of Gradient Descent. + * @param regParam Regularization parameter. + * @param numIterations Number of iterations of gradient descent to run. + * @return a SVMModel which has the weights and offset from training. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + regParam: Double) + : SVMModel = + { + train(input, numIterations, stepSize, regParam, 1.0) + } + + /** + * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using a step size of 1.0. We use the entire data set to update + * the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @return a SVMModel which has the weights and offset from training. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int) + : SVMModel = + { + train(input, numIterations, 1.0, 0.10, 1.0) + } + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: SVM ") + System.exit(1) + } + val sc = new SparkContext(args(0), "SVM") + val data = MLUtils.loadLabeledData(sc, args(1)) + val model = SVM.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala b/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala new file mode 100644 index 00000000000..a5e28373435 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala @@ -0,0 +1,45 @@ +package spark.mllib.classification + +import scala.util.Random +import scala.math.signum + +import org.jblas.DoubleMatrix + +import spark.{RDD, SparkContext} +import spark.mllib.util.MLUtils + +object LassoGenerator { + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: LassoGenerator " + + " ") + 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, "LassoGenerator") + + val globalRnd = new Random(94720) + val trueWeights = Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() } + + val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => + val rnd = new Random(42 + idx) + + val x = Array.fill[Double](nfeatures) { + rnd.nextDouble() * 2.0 - 1.0 + } + val y = signum(((1.0 +: x) zip trueWeights).map{wx => wx._1 * wx._2}.reduceLeft(_+_) + rnd.nextGaussian() * 0.1) + (y, x) + } + + MLUtils.saveLabeledData(data, outputPath) + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala index 90b0999a5ec..6ffc3b128b0 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala @@ -31,3 +31,31 @@ class LogisticGradient extends Gradient { (gradient, loss) } } + + +class SquaredGradient extends Gradient { + override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): + (DoubleMatrix, Double) = { + val diff: Double = data.dot(weights) - label + + val loss = 0.5 * diff * diff + val gradient = data.mul(diff) + + (gradient, loss) + } +} + + +class HingeGradient extends Gradient { + override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): + (DoubleMatrix, Double) = { + + val dotProduct = data.dot(weights) + + if (1.0 > label * dotProduct) + (data.mul(-label), 1.0 - label * dotProduct) + else + (DoubleMatrix.zeros(1,weights.length), 0.0) + } +} + diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index eff853f379b..bd8489c3864 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -19,6 +19,7 @@ object GradientDescent { * @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 regParam - regularization parameter * @param miniBatchFraction - fraction of the input data set that should be used for * one iteration of SGD. Default value 1.0. * @@ -31,6 +32,7 @@ object GradientDescent { updater: Updater, stepSize: Double, numIters: Int, + regParam: Double, miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { val lossHistory = new ArrayBuffer[Double](numIters) @@ -51,10 +53,14 @@ object GradientDescent { (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) + val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i, regParam) weights = update._1 reg_val = update._2 + lossHistory.append(lossSum / miniBatchSize + reg_val) + /*** + Xinghao: The loss here is sum of lossSum computed using the weights before applying updater, + and reg_val using weights after applying updater + ***/ } (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 index ea80bfcbfd4..64c54dfb0d5 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -1,5 +1,6 @@ package spark.mllib.optimization +import scala.math._ import org.jblas.DoubleMatrix abstract class Updater extends Serializable { @@ -10,18 +11,44 @@ abstract class Updater extends Serializable { * @param gradient - Column matrix of size nx1 where n is the number of features. * @param stepSize - step size across iterations * @param iter - Iteration number + * @param regParam - Regularization parameter * * @return weightsNew - Column matrix containing updated weights * @return reg_val - regularization value */ - def compute(weightsOlds: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int): + def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) } class SimpleUpdater extends Updater { override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, - stepSize: Double, iter: Int): (DoubleMatrix, Double) = { + stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { val normGradient = gradient.mul(stepSize / math.sqrt(iter)) (weightsOld.sub(normGradient), 0) } } + +class L1Updater extends Updater { + override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, + stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { + val thisIterStepSize = stepSize / math.sqrt(iter) + val normGradient = gradient.mul(thisIterStepSize) + val newWeights = weightsOld.sub(normGradient) + (0 until newWeights.length).foreach(i => { + val wi = newWeights.get(i) + newWeights.put(i, signum(wi) * max(0.0, abs(wi) - regParam * thisIterStepSize)) + }) + (newWeights, newWeights.norm1 * regParam) + } +} + +class SquaredL2Updater extends Updater { + override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, + stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { + val thisIterStepSize = stepSize / math.sqrt(iter) + val normGradient = gradient.mul(thisIterStepSize) + val newWeights = weightsOld.sub(normGradient).div(2.0 * thisIterStepSize * regParam + 1.0) + (newWeights, pow(newWeights.norm2,2.0) * regParam) + } +} + diff --git a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala new file mode 100644 index 00000000000..de410711a20 --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala @@ -0,0 +1,167 @@ +package spark.mllib.regression + +import spark.{Logging, RDD, SparkContext} +import spark.mllib.optimization._ +import spark.mllib.util.MLUtils + +import org.jblas.DoubleMatrix + +/** + * Lasso using Stochastic Gradient Descent. + */ +class LassoModel( + val weights: DoubleMatrix, + val intercept: Double, + val losses: Array[Double]) extends RegressionModel { + + override def predict(testData: spark.RDD[Array[Double]]) = { + testData.map { x => + new DoubleMatrix(1, x.length, x:_*).dot(this.weights) + this.intercept + } + } + + override def predict(testData: Array[Double]): Double = { + val dataMat = new DoubleMatrix(1, testData.length, testData:_*) + dataMat.dot(this.weights) + this.intercept + } +} + +class Lasso private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, + var numIters: Int) + extends Logging { + + /** + * Construct a Lasso object with default parameters + */ + def this() = this(1.0, 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 the regularization parameter. Default 1.0. + */ + def setRegParam(param: Double) = { + this.regParam = param + 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])]): LassoModel = { + // 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 SquaredGradient(), new L1Updater(), stepSize, numIters, regParam, miniBatchFraction) + + val weightsScaled = weights.getRange(1, weights.length) + val intercept = weights.get(0) + + val model = new LassoModel(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 Lasso. + */ +object Lasso { + + /** + * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double) + : LassoModel = + { + new Lasso(stepSize, regParam, miniBatchFraction, numIterations).train(input) + } + + /** + * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. We use the entire data set to update + * the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param stepSize Step size to be used for each iteration of Gradient Descent. + * @param regParam Regularization parameter. + * @param numIterations Number of iterations of gradient descent to run. + * @return a LassoModel which has the weights and offset from training. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + regParam: Double) + : LassoModel = + { + train(input, numIterations, stepSize, regParam, 1.0) + } + + /** + * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using a step size of 1.0. We use the entire data set to update + * the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @return a LassoModel which has the weights and offset from training. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int) + : LassoModel = + { + train(input, numIterations, 1.0, 0.10, 1.0) + } + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: Lasso ") + System.exit(1) + } + val sc = new SparkContext(args(0), "Lasso") + val data = MLUtils.loadLabeledData(sc, args(1)) + val model = Lasso.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/mllib/regression/LassoGenerator.scala b/mllib/src/main/scala/spark/mllib/regression/LassoGenerator.scala new file mode 100644 index 00000000000..d2d3bb33c7d --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/regression/LassoGenerator.scala @@ -0,0 +1,44 @@ +package spark.mllib.regression + +import scala.util.Random + +import org.jblas.DoubleMatrix + +import spark.{RDD, SparkContext} +import spark.mllib.util.MLUtils + +object LassoGenerator { + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: LassoGenerator " + + " ") + 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, "LassoGenerator") + + val globalRnd = new Random(94720) + val trueWeights = Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() } + + val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => + val rnd = new Random(42 + idx) + + val x = Array.fill[Double](nfeatures) { + rnd.nextDouble() * 2.0 - 1.0 + } + val y = ((1.0 +: x) zip trueWeights).map{wx => wx._1 * wx._2}.reduceLeft(_+_) + rnd.nextGaussian() * 0.1 + (y, x) + } + + MLUtils.saveLabeledData(data, outputPath) + sc.stop() + } +} diff --git a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala similarity index 97% rename from mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala rename to mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index 04d3400cb42..13612e9a4a9 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -1,4 +1,4 @@ -package spark.mllib.regression +package spark.mllib.classification import scala.util.Random diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala new file mode 100644 index 00000000000..e3a6681ab28 --- /dev/null +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -0,0 +1,61 @@ +package spark.mllib.classification + +import scala.util.Random +import scala.math.signum + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import spark.SparkContext +import spark.SparkContext._ + +import java.io._ + +class SVMSuite extends FunSuite with BeforeAndAfterAll { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + test("SVM") { + val nPoints = 10000 + val rnd = new Random(42) + + val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + val x2 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + + val A = 2.0 + val B = -1.5 + val C = 1.0 + + val y = (0 until nPoints).map { i => + signum(A + B * x1(i) + C * x2(i) + 0.0*rnd.nextGaussian()) + } + + val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i),x2(i)))).toArray + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + val writer_data = new PrintWriter(new File("svmtest.dat")) + testData.foreach(yx => { + writer_data.write(yx._1 + "") + yx._2.foreach(xi => writer_data.write("\t" + xi)) + writer_data.write("\n")}) + writer_data.close() + + val svm = new SVM().setStepSize(1.0) + .setRegParam(1.0) + .setNumIterations(100) + + val model = svm.train(testRDD) + + val yPredict = (0 until nPoints).map(i => model.predict(Array(x1(i),x2(i)))) + + val accuracy = ((y zip yPredict).map(yy => if (yy._1==yy._2) 1 else 0).reduceLeft(_+_).toDouble / nPoints.toDouble) + + assert(accuracy >= 0.90, "Accuracy (" + accuracy + ") too low") + } +} diff --git a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala new file mode 100644 index 00000000000..90fedb3e84b --- /dev/null +++ b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala @@ -0,0 +1,51 @@ +package spark.mllib.regression + +import scala.util.Random + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import spark.SparkContext +import spark.SparkContext._ + + +class LassoSuite extends FunSuite with BeforeAndAfterAll { + val sc = new SparkContext("local", "test") + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + test("Lasso") { + val nPoints = 10000 + val rnd = new Random(42) + + val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + val x2 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + + val A = 2.0 + val B = -1.5 + val C = 1.0e-2 + + val y = (0 until nPoints).map { i => + A + B * x1(i) + C * x2(i) + 0.1*rnd.nextGaussian() + } + + val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i),x2(i)))).toArray + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + val ls = new Lasso().setStepSize(1.0) + .setRegParam(0.01) + .setNumIterations(20) + + val model = ls.train(testRDD) + + val weight0 = model.weights.get(0) + val weight1 = model.weights.get(1) + assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") + assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]") + assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + } +} From c258718606a2960649dde0a4925fcf385d617c37 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 24 Jul 2013 16:12:32 -0700 Subject: [PATCH 0462/2521] 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 1fe288b92cf..4cf9589b07e 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 3dfba5808c9..1125aba4f17 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 + + + + + + From 8e0939f5a9c0823d51d0e94185b55d4a02628a53 Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 20:43:57 -0700 Subject: [PATCH 0463/2521] refactor Kryo serializer support to use chill/chill-java --- .../src/main/scala/spark/KryoSerializer.scala | 143 ++++-------------- .../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 ee37da79483..2ea90d41c34 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 c3323dcbb33..cb59bb9b657 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 d4d70afdd56..9920e00a674 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) { From a1c515fb02646c857c607949120c7d3cb29f4dce Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 20:50:32 -0700 Subject: [PATCH 0464/2521] add copyright back in --- .../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 cb59bb9b657..f6e3eec5466 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 From fc4b025314232fbeeaacb84ec3e2491e9c4ddd1b Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 20:53:15 -0700 Subject: [PATCH 0465/2521] 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 f6e3eec5466..30d2d5282bc 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") { From 30a369a808cdc16effe9c5755c74fc5a3f3d7cac Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 20:55:48 -0700 Subject: [PATCH 0466/2521] update pom.xml --- pom.xml | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 4cf9589b07e..469dc64329c 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 From e56aa75de0f3c00e9942f0863c0fb8c57aab5321 Mon Sep 17 00:00:00 2001 From: ryanlecompte Date: Wed, 24 Jul 2013 22:08:09 -0700 Subject: [PATCH 0467/2521] 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 2ea90d41c34..eeb2993d8a5 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 = { From e2421c13117c7d9cc950da8c9de4903d0ea49c43 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 25 Jul 2013 00:05:43 -0700 Subject: [PATCH 0468/2521] 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 237d9887127..8c740e3887e 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 From 8eb8b52997da56acb3500fa48aac1ab9a3dfdddf Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 25 Jul 2013 08:58:02 -0700 Subject: [PATCH 0469/2521] 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 8c740e3887e..f0c936c86ac 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 469dc64329c..44729bd4225 100644 --- a/pom.xml +++ b/pom.xml @@ -194,7 +194,7 @@ com.twitter - chill + chill_2.9.3 0.3.0 From a6de90c927e5569e30a7bc6188253bd14c32c3a9 Mon Sep 17 00:00:00 2001 From: Charles Reiss Date: Thu, 25 Jul 2013 01:01:50 -0700 Subject: [PATCH 0470/2521] For standalone mode, get JAVA_HOME, SPARK_JAVA_OPTS, SPARK_LIBRARY_PATH from application env, not worker env --- .../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 8b51ff1c3a4..47d3390928b 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 From d4bbc8bd251937a778658e09791b0416fd54336e Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 25 Jul 2013 15:59:52 -0700 Subject: [PATCH 0471/2521] Shows totals for shuffle data and CPU time in Stage, homepage overviews including active time --- .../scala/spark/ui/exec/ExecutorsUI.scala | 10 ++--- .../main/scala/spark/ui/jobs/IndexPage.scala | 28 +++++++++++++- .../scala/spark/ui/jobs/JobProgressUI.scala | 38 ++++++++++++++++--- .../main/scala/spark/ui/jobs/StagePage.scala | 26 ++++++++++++- 4 files changed, 90 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index db1c9029551..e569c041952 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -113,7 +113,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { } private[spark] class ExecutorsListener extends SparkListener with Logging { - val executorToTasksActive = HashMap[String, HashSet[Long]]() + val executorToTasksActive = HashMap[String, HashSet[TaskInfo]]() val executorToTasksComplete = HashMap[String, Int]() val executorToTasksFailed = HashMap[String, Int]() val executorToTaskInfos = @@ -122,8 +122,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = taskStart.taskInfo.executorId if (!executorToTasksActive.contains(eid)) - executorToTasksActive(eid) = HashSet[Long]() - executorToTasksActive(eid) += taskStart.taskInfo.taskId + executorToTasksActive(eid) = HashSet[TaskInfo]() + executorToTasksActive(eid) += taskStart.taskInfo val taskList = executorToTaskInfos.getOrElse( eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList += ((taskStart.taskInfo, None, None)) @@ -133,8 +133,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId if (!executorToTasksActive.contains(eid)) - executorToTasksActive(eid) = HashSet[Long]() - executorToTasksActive(eid) -= taskEnd.taskInfo.taskId + executorToTasksActive(eid) = HashSet[TaskInfo]() + executorToTasksActive(eid) -= taskEnd.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index f31af3cda6e..41186fd9bce 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -28,6 +28,7 @@ import spark.scheduler.Stage import spark.ui.UIUtils._ import spark.ui.Page._ import spark.storage.StorageLevel +import spark.Utils /** Page showing list of all ongoing and recently finished stages */ private[spark] class IndexPage(parent: JobProgressUI) { @@ -39,6 +40,13 @@ private[spark] class IndexPage(parent: JobProgressUI) { val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq + var activeTime = 0L + listener.stageToTasksActive.foreach {s => + s._2.foreach { t => + activeTime += t.timeRunning(System.currentTimeMillis()) + } + } + /** Special table which merges two header cells. */ def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { @@ -57,11 +65,29 @@ private[spark] class IndexPage(parent: JobProgressUI) {
        } + val summary: NodeSeq = +
        +
          +
        • + CPU time: + {parent.formatDuration(listener.totalTime + activeTime)} +
        • +
        • + Shuffle read: + {Utils.memoryBytesToString(listener.totalShuffleRead)} +
        • +
        • + Shuffle write: + {Utils.memoryBytesToString(listener.totalShuffleWrite)} +
        • +
        +
        val activeStageTable: NodeSeq = stageTable(stageRow, activeStages) val completedStageTable = stageTable(stageRow, completedStages) val failedStageTable: NodeSeq = stageTable(stageRow, failedStages) - val content =

        Active Stages

        ++ activeStageTable ++ + val content = summary ++ +

        Active Stages

        ++ activeStageTable ++

        Completed Stages

        ++ completedStageTable ++

        Failed Stages

        ++ failedStageTable diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 6e332415dbc..3cc38ea4036 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -65,7 +65,14 @@ private[spark] class JobProgressListener extends SparkListener { val completedStages = ListBuffer[Stage]() val failedStages = ListBuffer[Stage]() - val stageToTasksActive = HashMap[Int, HashSet[Long]]() + var totalTime = 0L + var totalShuffleRead = 0L + var totalShuffleWrite = 0L + + val stageToTime = HashMap[Int, Long]() + val stageToShuffleRead = HashMap[Int, Long]() + val stageToShuffleWrite = HashMap[Int, Long]() + val stageToTasksActive = HashMap[Int, HashSet[TaskInfo]]() val stageToTasksComplete = HashMap[Int, Int]() val stageToTasksFailed = HashMap[Int, Int]() val stageToTaskInfos = @@ -97,8 +104,8 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { val sid = taskStart.task.stageId if (!stageToTasksActive.contains(sid)) - stageToTasksActive(sid) = HashSet[Long]() - stageToTasksActive(sid) += taskStart.taskInfo.taskId + stageToTasksActive(sid) = HashSet[TaskInfo]() + stageToTasksActive(sid) += taskStart.taskInfo val taskList = stageToTaskInfos.getOrElse( sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList += ((taskStart.taskInfo, None, None)) @@ -108,8 +115,8 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId if (!stageToTasksActive.contains(sid)) - stageToTasksActive(sid) = HashSet[Long]() - stageToTasksActive(sid) -= taskEnd.taskInfo.taskId + stageToTasksActive(sid) = HashSet[TaskInfo]() + stageToTasksActive(sid) -= taskEnd.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => @@ -119,6 +126,27 @@ private[spark] class JobProgressListener extends SparkListener { stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 (None, Some(taskEnd.taskMetrics)) } + + if (!stageToTime.contains(sid)) + stageToTime(sid) = 0L + val time = metrics.map(m => m.executorRunTime).getOrElse(1) + stageToTime(sid) += time + totalTime += time + + if (!stageToShuffleRead.contains(sid)) + stageToShuffleRead(sid) = 0L + val shuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => + s.remoteBytesRead).getOrElse(0L) + stageToShuffleRead(sid) += shuffleRead + totalShuffleRead += shuffleRead + + if (!stageToShuffleWrite.contains(sid)) + stageToShuffleWrite(sid) = 0L + val shuffleWrite = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => + s.shuffleBytesWritten).getOrElse(0L) + stageToShuffleWrite(sid) += shuffleWrite + totalShuffleWrite += shuffleWrite + val taskList = stageToTaskInfos.getOrElse( sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList -= ((taskEnd.taskInfo, None, None)) diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 654f3477235..41d12b1ef4d 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -52,6 +52,29 @@ private[spark] class StagePage(parent: JobProgressUI) { val shuffleRead = listener.hasShuffleRead(stageId) val shuffleWrite = listener.hasShuffleWrite(stageId) + var activeTime = 0L + listener.stageToTasksActive(stageId).foreach { t => + activeTime += t.timeRunning(System.currentTimeMillis()) + } + + val summary = +
        +
          +
        • + CPU time: + {parent.formatDuration(listener.stageToTime(stageId) + activeTime)} +
        • +
        • + Shuffle read: + {Utils.memoryBytesToString(listener.stageToShuffleRead(stageId))} +
        • +
        • + Shuffle write: + {Utils.memoryBytesToString(listener.stageToShuffleWrite(stageId))} +
        • +
        +
        + val taskHeaders: Seq[String] = Seq("Task ID", "Status", "Duration", "Locality Level", "Worker", "Launch Time") ++ {if (shuffleRead) Seq("Shuffle Read") else Nil} ++ @@ -98,7 +121,8 @@ private[spark] class StagePage(parent: JobProgressUI) { } val content = -

        Summary Metrics

        ++ summaryTable.getOrElse(Nil) ++

        Tasks

        ++ taskTable; + summary ++

        Summary Metrics

        ++ summaryTable.getOrElse(Nil) ++ +

        Tasks

        ++ taskTable; headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId), Jobs) } From 22faeab261302d1506321993ecacf9f9ab9aa5f6 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Thu, 25 Jul 2013 17:14:18 -0700 Subject: [PATCH 0472/2521] Split Shuffle Activity overview column for read/write --- .../main/scala/spark/ui/jobs/IndexPage.scala | 25 ++++++++++++------- 1 file changed, 16 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 41186fd9bce..9bfb9b90417 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -56,7 +56,8 @@ private[spark] class IndexPage(parent: JobProgressUI) { Submitted Duration Tasks: Complete/Total - Shuffle Activity + Shuffle Read + Shuffle Write Stored RDD @@ -120,13 +121,18 @@ private[spark] 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 shuffleInfo = (read, write) match { - case (true, true) => "Read/Write" - case (true, false) => "Read" - case (false, true) => "Write" - case _ => "" - } + + val shuffleRead = + if (!listener.hasShuffleRead(s.id)) + "" + else + Utils.memoryBytesToString(listener.stageToShuffleRead(s.id)) + val shuffleWrite = + if (!listener.hasShuffleWrite(s.id)) + "" + else + Utils.memoryBytesToString(listener.stageToShuffleWrite(s.id)) + val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) val totalTasks = s.numPartitions @@ -143,7 +149,8 @@ private[spark] class IndexPage(parent: JobProgressUI) { case _ => }} - {shuffleInfo} + {shuffleRead} + {shuffleWrite} {if (s.rdd.getStorageLevel != StorageLevel.NONE) { {Option(s.rdd.name).getOrElse(s.rdd.id)} From 3fbe9eaac08c13527e64e4fe1ce2748838707562 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 26 Jul 2013 11:51:38 -0700 Subject: [PATCH 0473/2521] Displys shuffle read/write only if exists, wraps if statements, trims old vals, grabs current time once --- .../scala/spark/ui/exec/ExecutorsUI.scala | 6 ++- .../main/scala/spark/ui/jobs/IndexPage.scala | 27 ++++++------ .../scala/spark/ui/jobs/JobProgressUI.scala | 41 ++++++++++++++----- .../main/scala/spark/ui/jobs/StagePage.scala | 20 +++++---- 4 files changed, 62 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index e569c041952..b15da81f7cc 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -121,8 +121,9 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = taskStart.taskInfo.executorId - if (!executorToTasksActive.contains(eid)) + if (!executorToTasksActive.contains(eid)) { executorToTasksActive(eid) = HashSet[TaskInfo]() + } executorToTasksActive(eid) += taskStart.taskInfo val taskList = executorToTaskInfos.getOrElse( eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) @@ -132,8 +133,9 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId - if (!executorToTasksActive.contains(eid)) + if (!executorToTasksActive.contains(eid)) { executorToTasksActive(eid) = HashSet[TaskInfo]() + } executorToTasksActive(eid) -= taskEnd.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 9bfb9b90417..b862c3539c9 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -39,12 +39,11 @@ private[spark] class IndexPage(parent: JobProgressUI) { val activeStages = listener.activeStages.toSeq val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq + val now = System.currentTimeMillis() var activeTime = 0L - listener.stageToTasksActive.foreach {s => - s._2.foreach { t => - activeTime += t.timeRunning(System.currentTimeMillis()) - } + for (tasks <- listener.stageToTasksActive.values; t <- tasks) { + activeTime += t.timeRunning(now) } /** Special table which merges two header cells. */ @@ -73,14 +72,18 @@ private[spark] class IndexPage(parent: JobProgressUI) { CPU time: {parent.formatDuration(listener.totalTime + activeTime)} -
      • - Shuffle read: - {Utils.memoryBytesToString(listener.totalShuffleRead)} -
      • -
      • - Shuffle write: - {Utils.memoryBytesToString(listener.totalShuffleWrite)} -
      • + {if (listener.totalShuffleRead > 0) +
      • + Shuffle read: + {Utils.memoryBytesToString(listener.totalShuffleRead)} +
      • + } + {if (listener.totalShuffleWrite > 0) +
      • + Shuffle write: + {Utils.memoryBytesToString(listener.totalShuffleWrite)} +
      • + }
      val activeStageTable: NodeSeq = stageTable(stageRow, activeStages) diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 3cc38ea4036..2fb4575551a 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -93,6 +93,12 @@ private[spark] class JobProgressListener extends SparkListener { val toRemove = RETAINED_STAGES / 10 stages.takeRight(toRemove).foreach( s => { stageToTaskInfos.remove(s.id) + stageToTime.remove(s.id) + stageToShuffleRead.remove(s.id) + stageToShuffleWrite.remove(s.id) + stageToTasksActive.remove(s.id) + stageToTasksComplete.remove(s.id) + stageToTasksFailed.remove(s.id) }) stages.trimEnd(toRemove) } @@ -103,8 +109,9 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { val sid = taskStart.task.stageId - if (!stageToTasksActive.contains(sid)) + if (!stageToTasksActive.contains(sid)) { stageToTasksActive(sid) = HashSet[TaskInfo]() + } stageToTasksActive(sid) += taskStart.taskInfo val taskList = stageToTaskInfos.getOrElse( sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) @@ -114,8 +121,9 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId - if (!stageToTasksActive.contains(sid)) + if (!stageToTasksActive.contains(sid)) { stageToTasksActive(sid) = HashSet[TaskInfo]() + } stageToTasksActive(sid) -= taskEnd.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { @@ -127,23 +135,36 @@ private[spark] class JobProgressListener extends SparkListener { (None, Some(taskEnd.taskMetrics)) } - if (!stageToTime.contains(sid)) + if (!stageToTime.contains(sid)) { stageToTime(sid) = 0L - val time = metrics.map(m => m.executorRunTime).getOrElse(1) + } + val time = if (metrics.isDefined) metrics.map(m => m.executorRunTime).getOrElse(0) else 0 stageToTime(sid) += time totalTime += time - if (!stageToShuffleRead.contains(sid)) + if (!stageToShuffleRead.contains(sid)) { stageToShuffleRead(sid) = 0L - val shuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => - s.remoteBytesRead).getOrElse(0L) + } + val shuffleRead = + if (metrics.isDefined) { + metrics.flatMap(m => m.shuffleReadMetrics).map(s => s.remoteBytesRead).getOrElse(0L) + } + else { + 0L + } stageToShuffleRead(sid) += shuffleRead totalShuffleRead += shuffleRead - if (!stageToShuffleWrite.contains(sid)) + if (!stageToShuffleWrite.contains(sid)) { stageToShuffleWrite(sid) = 0L - val shuffleWrite = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => - s.shuffleBytesWritten).getOrElse(0L) + } + val shuffleWrite = + if (metrics.isDefined) { + metrics.flatMap(m => m.shuffleWriteMetrics).map(s => s.shuffleBytesWritten).getOrElse(0L) + } + else { + 0L + } stageToShuffleWrite(sid) += shuffleWrite totalShuffleWrite += shuffleWrite diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 41d12b1ef4d..266f180a500 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -64,14 +64,18 @@ private[spark] class StagePage(parent: JobProgressUI) { CPU time: {parent.formatDuration(listener.stageToTime(stageId) + activeTime)} -
    • - Shuffle read: - {Utils.memoryBytesToString(listener.stageToShuffleRead(stageId))} -
    • -
    • - Shuffle write: - {Utils.memoryBytesToString(listener.stageToShuffleWrite(stageId))} -
    • + {if (listener.hasShuffleRead(stageId)) +
    • + Shuffle read: + {Utils.memoryBytesToString(listener.stageToShuffleRead(stageId))} +
    • + } + {if (listener.hasShuffleWrite(stageId)) +
    • + Shuffle write: + {Utils.memoryBytesToString(listener.stageToShuffleWrite(stageId))} +
    • + }
    From 743fc4e7aa8a2ca4edbe731bbefb2127d5d1a7d4 Mon Sep 17 00:00:00 2001 From: harshars Date: Fri, 26 Jul 2013 14:35:17 -0700 Subject: [PATCH 0474/2521] Fix Bug in Partition Pruning, index of Pruned Partitions should inherit from parent --- .../scala/spark/rdd/PartitionPruningRDD.scala | 6 ++-- .../spark/PartitionPruningRDDSuite.scala | 28 +++++++++++++++++++ 2 files changed, 32 insertions(+), 2 deletions(-) create mode 100644 core/src/test/scala/spark/PartitionPruningRDDSuite.scala diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index 41ff62dd228..6fe004a0098 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -16,8 +16,9 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo extends NarrowDependency[T](rdd) { @transient - val partitions: Array[Partition] = rdd.partitions.filter(s => partitionFilterFunc(s.index)) - .zipWithIndex.map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } + val partitions: Array[Partition] = rdd.partitions. + zipWithIndex.filter(s => partitionFilterFunc(s._2)). + map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } override def getParents(partitionId: Int) = List(partitions(partitionId).index) } @@ -39,6 +40,7 @@ class PartitionPruningRDD[T: ClassManifest]( override protected def getPartitions: Array[Partition] = getDependencies.head.asInstanceOf[PruneDependency[T]].partitions + } diff --git a/core/src/test/scala/spark/PartitionPruningRDDSuite.scala b/core/src/test/scala/spark/PartitionPruningRDDSuite.scala new file mode 100644 index 00000000000..a0e64131602 --- /dev/null +++ b/core/src/test/scala/spark/PartitionPruningRDDSuite.scala @@ -0,0 +1,28 @@ +package spark + +import org.scalatest.FunSuite +import spark.SparkContext._ +import spark.rdd.PartitionPruningRDD + + +class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { + + test("Pruned Partitions inherit locality prefs correctly") { + class TestPartition(i: Int) extends Partition { + def index = i + } + val rdd = new RDD[Int](sc, Nil) { + override protected def getPartitions = { + Array[Partition]( + new TestPartition(1), + new TestPartition(2), + new TestPartition(3)) + } + def compute(split: Partition, context: TaskContext) = {Iterator()} + } + val prunedRDD = PartitionPruningRDD.create(rdd, {x => if (x==2) true else false}) + println(prunedRDD.partitions.length) + val p = prunedRDD.partitions(0) + assert(p.index == 2) + } +} \ No newline at end of file From 822aac8f5afd1342bb1cbadd7a2c2821bda81988 Mon Sep 17 00:00:00 2001 From: harshars Date: Fri, 26 Jul 2013 15:10:32 -0700 Subject: [PATCH 0475/2521] Indentation --- core/src/main/scala/spark/rdd/PartitionPruningRDD.scala | 1 - core/src/test/scala/spark/PartitionPruningRDDSuite.scala | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index 6fe004a0098..04e22574af2 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -40,7 +40,6 @@ class PartitionPruningRDD[T: ClassManifest]( override protected def getPartitions: Array[Partition] = getDependencies.head.asInstanceOf[PruneDependency[T]].partitions - } diff --git a/core/src/test/scala/spark/PartitionPruningRDDSuite.scala b/core/src/test/scala/spark/PartitionPruningRDDSuite.scala index a0e64131602..4fd1f55a456 100644 --- a/core/src/test/scala/spark/PartitionPruningRDDSuite.scala +++ b/core/src/test/scala/spark/PartitionPruningRDDSuite.scala @@ -25,4 +25,4 @@ class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { val p = prunedRDD.partitions(0) assert(p.index == 2) } -} \ No newline at end of file +} From 72cf7ec0e5736d8cc5a691b36c78a79c9a4bf118 Mon Sep 17 00:00:00 2001 From: harshars Date: Fri, 26 Jul 2013 15:16:41 -0700 Subject: [PATCH 0476/2521] Indentation --- core/src/main/scala/spark/rdd/PartitionPruningRDD.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index 04e22574af2..eaac134ef47 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -16,9 +16,9 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo extends NarrowDependency[T](rdd) { @transient - val partitions: Array[Partition] = rdd.partitions. - zipWithIndex.filter(s => partitionFilterFunc(s._2)). - map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } + val partitions: Array[Partition] = rdd.partitions.zipWithIndex + .filter(s => partitionFilterFunc(s._2)) + .map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } override def getParents(partitionId: Int) = List(partitions(partitionId).index) } From 392d7474fde6f18bcd9cece5c4ed057ac1d24485 Mon Sep 17 00:00:00 2001 From: harshars Date: Fri, 26 Jul 2013 15:23:15 -0700 Subject: [PATCH 0477/2521] Code review --- core/src/main/scala/spark/rdd/PartitionPruningRDD.scala | 4 ++-- core/src/test/scala/spark/PartitionPruningRDDSuite.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala index eaac134ef47..fa2f5c26110 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala @@ -17,8 +17,8 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo @transient val partitions: Array[Partition] = rdd.partitions.zipWithIndex - .filter(s => partitionFilterFunc(s._2)) - .map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } + .filter(s => partitionFilterFunc(s._2)) + .map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition } override def getParents(partitionId: Int) = List(partitions(partitionId).index) } diff --git a/core/src/test/scala/spark/PartitionPruningRDDSuite.scala b/core/src/test/scala/spark/PartitionPruningRDDSuite.scala index 4fd1f55a456..88352b639f9 100644 --- a/core/src/test/scala/spark/PartitionPruningRDDSuite.scala +++ b/core/src/test/scala/spark/PartitionPruningRDDSuite.scala @@ -21,8 +21,8 @@ class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { def compute(split: Partition, context: TaskContext) = {Iterator()} } val prunedRDD = PartitionPruningRDD.create(rdd, {x => if (x==2) true else false}) - println(prunedRDD.partitions.length) val p = prunedRDD.partitions(0) assert(p.index == 2) + assert(prunedRDD.partitions.length == 1) } } From 3fc64089032707393cc016b17ccd3dade20f79fe Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 26 Jul 2013 16:10:20 -0700 Subject: [PATCH 0478/2521] Added missing scalatest dependency --- tools/pom.xml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/tools/pom.xml b/tools/pom.xml index 1125aba4f17..878eb82f182 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -30,6 +30,14 @@ Spark Project Tools http://spark-project.org/ + + + org.scalatest + scalatest_${scala.version} + test + + + target/scala-${scala.version}/classes target/scala-${scala.version}/test-classes From bd4cc52e309667296ef60557e17e8a7263c51fd8 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Fri, 26 Jul 2013 17:23:18 -0700 Subject: [PATCH 0479/2521] Made metrics Option instead of Some, fixed NullPointerException --- .../scala/spark/ui/exec/ExecutorsUI.scala | 2 +- .../scala/spark/ui/jobs/JobProgressUI.scala | 22 +++++-------------- 2 files changed, 7 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 b15da81f7cc..80d00c68735 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -144,7 +144,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { (Some(e), e.metrics) case _ => executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 - (None, Some(taskEnd.taskMetrics)) + (None, Option(taskEnd.taskMetrics)) } val taskList = executorToTaskInfos.getOrElse( eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index 2fb4575551a..e7fbff7f738 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala @@ -132,39 +132,29 @@ private[spark] class JobProgressListener extends SparkListener { (Some(e), e.metrics) case _ => stageToTasksComplete(sid) = stageToTasksComplete.getOrElse(sid, 0) + 1 - (None, Some(taskEnd.taskMetrics)) + (None, Option(taskEnd.taskMetrics)) } if (!stageToTime.contains(sid)) { stageToTime(sid) = 0L } - val time = if (metrics.isDefined) metrics.map(m => m.executorRunTime).getOrElse(0) else 0 + val time = metrics.map(m => m.executorRunTime).getOrElse(0) stageToTime(sid) += time totalTime += time if (!stageToShuffleRead.contains(sid)) { stageToShuffleRead(sid) = 0L } - val shuffleRead = - if (metrics.isDefined) { - metrics.flatMap(m => m.shuffleReadMetrics).map(s => s.remoteBytesRead).getOrElse(0L) - } - else { - 0L - } + val shuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => + s.remoteBytesRead).getOrElse(0L) stageToShuffleRead(sid) += shuffleRead totalShuffleRead += shuffleRead if (!stageToShuffleWrite.contains(sid)) { stageToShuffleWrite(sid) = 0L } - val shuffleWrite = - if (metrics.isDefined) { - metrics.flatMap(m => m.shuffleWriteMetrics).map(s => s.shuffleBytesWritten).getOrElse(0L) - } - else { - 0L - } + val shuffleWrite = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => + s.shuffleBytesWritten).getOrElse(0L) stageToShuffleWrite(sid) += shuffleWrite totalShuffleWrite += shuffleWrite From f74a03c6d82f47e004d32de5bd4e17be91d35888 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Fri, 26 Jul 2013 17:29:44 -0700 Subject: [PATCH 0480/2521] Multiple changes - Changed LogisticRegression regularization parameter to 0 - Removed println from SVM predict function - Fixed "Lasso" -> "SVM" in SVMGenerator - Added comment in Updater.scala to indicate L1 regularization leads to soft thresholding proximal function --- .../spark/mllib/classification/LogisticRegression.scala | 4 ++-- mllib/src/main/scala/spark/mllib/classification/SVM.scala | 4 +--- .../scala/spark/mllib/classification/SVMGenerator.scala | 6 +++--- mllib/src/main/scala/spark/mllib/optimization/Updater.scala | 3 +++ 4 files changed, 9 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index f39c1ec52eb..876197a14f4 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -36,7 +36,7 @@ class LogisticRegression private (var stepSize: Double, var regParam: Double, va /** * Construct a LogisticRegression object with default parameters */ - def this() = this(1.0, 1.0, 1.0, 100) + def this() = this(1.0, 0.0, 1.0, 100) /** * Set the step size per-iteration of SGD. Default 1.0. @@ -145,7 +145,7 @@ object LogisticRegression { numIterations: Int) : LogisticRegressionModel = { - train(input, numIterations, 1.0, 1.0, 1.0) + train(input, numIterations, 1.0, 0.0, 1.0) } def main(args: Array[String]) { diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index aceb903f1d6..c8da8b7c297 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -16,11 +16,9 @@ class SVMModel( val losses: Array[Double]) extends ClassificationModel { override def predict(testData: spark.RDD[Array[Double]]) = { - testData.map { x => { - println("Predicting " + x) + testData.map { x => signum(new DoubleMatrix(1, x.length, x:_*).dot(this.weights) + this.intercept) } - } } override def predict(testData: Array[Double]): Double = { diff --git a/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala b/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala index a5e28373435..029f262660e 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala @@ -8,11 +8,11 @@ import org.jblas.DoubleMatrix import spark.{RDD, SparkContext} import spark.mllib.util.MLUtils -object LassoGenerator { +object SVMGenerator { def main(args: Array[String]) { if (args.length != 5) { - println("Usage: LassoGenerator " + + println("Usage: SVMGenerator " + " ") System.exit(1) } @@ -24,7 +24,7 @@ object LassoGenerator { val parts: Int = if (args.length > 4) args(4).toInt else 2 val eps = 3 - val sc = new SparkContext(sparkMaster, "LassoGenerator") + val sc = new SparkContext(sparkMaster, "SVMGenerator") val globalRnd = new Random(94720) val trueWeights = Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() } diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index 64c54dfb0d5..26f06e86dcb 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -28,6 +28,9 @@ class SimpleUpdater extends Updater { } } +/** +L1 regularization -- corresponding proximal operator is the soft-thresholding function +**/ class L1Updater extends Updater { override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { From f0a1f95228bdb3f2f4c8a869a96c8df1a066edf3 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Fri, 26 Jul 2013 17:36:14 -0700 Subject: [PATCH 0481/2521] Rename LogisticRegression, SVM and Lasso to *_LocalRandomSGD --- .../spark/mllib/classification/LogisticRegression.scala | 8 ++++---- mllib/src/main/scala/spark/mllib/classification/SVM.scala | 8 ++++---- mllib/src/main/scala/spark/mllib/regression/Lasso.scala | 8 ++++---- .../mllib/classification/LogisticRegressionSuite.scala | 4 ++-- .../test/scala/spark/mllib/classification/SVMSuite.scala | 4 ++-- .../test/scala/spark/mllib/regression/LassoSuite.scala | 4 ++-- 6 files changed, 18 insertions(+), 18 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index 876197a14f4..243a3463646 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -29,7 +29,7 @@ class LogisticRegressionModel( } } -class LogisticRegression private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, +class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -86,7 +86,7 @@ class LogisticRegression private (var stepSize: Double, var regParam: Double, va /** * Top-level methods for calling Logistic Regression. */ -object LogisticRegression { +object LogisticRegression_LocalRandomSGD { /** * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number @@ -107,7 +107,7 @@ object LogisticRegression { miniBatchFraction: Double) : LogisticRegressionModel = { - new LogisticRegression(stepSize, regParam, miniBatchFraction, numIterations).train(input) + new LogisticRegression_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) } /** @@ -155,7 +155,7 @@ object LogisticRegression { } val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = LogisticRegression.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = LogisticRegression_LocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index c8da8b7c297..bf10493bf50 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -27,7 +27,7 @@ class SVMModel( } } -class SVM private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, +class SVM_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -92,7 +92,7 @@ class SVM private (var stepSize: Double, var regParam: Double, var miniBatchFrac /** * Top-level methods for calling SVM. */ -object SVM { +object SVM_LocalRandomSGD { /** * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number @@ -113,7 +113,7 @@ object SVM { miniBatchFraction: Double) : SVMModel = { - new SVM(stepSize, regParam, miniBatchFraction, numIterations).train(input) + new SVM_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) } /** @@ -161,7 +161,7 @@ object SVM { } val sc = new SparkContext(args(0), "SVM") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = SVM.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = SVM_LocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala index de410711a20..bb2305c8119 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala @@ -26,7 +26,7 @@ class LassoModel( } } -class Lasso private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, +class Lasso_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -91,7 +91,7 @@ class Lasso private (var stepSize: Double, var regParam: Double, var miniBatchFr /** * Top-level methods for calling Lasso. */ -object Lasso { +object Lasso_LocalRandomSGD { /** * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number @@ -112,7 +112,7 @@ object Lasso { miniBatchFraction: Double) : LassoModel = { - new Lasso(stepSize, regParam, miniBatchFraction, numIterations).train(input) + new Lasso_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) } /** @@ -160,7 +160,7 @@ object Lasso { } val sc = new SparkContext(args(0), "Lasso") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = Lasso.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = Lasso_LocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index 13612e9a4a9..5741906a14e 100644 --- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -18,7 +18,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { } // Test if we can correctly learn A, B where Y = logistic(A + B*X) - test("logistic regression") { + test("LogisticRegression_LocalRandomSGD") { val nPoints = 10000 val rnd = new Random(42) @@ -45,7 +45,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val lr = new LogisticRegression().setStepSize(10.0) + val lr = new LogisticRegression_LocalRandomSGD().setStepSize(10.0) .setNumIterations(20) val model = lr.train(testRDD) diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index e3a6681ab28..50cf260f490 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -19,7 +19,7 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } - test("SVM") { + test("SVM_LocalRandomSGD") { val nPoints = 10000 val rnd = new Random(42) @@ -46,7 +46,7 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { writer_data.write("\n")}) writer_data.close() - val svm = new SVM().setStepSize(1.0) + val svm = new SVM_LocalRandomSGD().setStepSize(1.0) .setRegParam(1.0) .setNumIterations(100) diff --git a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala index 90fedb3e84b..9836ac54c12 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala @@ -17,7 +17,7 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } - test("Lasso") { + test("Lasso_LocalRandomSGD") { val nPoints = 10000 val rnd = new Random(42) @@ -36,7 +36,7 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val ls = new Lasso().setStepSize(1.0) + val ls = new Lasso_LocalRandomSGD().setStepSize(1.0) .setRegParam(0.01) .setNumIterations(20) From 10fd3949e6ccfca896ccfefe5895fda7f40944cf Mon Sep 17 00:00:00 2001 From: Xinghao Date: Fri, 26 Jul 2013 17:49:11 -0700 Subject: [PATCH 0482/2521] Making ClassificationModel serializable --- .../main/scala/spark/mllib/classification/Classification.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/Classification.scala b/mllib/src/main/scala/spark/mllib/classification/Classification.scala index 7f1eb210792..96d7a54f187 100644 --- a/mllib/src/main/scala/spark/mllib/classification/Classification.scala +++ b/mllib/src/main/scala/spark/mllib/classification/Classification.scala @@ -2,7 +2,7 @@ package spark.mllib.classification import spark.RDD -trait ClassificationModel { +trait ClassificationModel extends Serializable { /** * Predict values for the given data set using the model trained. * From b0bbc7f6a8da8e4c8e4e5bb656d7c7eed9b24511 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Fri, 26 Jul 2013 18:57:39 -0700 Subject: [PATCH 0483/2521] Resolve conflicts with master, removed regParam for LogisticRegression --- .../classification/LogisticRegression.scala | 120 +++++++++++++++--- .../spark/mllib/classification/SVM.scala | 96 ++++++++++++-- .../spark/mllib/optimization/Gradient.scala | 18 ++- .../mllib/optimization/GradientDescent.scala | 37 ++++-- .../scala/spark/mllib/regression/Lasso.scala | 97 ++++++++++++-- .../LogisticRegressionSuite.scala | 108 ++++++++++++++-- 6 files changed, 412 insertions(+), 64 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index 243a3463646..40b96fbe3ab 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/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.classification import spark.{Logging, RDD, SparkContext} @@ -11,32 +28,39 @@ 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 losses: Array[Double]) extends ClassificationModel { + val stochasticLosses: Array[Double]) extends ClassificationModel { + + // 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]]) = { + // 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(this.weights).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)) } } 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)) } } -class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, +class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { /** * Construct a LogisticRegression object with default parameters */ - def this() = this(1.0, 0.0, 1.0, 100) + def this() = this(1.0, 1.0, 100) /** * Set the step size per-iteration of SGD. Default 1.0. @@ -46,6 +70,14 @@ class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var regPa this } + + + + + + + + /** * Set fraction of data to be used for each SGD iteration. Default 1.0. */ @@ -63,31 +95,77 @@ class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var regPa } 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 (weights, losses) = GradientDescent.runMiniBatchSGD( - data, new LogisticGradient(), new SimpleUpdater(), stepSize, numIters, regParam, miniBatchFraction) + val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) - val weightsScaled = weights.getRange(1, weights.length) - val intercept = weights.get(0) + val (weights, stochasticLosses) = GradientDescent.runMiniBatchSGD( + data, + new LogisticGradient(), + new SimpleUpdater(), + stepSize, + numIters, + 0.0, + initalWeightsWithIntercept, + miniBatchFraction) - val model = new LogisticRegressionModel(weightsScaled, intercept, losses) + val intercept = weights(0) + val weightsScaled = weights.tail - logInfo("Final model weights " + model.weights) + val model = new LogisticRegressionModel(weightsScaled, intercept, stochasticLosses) + + logInfo("Final model weights " + model.weights.mkString(",")) 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_LocalRandomSGD { + /** + * 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_LocalRandomSGD(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 @@ -96,18 +174,18 @@ object LogisticRegression_LocalRandomSGD { * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. * @param stepSize Step size to be used for each iteration of gradient descent. - * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. */ def train( input: RDD[(Double, Array[Double])], numIterations: Int, stepSize: Double, - regParam: Double, + miniBatchFraction: Double) : LogisticRegressionModel = { - new LogisticRegression_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) + new LogisticRegression_LocalRandomSGD(stepSize, miniBatchFraction, numIterations).train(input) } /** @@ -117,18 +195,18 @@ object LogisticRegression_LocalRandomSGD { * * @param input RDD of (label, array of features) pairs. * @param stepSize Step size to be used for each iteration of Gradient Descent. - * @param regParam Regularization parameter. + * @param numIterations Number of iterations of gradient descent to run. * @return a LogisticRegressionModel which has the weights and offset from training. */ def train( input: RDD[(Double, Array[Double])], numIterations: Int, - stepSize: Double, - regParam: Double) + stepSize: Double + ) : LogisticRegressionModel = { - train(input, numIterations, stepSize, regParam, 1.0) + train(input, numIterations, stepSize, 1.0) } /** @@ -145,7 +223,7 @@ object LogisticRegression_LocalRandomSGD { numIterations: Int) : LogisticRegressionModel = { - train(input, numIterations, 1.0, 0.0, 1.0) + train(input, numIterations, 1.0, 1.0) } def main(args: Array[String]) { diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index bf10493bf50..2cd1d668eb1 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.classification import scala.math.signum @@ -11,22 +28,31 @@ import org.jblas.DoubleMatrix * SVM using Stochastic Gradient Descent. */ class SVMModel( - val weights: DoubleMatrix, + val weights: Array[Double], val intercept: Double, - val losses: Array[Double]) extends ClassificationModel { + val stochasticLosses: Array[Double]) extends ClassificationModel { + + // 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]]) = { + // 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 => - signum(new DoubleMatrix(1, x.length, x:_*).dot(this.weights) + this.intercept) + signum(new DoubleMatrix(1, x.length, x:_*).dot(localWeights) + localIntercept) } } override def predict(testData: Array[Double]): Double = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) - signum(dataMat.dot(this.weights) + this.intercept) + signum(dataMat.dot(weightsMatrix) + this.intercept) } } + + class SVM_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -69,31 +95,77 @@ class SVM_LocalRandomSGD private (var stepSize: Double, var regParam: Double, va } def train(input: RDD[(Double, Array[Double])]): SVMModel = { + 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]): SVMModel = { + // 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 HingeGradient(), new SquaredL2Updater(), stepSize, numIters, regParam, miniBatchFraction) + val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) + + val (weights, stochasticLosses) = GradientDescent.runMiniBatchSGD( + data, + new HingeGradient(), + new SquaredL2Updater(), + stepSize, + numIters, + regParam, + initalWeightsWithIntercept, + miniBatchFraction) - val weightsScaled = weights.getRange(1, weights.length) - val intercept = weights.get(0) + val intercept = weights(0) + val weightsScaled = weights.tail - val model = new SVMModel(weightsScaled, intercept, losses) + val model = new SVMModel(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 losses " + model.losses.takeRight(10).mkString(", ")) + logInfo("Last 10 stochasticLosses " + model.stochasticLosses.takeRight(10).mkString(", ")) model } } /** * Top-level methods for calling SVM. + + */ object SVM_LocalRandomSGD { + /** + * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in + * gradient descent are initialized using the initial weights provided. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + * @param initialWeights Initial set of weights to be used. Array should be equal in size to + * the number of features in the data. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, + initialWeights: Array[Double]) + : SVMModel = + { + new SVM_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) + } + /** * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number * of iterations of gradient descent using the specified step size. Each iteration uses @@ -151,7 +223,7 @@ object SVM_LocalRandomSGD { numIterations: Int) : SVMModel = { - train(input, numIterations, 1.0, 0.10, 1.0) + train(input, numIterations, 1.0, 1.0, 1.0) } def main(args: Array[String]) { diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala index 6ffc3b128b0..4864ab7ccfc 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 @@ -58,4 +75,3 @@ class HingeGradient extends Gradient { (DoubleMatrix.zeros(1,weights.length), 0.0) } } - diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index bd8489c3864..8387d4939b9 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} @@ -23,8 +40,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 lossHistory - Array containing the 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])], @@ -33,16 +51,16 @@ object GradientDescent { stepSize: Double, numIters: Int, regParam: Double, - miniBatchFraction: Double=1.0) : (DoubleMatrix, Array[Double]) = { + initialWeights: Array[Double], + miniBatchFraction: Double=1.0) : (Array[Double], Array[Double]) = { - val lossHistory = new ArrayBuffer[Double](numIters) + val stochasticLossHistory = 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) + // Initialize weights as a column vector + var weights = new DoubleMatrix(initialWeights.length, 1, initialWeights:_*) var reg_val = 0.0 for (i <- 1 to numIters) { @@ -53,16 +71,17 @@ object GradientDescent { (grad, loss) }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) + stochasticLossHistory.append(lossSum / miniBatchSize + reg_val) val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i, regParam) weights = update._1 reg_val = update._2 - lossHistory.append(lossSum / miniBatchSize + reg_val) + stochasticLossHistory.append(lossSum / miniBatchSize + reg_val) /*** Xinghao: The loss here is sum of lossSum computed using the weights before applying updater, and reg_val using weights after applying updater ***/ } - (weights, lossHistory.toArray) + (weights.toArray, stochasticLossHistory.toArray) } } diff --git a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala index bb2305c8119..64364323a25 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR 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} @@ -8,24 +25,34 @@ import org.jblas.DoubleMatrix /** * Lasso using Stochastic Gradient Descent. + * */ class LassoModel( - val weights: DoubleMatrix, + val weights: Array[Double], val intercept: Double, - val losses: Array[Double]) extends RegressionModel { + 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]]) = { + // 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 => - new DoubleMatrix(1, x.length, x:_*).dot(this.weights) + this.intercept + new DoubleMatrix(1, x.length, x:_*).dot(localWeights) + localIntercept } } + override def predict(testData: Array[Double]): Double = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) - dataMat.dot(this.weights) + this.intercept + dataMat.dot(weightsMatrix) + this.intercept } } + class Lasso_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -68,31 +95,77 @@ class Lasso_LocalRandomSGD private (var stepSize: Double, var regParam: Double, } def train(input: RDD[(Double, Array[Double])]): LassoModel = { + 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]): LassoModel = { + // 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 SquaredGradient(), new L1Updater(), stepSize, numIters, regParam, miniBatchFraction) + val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) + + val (weights, stochasticLosses) = GradientDescent.runMiniBatchSGD( + data, + new SquaredGradient(), + new L1Updater(), + stepSize, + numIters, + regParam, + initalWeightsWithIntercept, + miniBatchFraction) - val weightsScaled = weights.getRange(1, weights.length) - val intercept = weights.get(0) + val intercept = weights(0) + val weightsScaled = weights.tail - val model = new LassoModel(weightsScaled, intercept, losses) + val model = new LassoModel(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 losses " + model.losses.takeRight(10).mkString(", ")) + logInfo("Last 10 stochasticLosses " + model.stochasticLosses.takeRight(10).mkString(", ")) model } } /** * Top-level methods for calling Lasso. + * + * */ object Lasso_LocalRandomSGD { + /** + * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in + * gradient descent are initialized using the initial weights provided. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + * @param initialWeights Initial set of weights to be used. Array should be equal in size to + * the number of features in the data. + */ + def train( + input: RDD[(Double, Array[Double])], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, + initialWeights: Array[Double]) + : LassoModel = + { + new Lasso_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) + } + /** * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number * of iterations of gradient descent using the specified step size. Each iteration uses @@ -150,7 +223,7 @@ object Lasso_LocalRandomSGD { numIterations: Int) : LassoModel = { - train(input, numIterations, 1.0, 0.10, 1.0) + train(input, numIterations, 1.0, 1.0, 1.0) } def main(args: Array[String]) { diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index 5741906a14e..827ca66330b 100644 --- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -1,4 +1,25 @@ +<<<<<<< HEAD:mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala package spark.mllib.classification +======= +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR 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 +>>>>>>> FETCH_HEAD:mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala import scala.util.Random @@ -6,7 +27,6 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext -import spark.SparkContext._ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { @@ -17,16 +37,23 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } +<<<<<<< HEAD:mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala // Test if we can correctly learn A, B where Y = logistic(A + B*X) test("LogisticRegression_LocalRandomSGD") { val nPoints = 10000 val rnd = new Random(42) +======= + // Generate input of the form Y = logistic(offset + scale*X) + def generateLogisticInput( + offset: Double, + scale: Double, + nPoints: Int, + seed: Int): Seq[(Double, Array[Double])] = { + val rnd = new Random(seed) +>>>>>>> FETCH_HEAD:mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala 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 => @@ -34,24 +61,87 @@ 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 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)))).toArray + val testData = (0 until nPoints).map(i => (y(i), Array(x1(i)))) + testData + } + + def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { + 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(numOffPredictions < input.length / 5) + } + + // 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, 42) val testRDD = sc.parallelize(testData, 2) testRDD.cache() +<<<<<<< HEAD:mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala val lr = new LogisticRegression_LocalRandomSGD().setStepSize(10.0) .setNumIterations(20) +======= + val lr = new LogisticRegression().setStepSize(10.0).setNumIterations(20) +>>>>>>> FETCH_HEAD:mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala val model = lr.train(testRDD) - val weight0 = model.weights.get(0) + // Test the weights + val weight0 = model.weights(0) + assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") + assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + + val validationData = generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) + } + + test("logistic regression with initial weights") { + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = generateLogisticInput(A, B, nPoints, 42) + + val initialB = -1.0 + val initialWeights = Array(initialB) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + // Use half as many iterations as the previous test. + val lr = new 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]") + + val validationData = generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) } } From 23f3e0f117a7cc4fab0c710ea3583f2aee68d663 Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Fri, 26 Jul 2013 19:15:11 -0700 Subject: [PATCH 0484/2521] mixing in SharedSparkContext for the kryo-collect test --- .../scala/spark/KryoSerializerSuite.scala | 24 +++++++++---------- project/SparkBuild.scala | 13 +++++----- 2 files changed, 19 insertions(+), 18 deletions(-) diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/spark/KryoSerializerSuite.scala index 83df0af1ae1..793b0b66c44 100644 --- a/core/src/test/scala/spark/KryoSerializerSuite.scala +++ b/core/src/test/scala/spark/KryoSerializerSuite.scala @@ -26,7 +26,7 @@ import com.esotericsoftware.kryo._ import SparkContext._ import spark.test.{ClassWithoutNoArgConstructor, MyRegistrator} -class KryoSerializerSuite extends FunSuite { +class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("basic types") { val ser = (new KryoSerializer).newInstance() def check[T](t: T) { @@ -128,23 +128,23 @@ class KryoSerializerSuite extends FunSuite { System.clearProperty("spark.kryo.registrator") } - test("kryo-collect") { + test("kryo with collect") { + val control = 1 :: 2 :: Nil + val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_)).collect().map(_.x) + assert(control == result.toSeq) + } + + override def beforeAll() { System.setProperty("spark.serializer", "spark.KryoSerializer") System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName) + super.beforeAll() + } - val sc = new SparkContext("local", "kryoTest") - try { - val control = 1 :: 2 :: Nil - val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_)).collect().map(_.x) - assert(control == result.toSeq) - } finally { - sc.stop() - } - + override def afterAll() { + super.afterAll() System.clearProperty("spark.kryo.registrator") System.clearProperty("spark.serializer") } - } package test { diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f3f67b57c89..998747c68f5 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -26,14 +26,14 @@ import AssemblyKeys._ object SparkBuild extends Build { // Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or // "1.0.4" for Apache releases, or "0.20.2-cdh3u5" for Cloudera Hadoop. - val HADOOP_VERSION = "1.0.4" - val HADOOP_MAJOR_VERSION = "1" - val HADOOP_YARN = false + //val HADOOP_VERSION = "1.0.4" + //val HADOOP_MAJOR_VERSION = "1" + //val HADOOP_YARN = false // For Hadoop 2 versions such as "2.0.0-mr1-cdh4.1.1", set the HADOOP_MAJOR_VERSION to "2" - //val HADOOP_VERSION = "2.0.0-mr1-cdh4.1.1" - //val HADOOP_MAJOR_VERSION = "2" - //val HADOOP_YARN = false + val HADOOP_VERSION = "2.0.0-cdh4.3.0" + val HADOOP_MAJOR_VERSION = "2" + val HADOOP_YARN = true // For Hadoop 2 YARN support //val HADOOP_VERSION = "2.0.2-alpha" @@ -185,6 +185,7 @@ object SparkBuild extends Build { if (HADOOP_YARN) { Seq( // Exclude rule required for all ? + "org.apache.hadoop" % "hadoop-common" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm), "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), From 0c391feb73610e56beb542bbf66c56efab01bada Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Mon, 22 Jul 2013 16:39:34 -0700 Subject: [PATCH 0485/2521] Maximum task failures configurable --- .../scala/spark/scheduler/cluster/ClusterTaskSetManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index d72b0bfc9f5..c15aeb5ff03 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -67,7 +67,7 @@ private[spark] class ClusterTaskSetManager( 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 + val MAX_TASK_FAILURES = System.getProperty("spark.task.max.fail","4").toInt // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble From 6a47cee72104c9f6d264cb2e61ae8a38523cfdad Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Mon, 22 Jul 2013 16:41:37 -0700 Subject: [PATCH 0486/2521] style --- .../scala/spark/scheduler/cluster/ClusterTaskSetManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index c15aeb5ff03..685fc016b2b 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -67,7 +67,7 @@ private[spark] class ClusterTaskSetManager( 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 = System.getProperty("spark.task.max.fail","4").toInt + val MAX_TASK_FAILURES = System.getProperty("spark.task.max.fail", "4").toInt // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble From 1714693324b5a3169d2e10dc16a2598f8e14d8b1 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Sat, 27 Jul 2013 12:24:41 -0700 Subject: [PATCH 0487/2521] Current time called once with value now --- core/src/main/scala/spark/ui/jobs/StagePage.scala | 3 ++- 1 file changed, 2 insertions(+), 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 266f180a500..5a18559ae92 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -34,6 +34,7 @@ import spark.executor.TaskMetrics private[spark] class StagePage(parent: JobProgressUI) { def listener = parent.listener val dateFmt = parent.dateFmt + val now = System.currentTimeMillis() def render(request: HttpServletRequest): Seq[Node] = { val stageId = request.getParameter("id").toInt @@ -54,7 +55,7 @@ private[spark] class StagePage(parent: JobProgressUI) { var activeTime = 0L listener.stageToTasksActive(stageId).foreach { t => - activeTime += t.timeRunning(System.currentTimeMillis()) + activeTime += t.timeRunning(now) } val summary = From dcc4743a950c7a9b767d7157ca62f512917ee6ae Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Sat, 27 Jul 2013 12:52:53 -0700 Subject: [PATCH 0488/2521] Moved val now to render --- 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 5a18559ae92..ef72f42e305 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -34,10 +34,10 @@ import spark.executor.TaskMetrics private[spark] class StagePage(parent: JobProgressUI) { def listener = parent.listener val dateFmt = parent.dateFmt - val now = System.currentTimeMillis() def render(request: HttpServletRequest): Seq[Node] = { val stageId = request.getParameter("id").toInt + val now = System.currentTimeMillis() if (!listener.stageToTaskInfos.contains(stageId)) { val content = From 5a93e3c58c69574eaac6458f8515579b5bd03fd9 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Sat, 27 Jul 2013 15:55:26 -0700 Subject: [PATCH 0489/2521] Cleaned up code based on pwendell's suggestions --- .../scala/spark/ui/exec/ExecutorsUI.scala | 12 ++---- .../main/scala/spark/ui/jobs/IndexPage.scala | 18 ++++---- .../scala/spark/ui/jobs/JobProgressUI.scala | 43 ++++--------------- 3 files changed, 20 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 80d00c68735..948b3017dbd 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -121,10 +121,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskStart(taskStart: SparkListenerTaskStart) { val eid = taskStart.taskInfo.executorId - if (!executorToTasksActive.contains(eid)) { - executorToTasksActive(eid) = HashSet[TaskInfo]() - } - executorToTasksActive(eid) += taskStart.taskInfo + val activeTasks = executorToTasksActive.getOrElseUpdate(eid, new HashSet[TaskInfo]()) + activeTasks += taskStart.taskInfo val taskList = executorToTaskInfos.getOrElse( eid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList += ((taskStart.taskInfo, None, None)) @@ -133,10 +131,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId - if (!executorToTasksActive.contains(eid)) { - executorToTasksActive(eid) = HashSet[TaskInfo]() - } - executorToTasksActive(eid) -= taskEnd.taskInfo + val activeTasks = executorToTasksActive.getOrElseUpdate(eid, new HashSet[TaskInfo]()) + activeTasks -= taskStart.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index b862c3539c9..7e504c5f9f9 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -125,16 +125,14 @@ private[spark] class IndexPage(parent: JobProgressUI) { case None => "Unknown" } - val shuffleRead = - if (!listener.hasShuffleRead(s.id)) - "" - else - Utils.memoryBytesToString(listener.stageToShuffleRead(s.id)) - val shuffleWrite = - if (!listener.hasShuffleWrite(s.id)) - "" - else - Utils.memoryBytesToString(listener.stageToShuffleWrite(s.id)) + val shuffleRead = listener.stageToShuffleRead(s.id) match { + case 0 => "" + case b => Utils.memoryBytesToString(b) + } + val shuffleWrite = listener.stageToShuffleWrite(s.id) match { + case 0 => "" + case b => Utils.memoryBytesToString(b) + } val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) val totalTasks = s.numPartitions diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala index e7fbff7f738..09d24b63020 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]() + // Total metrics reflect metrics only for completed tasks var totalTime = 0L var totalShuffleRead = 0L var totalShuffleWrite = 0L @@ -109,10 +110,8 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { val sid = taskStart.task.stageId - if (!stageToTasksActive.contains(sid)) { - stageToTasksActive(sid) = HashSet[TaskInfo]() - } - stageToTasksActive(sid) += taskStart.taskInfo + val tasksActive = stageToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]()) + tasksActive += taskStart.taskInfo val taskList = stageToTaskInfos.getOrElse( sid, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) taskList += ((taskStart.taskInfo, None, None)) @@ -121,10 +120,8 @@ private[spark] class JobProgressListener extends SparkListener { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val sid = taskEnd.task.stageId - if (!stageToTasksActive.contains(sid)) { - stageToTasksActive(sid) = HashSet[TaskInfo]() - } - stageToTasksActive(sid) -= taskEnd.taskInfo + val tasksActive = stageToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]()) + tasksActive -= taskEnd.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => @@ -135,24 +132,18 @@ private[spark] class JobProgressListener extends SparkListener { (None, Option(taskEnd.taskMetrics)) } - if (!stageToTime.contains(sid)) { - stageToTime(sid) = 0L - } + stageToTime.getOrElseUpdate(sid, 0L) val time = metrics.map(m => m.executorRunTime).getOrElse(0) stageToTime(sid) += time totalTime += time - if (!stageToShuffleRead.contains(sid)) { - stageToShuffleRead(sid) = 0L - } + stageToShuffleRead.getOrElseUpdate(sid, 0L) val shuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => s.remoteBytesRead).getOrElse(0L) stageToShuffleRead(sid) += shuffleRead totalShuffleRead += shuffleRead - if (!stageToShuffleWrite.contains(sid)) { - stageToShuffleWrite(sid) = 0L - } + stageToShuffleWrite.getOrElseUpdate(sid, 0L) val shuffleWrite = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => s.shuffleBytesWritten).getOrElse(0L) stageToShuffleWrite(sid) += shuffleWrite @@ -178,22 +169,4 @@ private[spark] class JobProgressListener extends SparkListener { case _ => } } - - /** 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.flatMap(m => m.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.flatMap(m => m.shuffleWriteMetrics).isDefined - } - return false // No tasks have finished for this stage - } } From c2223e68012a44e9c4bebd97dc1db772df8c597e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 15 Jul 2013 08:48:53 -0700 Subject: [PATCH 0490/2521] Improve catch scope and logging for client stop() This does two things: 1. Catches the more general `TimeoutException`, since those can be thrown. 2. Logs at info level when a timeout is detected. --- core/src/main/scala/spark/deploy/client/Client.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index edefa0292d0..6b038f5874e 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -22,7 +22,7 @@ import akka.actor._ import akka.pattern.ask import akka.util.Duration import akka.util.duration._ -import akka.pattern.AskTimeoutException +import java.util.concurrent.TimeoutException import spark.{SparkException, Logging} import akka.remote.RemoteClientLifeCycleEvent import akka.remote.RemoteClientShutdown @@ -134,7 +134,8 @@ private[spark] class Client( val future = actor.ask(StopClient)(timeout) Await.result(future, timeout) } catch { - case e: AskTimeoutException => // Ignore it, maybe master went away + case e: TimeoutException => + logInfo("Close request to Master timed out; it may already be shut down.") } actor = null } From 8177165ac41c1f39a09fa2a0b195f03928472fe5 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 27 Jul 2013 16:01:54 -0700 Subject: [PATCH 0491/2521] Log executor on finish --- .../scala/spark/scheduler/cluster/ClusterTaskSetManager.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 860a38e9f8a..bbf234febd7 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -564,8 +564,8 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: decreaseRunningTasks(1) if (!finished(index)) { tasksFinished += 1 - logInfo("Finished TID %s in %d ms (progress: %d/%d)".format( - tid, info.duration, tasksFinished, numTasks)) + logInfo("Finished TID %s in %d ms on %s (progress: %d/%d)".format( + tid, info.duration, info.hostPort, tasksFinished, numTasks)) // Deserialize task result and pass it to the scheduler try { val result = ser.deserialize[TaskResult[_]](serializedData) From bcafb36c1ebe3d8f377ba69952235ae0a829bff8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 27 Jul 2013 16:03:50 -0700 Subject: [PATCH 0492/2521] Slight wording change --- core/src/main/scala/spark/deploy/client/Client.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index 6b038f5874e..29e494f4959 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -135,7 +135,7 @@ private[spark] class Client( Await.result(future, timeout) } catch { case e: TimeoutException => - logInfo("Close request to Master timed out; it may already be shut down.") + logInfo("Stop request to Master timed out; it may already be shut down.") } actor = null } From 077f2dad226b4f817cb50ad5c07702e78506a698 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Sat, 27 Jul 2013 16:39:36 -0700 Subject: [PATCH 0493/2521] Fixed outdated bugs --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 2 +- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 4 ++-- core/src/main/scala/spark/ui/jobs/StagePage.scala | 8 ++++---- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 948b3017dbd..b70153fd30d 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -132,7 +132,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val eid = taskEnd.taskInfo.executorId val activeTasks = executorToTasksActive.getOrElseUpdate(eid, new HashSet[TaskInfo]()) - activeTasks -= taskStart.taskInfo + activeTasks -= taskEnd.taskInfo val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 7e504c5f9f9..c12a4fe2a5f 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -125,11 +125,11 @@ private[spark] class IndexPage(parent: JobProgressUI) { case None => "Unknown" } - val shuffleRead = listener.stageToShuffleRead(s.id) match { + val shuffleRead = listener.stageToShuffleRead.getOrElse(s.id, 0L) match { case 0 => "" case b => Utils.memoryBytesToString(b) } - val shuffleWrite = listener.stageToShuffleWrite(s.id) match { + val shuffleWrite = listener.stageToShuffleWrite.getOrElse(s.id, 0L) match { case 0 => "" case b => Utils.memoryBytesToString(b) } diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index ef72f42e305..e327cb39478 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -50,8 +50,8 @@ private[spark] class StagePage(parent: JobProgressUI) { val tasks = listener.stageToTaskInfos(stageId) - val shuffleRead = listener.hasShuffleRead(stageId) - val shuffleWrite = listener.hasShuffleWrite(stageId) + val shuffleRead = listener.stageToShuffleRead(stageId) > 0 + val shuffleWrite = listener.stageToShuffleWrite(stageId) > 0 var activeTime = 0L listener.stageToTasksActive(stageId).foreach { t => @@ -65,13 +65,13 @@ private[spark] class StagePage(parent: JobProgressUI) { CPU time: {parent.formatDuration(listener.stageToTime(stageId) + activeTime)} - {if (listener.hasShuffleRead(stageId)) + {if (shuffleRead)
  • Shuffle read: {Utils.memoryBytesToString(listener.stageToShuffleRead(stageId))}
  • } - {if (listener.hasShuffleWrite(stageId)) + {if (shuffleWrite)
  • Shuffle write: {Utils.memoryBytesToString(listener.stageToShuffleWrite(stageId))} From f11ad72d4ee2c6821749e1bf95c46d3f2c2cd860 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 27 Jul 2013 21:11:28 -0400 Subject: [PATCH 0494/2521] Some fixes to Python examples (style and package name for LR) --- python/examples/als.py | 5 ++--- python/examples/kmeans.py | 3 +-- python/examples/logistic_regression.py | 5 ++--- python/examples/pi.py | 3 +-- python/examples/transitive_closure.py | 5 ++--- python/examples/wordcount.py | 3 +-- 6 files changed, 9 insertions(+), 15 deletions(-) mode change 100644 => 100755 python/examples/kmeans.py mode change 100644 => 100755 python/examples/pi.py mode change 100644 => 100755 python/examples/transitive_closure.py mode change 100644 => 100755 python/examples/wordcount.py diff --git a/python/examples/als.py b/python/examples/als.py index f2b2eee64cd..a77dfb25778 100755 --- a/python/examples/als.py +++ b/python/examples/als.py @@ -48,8 +48,7 @@ def update(i, vec, mat, ratings): if __name__ == "__main__": if len(sys.argv) < 2: - print >> sys.stderr, \ - "Usage: PythonALS " + print >> sys.stderr, "Usage: als " exit(-1) sc = SparkContext(sys.argv[1], "PythonALS", pyFiles=[realpath(__file__)]) M = int(sys.argv[2]) if len(sys.argv) > 2 else 100 @@ -84,5 +83,5 @@ def update(i, vec, mat, ratings): usb = sc.broadcast(us) error = rmse(R, ms, us) - print "Iteration %d:" % i + print "Iteration %d:" % i print "\nRMSE: %5.4f\n" % error diff --git a/python/examples/kmeans.py b/python/examples/kmeans.py old mode 100644 new mode 100755 index c670556f2b8..ba31af92fca --- a/python/examples/kmeans.py +++ b/python/examples/kmeans.py @@ -41,8 +41,7 @@ def closestPoint(p, centers): if __name__ == "__main__": if len(sys.argv) < 5: - print >> sys.stderr, \ - "Usage: PythonKMeans " + print >> sys.stderr, "Usage: kmeans " exit(-1) sc = SparkContext(sys.argv[1], "PythonKMeans") lines = sc.textFile(sys.argv[2]) diff --git a/python/examples/logistic_regression.py b/python/examples/logistic_regression.py index 54d227d0d33..3ac1bae4e9a 100755 --- a/python/examples/logistic_regression.py +++ b/python/examples/logistic_regression.py @@ -35,7 +35,7 @@ DataPoint = namedtuple("DataPoint", ['x', 'y']) -from lr import DataPoint # So that DataPoint is properly serialized +from logistic_regression import DataPoint # So that DataPoint is properly serialized def generateData(): @@ -48,8 +48,7 @@ def generatePoint(i): if __name__ == "__main__": if len(sys.argv) == 1: - print >> sys.stderr, \ - "Usage: PythonLR []" + print >> sys.stderr, "Usage: logistic_regression []" exit(-1) sc = SparkContext(sys.argv[1], "PythonLR", pyFiles=[realpath(__file__)]) slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2 diff --git a/python/examples/pi.py b/python/examples/pi.py old mode 100644 new mode 100755 index 33c026e824b..ab0645fc2f3 --- a/python/examples/pi.py +++ b/python/examples/pi.py @@ -24,8 +24,7 @@ if __name__ == "__main__": if len(sys.argv) == 1: - print >> sys.stderr, \ - "Usage: PythonPi []" + print >> sys.stderr, "Usage: pi []" exit(-1) sc = SparkContext(sys.argv[1], "PythonPi") slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2 diff --git a/python/examples/transitive_closure.py b/python/examples/transitive_closure.py old mode 100644 new mode 100755 index 40be3b50007..744cce66516 --- a/python/examples/transitive_closure.py +++ b/python/examples/transitive_closure.py @@ -37,10 +37,9 @@ def generateGraph(): if __name__ == "__main__": if len(sys.argv) == 1: - print >> sys.stderr, \ - "Usage: PythonTC []" + print >> sys.stderr, "Usage: transitive_closure []" exit(-1) - sc = SparkContext(sys.argv[1], "PythonTC") + sc = SparkContext(sys.argv[1], "PythonTransitiveClosure") slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2 tc = sc.parallelize(generateGraph(), slices).cache() diff --git a/python/examples/wordcount.py b/python/examples/wordcount.py old mode 100644 new mode 100755 index 41c846ba79b..a6de22766ab --- a/python/examples/wordcount.py +++ b/python/examples/wordcount.py @@ -23,8 +23,7 @@ if __name__ == "__main__": if len(sys.argv) < 3: - print >> sys.stderr, \ - "Usage: PythonWordCount " + print >> sys.stderr, "Usage: wordcount " exit(-1) sc = SparkContext(sys.argv[1], "PythonWordCount") lines = sc.textFile(sys.argv[2], 1) From b241fcfb35aeaedbcbc35df3f12ecc300c4302ab Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Sat, 27 Jul 2013 22:53:01 -0700 Subject: [PATCH 0495/2521] reverting changes to SparkBuild.scala --- project/SparkBuild.scala | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 998747c68f5..f3f67b57c89 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -26,14 +26,14 @@ import AssemblyKeys._ object SparkBuild extends Build { // Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or // "1.0.4" for Apache releases, or "0.20.2-cdh3u5" for Cloudera Hadoop. - //val HADOOP_VERSION = "1.0.4" - //val HADOOP_MAJOR_VERSION = "1" - //val HADOOP_YARN = false + val HADOOP_VERSION = "1.0.4" + val HADOOP_MAJOR_VERSION = "1" + val HADOOP_YARN = false // For Hadoop 2 versions such as "2.0.0-mr1-cdh4.1.1", set the HADOOP_MAJOR_VERSION to "2" - val HADOOP_VERSION = "2.0.0-cdh4.3.0" - val HADOOP_MAJOR_VERSION = "2" - val HADOOP_YARN = true + //val HADOOP_VERSION = "2.0.0-mr1-cdh4.1.1" + //val HADOOP_MAJOR_VERSION = "2" + //val HADOOP_YARN = false // For Hadoop 2 YARN support //val HADOOP_VERSION = "2.0.2-alpha" @@ -185,7 +185,6 @@ object SparkBuild extends Build { if (HADOOP_YARN) { Seq( // Exclude rule required for all ? - "org.apache.hadoop" % "hadoop-common" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm), "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), From f5067abe85922b8075e03f39a45e4ac39267c62e Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Sat, 27 Jul 2013 23:08:00 -0700 Subject: [PATCH 0496/2521] changes per comments. --- .../spark/scheduler/cluster/ClusterTaskSetManager.scala | 2 +- docs/configuration.md | 8 ++++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 685fc016b2b..d5412345482 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -67,7 +67,7 @@ private[spark] class ClusterTaskSetManager( 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 = System.getProperty("spark.task.max.fail", "4").toInt + val MAX_TASK_FAILURES = System.getProperty("spark.task.maxFailures", "4").toInt // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble diff --git a/docs/configuration.md b/docs/configuration.md index 17fdbf04d1c..e217b499c7b 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -260,6 +260,14 @@ 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.task.maxFailures + 4 + + Number of individual task failures before giving up on the job. + Should greater or equal to 1. Number of allowed retries = this value - 1. + + From 0862494d442199402e37dc968dadf9a5bec9ae78 Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Sat, 27 Jul 2013 23:16:20 -0700 Subject: [PATCH 0497/2521] typo --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index e217b499c7b..6ca5cb99f9e 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -265,7 +265,7 @@ Apart from these, the following properties are also available, and may be useful 4 Number of individual task failures before giving up on the job. - Should greater or equal to 1. Number of allowed retries = this value - 1. + Should be greater than or equal to 1. Number of allowed retries = this value - 1. From ccfa362ddec1bc942785798dea41c3aac52df60f Mon Sep 17 00:00:00 2001 From: Xinghao Date: Sun, 28 Jul 2013 10:33:57 -0700 Subject: [PATCH 0498/2521] Change *_LocalRandomSGD to *LocalRandomSGD --- .../classification/LogisticRegression.scala | 10 ++++---- .../spark/mllib/classification/SVM.scala | 10 ++++---- .../scala/spark/mllib/regression/Lasso.scala | 10 ++++---- .../LogisticRegressionSuite.scala | 23 +++---------------- .../spark/mllib/classification/SVMSuite.scala | 4 ++-- .../spark/mllib/regression/LassoSuite.scala | 8 +++---- 6 files changed, 24 insertions(+), 41 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index 40b96fbe3ab..1b093187f29 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -53,7 +53,7 @@ class LogisticRegressionModel( } } -class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var miniBatchFraction: Double, +class LogisticRegressionLocalRandomSGD private (var stepSize: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -138,7 +138,7 @@ class LogisticRegression_LocalRandomSGD private (var stepSize: Double, var miniB * NOTE(shivaram): We use multiple train methods instead of default arguments to support * Java programs. */ -object LogisticRegression_LocalRandomSGD { +object LogisticRegressionLocalRandomSGD { /** * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed number @@ -163,7 +163,7 @@ object LogisticRegression_LocalRandomSGD { initialWeights: Array[Double]) : LogisticRegressionModel = { - new LogisticRegression_LocalRandomSGD(stepSize, miniBatchFraction, numIterations).train(input, initialWeights) + new LogisticRegressionLocalRandomSGD(stepSize, miniBatchFraction, numIterations).train(input, initialWeights) } /** @@ -185,7 +185,7 @@ object LogisticRegression_LocalRandomSGD { miniBatchFraction: Double) : LogisticRegressionModel = { - new LogisticRegression_LocalRandomSGD(stepSize, miniBatchFraction, numIterations).train(input) + new LogisticRegressionLocalRandomSGD(stepSize, miniBatchFraction, numIterations).train(input) } /** @@ -233,7 +233,7 @@ object LogisticRegression_LocalRandomSGD { } val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = LogisticRegression_LocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = LogisticRegressionLocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index 2cd1d668eb1..76844f6b9c3 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -53,7 +53,7 @@ class SVMModel( -class SVM_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, +class SVMLocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -138,7 +138,7 @@ class SVM_LocalRandomSGD private (var stepSize: Double, var regParam: Double, va */ -object SVM_LocalRandomSGD { +object SVMLocalRandomSGD { /** * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number @@ -163,7 +163,7 @@ object SVM_LocalRandomSGD { initialWeights: Array[Double]) : SVMModel = { - new SVM_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) + new SVMLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) } /** @@ -185,7 +185,7 @@ object SVM_LocalRandomSGD { miniBatchFraction: Double) : SVMModel = { - new SVM_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) + new SVMLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) } /** @@ -233,7 +233,7 @@ object SVM_LocalRandomSGD { } val sc = new SparkContext(args(0), "SVM") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = SVM_LocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = SVMLocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala index 64364323a25..1952658bb20 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala @@ -53,7 +53,7 @@ class LassoModel( } -class Lasso_LocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, +class LassoLocalRandomSGD private (var stepSize: Double, var regParam: Double, var miniBatchFraction: Double, var numIters: Int) extends Logging { @@ -138,7 +138,7 @@ class Lasso_LocalRandomSGD private (var stepSize: Double, var regParam: Double, * * */ -object Lasso_LocalRandomSGD { +object LassoLocalRandomSGD { /** * Train a Lasso model given an RDD of (label, features) pairs. We run a fixed number @@ -163,7 +163,7 @@ object Lasso_LocalRandomSGD { initialWeights: Array[Double]) : LassoModel = { - new Lasso_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) + new LassoLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input, initialWeights) } /** @@ -185,7 +185,7 @@ object Lasso_LocalRandomSGD { miniBatchFraction: Double) : LassoModel = { - new Lasso_LocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) + new LassoLocalRandomSGD(stepSize, regParam, miniBatchFraction, numIterations).train(input) } /** @@ -233,7 +233,7 @@ object Lasso_LocalRandomSGD { } val sc = new SparkContext(args(0), "Lasso") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = Lasso_LocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = LassoLocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() } diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index 827ca66330b..144b8b1bc7f 100644 --- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -1,6 +1,3 @@ -<<<<<<< HEAD:mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala -package spark.mllib.classification -======= /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -18,8 +15,7 @@ package spark.mllib.classification * limitations under the License. */ -package spark.mllib.regression ->>>>>>> FETCH_HEAD:mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala +package spark.mllib.classification import scala.util.Random @@ -37,13 +33,6 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } -<<<<<<< HEAD:mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala - // Test if we can correctly learn A, B where Y = logistic(A + B*X) - test("LogisticRegression_LocalRandomSGD") { - val nPoints = 10000 - val rnd = new Random(42) - -======= // Generate input of the form Y = logistic(offset + scale*X) def generateLogisticInput( offset: Double, @@ -51,7 +40,6 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { nPoints: Int, seed: Int): Seq[(Double, Array[Double])] = { val rnd = new Random(seed) ->>>>>>> FETCH_HEAD:mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala 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) @@ -91,12 +79,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { val testRDD = sc.parallelize(testData, 2) testRDD.cache() -<<<<<<< HEAD:mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala - val lr = new LogisticRegression_LocalRandomSGD().setStepSize(10.0) - .setNumIterations(20) -======= - val lr = new LogisticRegression().setStepSize(10.0).setNumIterations(20) ->>>>>>> FETCH_HEAD:mllib/src/test/scala/spark/mllib/regression/LogisticRegressionSuite.scala + val lr = new LogisticRegressionLocalRandomSGD().setStepSize(10.0).setNumIterations(20) val model = lr.train(testRDD) @@ -128,7 +111,7 @@ 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 LogisticRegressionLocalRandomSGD().setStepSize(10.0).setNumIterations(10) val model = lr.train(testRDD, initialWeights) diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index 50cf260f490..0d781c310c0 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -19,7 +19,7 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } - test("SVM_LocalRandomSGD") { + test("SVMLocalRandomSGD") { val nPoints = 10000 val rnd = new Random(42) @@ -46,7 +46,7 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { writer_data.write("\n")}) writer_data.close() - val svm = new SVM_LocalRandomSGD().setStepSize(1.0) + val svm = new SVMLocalRandomSGD().setStepSize(1.0) .setRegParam(1.0) .setNumIterations(100) diff --git a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala index 9836ac54c12..0c39e1e09b2 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala @@ -17,7 +17,7 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } - test("Lasso_LocalRandomSGD") { + test("LassoLocalRandomSGD") { val nPoints = 10000 val rnd = new Random(42) @@ -36,14 +36,14 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val ls = new Lasso_LocalRandomSGD().setStepSize(1.0) + val ls = new LassoLocalRandomSGD().setStepSize(1.0) .setRegParam(0.01) .setNumIterations(20) val model = ls.train(testRDD) - val weight0 = model.weights.get(0) - val weight1 = model.weights.get(1) + val weight0 = model.weights(0) + val weight1 = model.weights(1) assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]") assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") From 72ff62a37c7310bab02f0231e91d3ba4d423217a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 28 Jul 2013 22:21:04 -0400 Subject: [PATCH 0499/2521] Two fixes to IPython support: - Don't attempt to run worker processes with ipython (that can cause some crashes as ipython prints things to standard out) - Allow passing some IPYTHON_OPTS to launch things like the notebook --- pyspark | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/pyspark b/pyspark index 37a355462e7..801239c108b 100755 --- a/pyspark +++ b/pyspark @@ -53,9 +53,13 @@ if [[ "$SPARK_LAUNCH_WITH_SCALA" != "0" ]] ; then export SPARK_LAUNCH_WITH_SCALA=1 fi +if [ -n "$IPYTHON_OPTS" ]; then + IPYTHON=1 +fi + if [[ "$IPYTHON" = "1" ]] ; then - export PYSPARK_PYTHON="ipython" - exec "$PYSPARK_PYTHON" -i -c "%run $PYTHONSTARTUP" + IPYTHON_OPTS=${IPYTHON_OPTS:--i} + exec ipython "$IPYTHON_OPTS" -c "%run $PYTHONSTARTUP" else - exec "$PYSPARK_PYTHON" "$@" + exec "$PYSPARK_PYTHON" "$@" fi From 29e042940ac79e42e2f8818ceda6a962a76948ac Mon Sep 17 00:00:00 2001 From: Xinghao Date: Sun, 28 Jul 2013 20:39:52 -0700 Subject: [PATCH 0500/2521] Move data generators to util --- .../LassoGenerator.scala => util/LassoDataGenerator.scala} | 0 .../SVMGenerator.scala => util/SVMDataGenerator.scala} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename mllib/src/main/scala/spark/mllib/{regression/LassoGenerator.scala => util/LassoDataGenerator.scala} (100%) rename mllib/src/main/scala/spark/mllib/{classification/SVMGenerator.scala => util/SVMDataGenerator.scala} (100%) diff --git a/mllib/src/main/scala/spark/mllib/regression/LassoGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala similarity index 100% rename from mllib/src/main/scala/spark/mllib/regression/LassoGenerator.scala rename to mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala diff --git a/mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala b/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala similarity index 100% rename from mllib/src/main/scala/spark/mllib/classification/SVMGenerator.scala rename to mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala From 67de051bbb81096dc37ea6f92a82a9224b4af61e Mon Sep 17 00:00:00 2001 From: Xinghao Date: Sun, 28 Jul 2013 21:09:56 -0700 Subject: [PATCH 0501/2521] SVMSuite and LassoSuite rewritten to follow closely with LogisticRegressionSuite --- .../spark/mllib/classification/SVMSuite.scala | 99 ++++++++++++++----- .../spark/mllib/regression/LassoSuite.scala | 97 +++++++++++++++--- 2 files changed, 161 insertions(+), 35 deletions(-) diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index 0d781c310c0..2a23825accc 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package spark.mllib.classification import scala.util.Random @@ -7,7 +24,6 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext -import spark.SparkContext._ import java.io._ @@ -19,43 +35,82 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } + // Generate noisy input of the form Y = signum(x.dot(weights) + intercept + noise) + def generateSVMInput( + intercept: Double, + weights: Array[Double], + nPoints: Int, + seed: Int): Seq[(Double, Array[Double])] = { + val rnd = new Random(seed) + val x = Array.fill[Array[Double]](nPoints)(Array.fill[Double](weights.length)(rnd.nextGaussian())) + val y = x.map(xi => + signum((xi zip weights).map(xw => xw._1*xw._2).reduce(_+_) + intercept + 0.1 * rnd.nextGaussian()) + ) + y zip x + } + + def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { + 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(numOffPredictions < input.length / 5) + } + test("SVMLocalRandomSGD") { val nPoints = 10000 - val rnd = new Random(42) - val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - val x2 = Array.fill[Double](nPoints)(rnd.nextGaussian()) + val A = 2.0 + val B = -1.5 + val C = 1.0 + + val testData = generateSVMInput(A, Array[Double](B,C), nPoints, 42) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + val svm = new SVMLocalRandomSGD().setStepSize(1.0).setRegParam(1.0).setNumIterations(100) + + val model = svm.train(testRDD) + + val validationData = generateSVMInput(A, Array[Double](B,C), nPoints, 17) + val validationRDD = sc.parallelize(validationData,2) + + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) + } + + test("SVMLocalRandomSGD with initial weights") { + val nPoints = 10000 val A = 2.0 val B = -1.5 val C = 1.0 - val y = (0 until nPoints).map { i => - signum(A + B * x1(i) + C * x2(i) + 0.0*rnd.nextGaussian()) - } + val testData = generateSVMInput(A, Array[Double](B,C), nPoints, 42) - val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i),x2(i)))).toArray + val initialB = -1.0 + val initialC = -1.0 + val initialWeights = Array(initialB,initialC) val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val writer_data = new PrintWriter(new File("svmtest.dat")) - testData.foreach(yx => { - writer_data.write(yx._1 + "") - yx._2.foreach(xi => writer_data.write("\t" + xi)) - writer_data.write("\n")}) - writer_data.close() + val svm = new SVMLocalRandomSGD().setStepSize(1.0).setRegParam(1.0).setNumIterations(100) - val svm = new SVMLocalRandomSGD().setStepSize(1.0) - .setRegParam(1.0) - .setNumIterations(100) - - val model = svm.train(testRDD) + val model = svm.train(testRDD, initialWeights) - val yPredict = (0 until nPoints).map(i => model.predict(Array(x1(i),x2(i)))) + val validationData = generateSVMInput(A, Array[Double](B,C), nPoints, 17) + val validationRDD = sc.parallelize(validationData,2) - val accuracy = ((y zip yPredict).map(yy => if (yy._1==yy._2) 1 else 0).reduceLeft(_+_).toDouble / nPoints.toDouble) + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) - assert(accuracy >= 0.90, "Accuracy (" + accuracy + ") too low") + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) } } diff --git a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala index 0c39e1e09b2..33e87dfd9fc 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR 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 @@ -6,7 +23,6 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext -import spark.SparkContext._ class LassoSuite extends FunSuite with BeforeAndAfterAll { @@ -17,35 +33,90 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } + // Generate noisy input of the form Y = x.dot(weights) + intercept + noise + def generateLassoInput( + intercept: Double, + weights: Array[Double], + nPoints: Int, + seed: Int): Seq[(Double, Array[Double])] = { + val rnd = new Random(seed) + val x = Array.fill[Array[Double]](nPoints)(Array.fill[Double](weights.length)(rnd.nextGaussian())) + val y = x.map(xi => (xi zip weights).map(xw => xw._1*xw._2).reduce(_+_) + intercept + 0.1 * rnd.nextGaussian()) + y zip x + } + + def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { + 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(numOffPredictions < input.length / 5) + } + test("LassoLocalRandomSGD") { val nPoints = 10000 - val rnd = new Random(42) - - val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - val x2 = Array.fill[Double](nPoints)(rnd.nextGaussian()) val A = 2.0 val B = -1.5 val C = 1.0e-2 - val y = (0 until nPoints).map { i => - A + B * x1(i) + C * x2(i) + 0.1*rnd.nextGaussian() - } - - val testData = (0 until nPoints).map(i => (y(i).toDouble, Array(x1(i),x2(i)))).toArray + val testData = generateLassoInput(A, Array[Double](B,C), nPoints, 42) val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val ls = new LassoLocalRandomSGD().setStepSize(1.0) - .setRegParam(0.01) - .setNumIterations(20) + val ls = new LassoLocalRandomSGD().setStepSize(1.0).setRegParam(0.01).setNumIterations(20) val model = ls.train(testRDD) val weight0 = model.weights(0) val weight1 = model.weights(1) + assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]") + + val validationData = generateLassoInput(A, Array[Double](B,C), nPoints, 17) + val validationRDD = sc.parallelize(validationData,2) + + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) + } + + test("LassoLocalRandomSGD with initial weights") { + val nPoints = 10000 + + val A = 2.0 + val B = -1.5 + val C = 1.0e-2 + + val testData = generateLassoInput(A, Array[Double](B,C), nPoints, 42) + + val initialB = -1.0 + val initialC = -1.0 + val initialWeights = Array(initialB,initialC) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + val ls = new LassoLocalRandomSGD().setStepSize(1.0).setRegParam(0.01).setNumIterations(20) + + val model = ls.train(testRDD, initialWeights) + + val weight0 = model.weights(0) + val weight1 = model.weights(1) assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") + assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]") + + val validationData = generateLassoInput(A, Array[Double](B,C), nPoints, 17) + val validationRDD = sc.parallelize(validationData,2) + + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_._2)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row._2)), validationData) } } From 9398dced0331c0ec098ef5eb4616571874ceefb6 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Sun, 28 Jul 2013 21:39:19 -0700 Subject: [PATCH 0502/2521] Changed Classification to return Int instead of Double Also minor changes to formatting and comments --- .../mllib/classification/Classification.scala | 8 ++++---- .../classification/LogisticRegression.scala | 16 ++++------------ .../scala/spark/mllib/classification/SVM.scala | 8 ++++---- .../spark/mllib/optimization/Gradient.scala | 4 ++-- .../mllib/optimization/GradientDescent.scala | 8 ++++---- .../scala/spark/mllib/optimization/Updater.scala | 12 ++++++++++-- .../spark/mllib/classification/SVMSuite.scala | 2 -- 7 files changed, 28 insertions(+), 30 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/Classification.scala b/mllib/src/main/scala/spark/mllib/classification/Classification.scala index 96d7a54f187..d6154b66aed 100644 --- a/mllib/src/main/scala/spark/mllib/classification/Classification.scala +++ b/mllib/src/main/scala/spark/mllib/classification/Classification.scala @@ -7,15 +7,15 @@ trait ClassificationModel extends Serializable { * Predict values for the given data set using the model trained. * * @param testData RDD representing data points to be predicted - * @return RDD[Double] where each entry contains the corresponding prediction + * @return RDD[Int] where each entry contains the corresponding prediction */ - def predict(testData: RDD[Array[Double]]): RDD[Double] + def predict(testData: RDD[Array[Double]]): RDD[Int] /** * 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 + * @return Int prediction from the trained model */ - def predict(testData: Array[Double]): Double + def predict(testData: Array[Double]): Int } diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index 1b093187f29..0a7effb1d77 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -35,21 +35,21 @@ class LogisticRegressionModel( // 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]]) = { + override def predict(testData: spark.RDD[Array[Double]]): RDD[Int] = { // 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(localWeights).get(0) + localIntercept - 1.0/ (1.0 + math.exp(margin * -1)) + (1.0/ (1.0 + math.exp(margin * -1))).toInt } } - override def predict(testData: Array[Double]): Double = { + override def predict(testData: Array[Double]): Int = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) val margin = dataMat.mmul(weightsMatrix).get(0) + this.intercept - 1.0/ (1.0 + math.exp(margin * -1)) + (1.0/ (1.0 + math.exp(margin * -1))).toInt } } @@ -70,14 +70,6 @@ class LogisticRegressionLocalRandomSGD private (var stepSize: Double, var miniBa this } - - - - - - - - /** * Set fraction of data to be used for each SGD iteration. Default 1.0. */ diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index 76844f6b9c3..30766a4c64c 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -35,19 +35,19 @@ class SVMModel( // 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]]) = { + override def predict(testData: spark.RDD[Array[Double]]): RDD[Int] = { // 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 => - signum(new DoubleMatrix(1, x.length, x:_*).dot(localWeights) + localIntercept) + signum(new DoubleMatrix(1, x.length, x:_*).dot(localWeights) + localIntercept).toInt } } - override def predict(testData: Array[Double]): Double = { + override def predict(testData: Array[Double]): Int = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) - signum(dataMat.dot(weightsMatrix) + this.intercept) + signum(dataMat.dot(weightsMatrix) + this.intercept).toInt } } diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala index 4864ab7ccfc..22b2ec5ed60 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala @@ -70,8 +70,8 @@ class HingeGradient extends Gradient { val dotProduct = data.dot(weights) if (1.0 > label * dotProduct) - (data.mul(-label), 1.0 - label * dotProduct) + (data.mul(-label), 1.0 - label * dotProduct) else - (DoubleMatrix.zeros(1,weights.length), 0.0) + (DoubleMatrix.zeros(1,weights.length), 0.0) } } diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index 8387d4939b9..d4b83a14561 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -76,10 +76,10 @@ object GradientDescent { weights = update._1 reg_val = update._2 stochasticLossHistory.append(lossSum / miniBatchSize + reg_val) - /*** - Xinghao: The loss here is sum of lossSum computed using the weights before applying updater, - and reg_val using weights after applying updater - ***/ + /* + * NOTE(Xinghao): The loss here is sum of lossSum computed using the weights before applying updater, + * and reg_val using weights after applying updater + */ } (weights.toArray, stochasticLossHistory.toArray) diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index cd344a66804..188fe7d972d 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -46,17 +46,25 @@ class SimpleUpdater extends Updater { } /** -L1 regularization -- corresponding proximal operator is the soft-thresholding function +* L1 regularization -- corresponding proximal operator is the soft-thresholding function +* That is, each weight component is shrunk towards 0 by shrinkageVal +* If w > shrinkageVal, set weight component to w-shrinkageVal. +* If w < -shrinkageVal, set weight component to w+shrinkageVal. +* If -shrinkageVal < w < shrinkageVal, set weight component to 0. +* Equivalently, set weight component to signum(w) * max(0.0, abs(w) - shrinkageVal) **/ class L1Updater extends Updater { override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { val thisIterStepSize = stepSize / math.sqrt(iter) val normGradient = gradient.mul(thisIterStepSize) + // Take gradient step val newWeights = weightsOld.sub(normGradient) + // Soft thresholding + val shrinkageVal = regParam * thisIterStepSize (0 until newWeights.length).foreach(i => { val wi = newWeights.get(i) - newWeights.put(i, signum(wi) * max(0.0, abs(wi) - regParam * thisIterStepSize)) + newWeights.put(i, signum(wi) * max(0.0, abs(wi) - shrinkageVal)) }) (newWeights, newWeights.norm1 * regParam) } diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index 2a23825accc..91c037e9b16 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -25,8 +25,6 @@ import org.scalatest.FunSuite import spark.SparkContext -import java.io._ - class SVMSuite extends FunSuite with BeforeAndAfterAll { val sc = new SparkContext("local", "test") From 96e04f4cb7de3a7c9d31aa7acba496d81066634e Mon Sep 17 00:00:00 2001 From: Xinghao Date: Sun, 28 Jul 2013 22:12:39 -0700 Subject: [PATCH 0503/2521] Fixed SVM and LR train functions to take Int instead of Double for Classification --- .../classification/LogisticRegression.scala | 17 ++++++++--------- .../scala/spark/mllib/classification/SVM.scala | 16 ++++++++-------- .../LogisticRegressionSuite.scala | 10 +++++----- 3 files changed, 21 insertions(+), 22 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index 0a7effb1d77..cbc0d03ae1a 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -86,19 +86,19 @@ class LogisticRegressionLocalRandomSGD private (var stepSize: Double, var miniBa this } - def train(input: RDD[(Double, Array[Double])]): LogisticRegressionModel = { + def train(input: RDD[(Int, 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])], + input: RDD[(Int, 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:_*)) + (y.toDouble, Array(1.0, features:_*)) } val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) @@ -141,13 +141,12 @@ object LogisticRegressionLocalRandomSGD { * @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])], + input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, @@ -170,7 +169,7 @@ object LogisticRegressionLocalRandomSGD { * @param miniBatchFraction Fraction of data to be used per iteration. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, @@ -192,7 +191,7 @@ object LogisticRegressionLocalRandomSGD { * @return a LogisticRegressionModel which has the weights and offset from training. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double ) @@ -211,7 +210,7 @@ object LogisticRegressionLocalRandomSGD { * @return a LogisticRegressionModel which has the weights and offset from training. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int) : LogisticRegressionModel = { @@ -224,7 +223,7 @@ object LogisticRegressionLocalRandomSGD { System.exit(1) } val sc = new SparkContext(args(0), "LogisticRegression") - val data = MLUtils.loadLabeledData(sc, args(1)) + val data = MLUtils.loadLabeledData(sc, args(1)).map(yx => (yx._1.toInt, yx._2)) val model = LogisticRegressionLocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index 30766a4c64c..15b689e7e00 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -94,19 +94,19 @@ class SVMLocalRandomSGD private (var stepSize: Double, var regParam: Double, var this } - def train(input: RDD[(Double, Array[Double])]): SVMModel = { + def train(input: RDD[(Int, Array[Double])]): SVMModel = { 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])], + input: RDD[(Int, Array[Double])], initialWeights: Array[Double]): SVMModel = { // 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:_*)) + (y.toDouble, Array(1.0, features:_*)) } val initalWeightsWithIntercept = Array(1.0, initialWeights:_*) @@ -155,7 +155,7 @@ object SVMLocalRandomSGD { * the number of features in the data. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, regParam: Double, @@ -178,7 +178,7 @@ object SVMLocalRandomSGD { * @param miniBatchFraction Fraction of data to be used per iteration. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, regParam: Double, @@ -200,7 +200,7 @@ object SVMLocalRandomSGD { * @return a SVMModel which has the weights and offset from training. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, regParam: Double) @@ -219,7 +219,7 @@ object SVMLocalRandomSGD { * @return a SVMModel which has the weights and offset from training. */ def train( - input: RDD[(Double, Array[Double])], + input: RDD[(Int, Array[Double])], numIterations: Int) : SVMModel = { @@ -232,7 +232,7 @@ object SVMLocalRandomSGD { System.exit(1) } val sc = new SparkContext(args(0), "SVM") - val data = MLUtils.loadLabeledData(sc, args(1)) + val data = MLUtils.loadLabeledData(sc, args(1)).map(yx => (yx._1.toInt, yx._2)) val model = SVMLocalRandomSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) sc.stop() diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index 144b8b1bc7f..3aa9fe6d121 100644 --- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -38,7 +38,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { offset: Double, scale: Double, nPoints: Int, - seed: Int): Seq[(Double, Array[Double])] = { + seed: Int): Seq[(Int, Array[Double])] = { val rnd = new Random(seed) val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) @@ -51,19 +51,19 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { // y <- A + B*x + rLogis() // y <- as.numeric(y > 0) - val y: Seq[Double] = (0 until nPoints).map { i => + val y: Seq[Int] = (0 until nPoints).map { i => val yVal = offset + scale * x1(i) + rLogis(i) - if (yVal > 0) 1.0 else 0.0 + if (yVal > 0) 1 else 0 } val testData = (0 until nPoints).map(i => (y(i), Array(x1(i)))) testData } - def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { + def validatePrediction(predictions: Seq[Int], input: Seq[(Int, Array[Double])]) { 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 + math.abs(prediction.toDouble - expected.toDouble) > 0.5 }.size // At least 80% of the predictions should be on. assert(numOffPredictions < input.length / 5) From c823ee1e2bea7cde61cb4411a0f0db91f1df2af2 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Sun, 28 Jul 2013 22:17:53 -0700 Subject: [PATCH 0504/2521] Replace map-reduce with dot operator using DoubleMatrix --- .../main/scala/spark/mllib/util/LassoDataGenerator.scala | 4 ++-- .../main/scala/spark/mllib/util/SVMDataGenerator.scala | 6 ++++-- .../test/scala/spark/mllib/classification/SVMSuite.scala | 9 ++++++--- .../test/scala/spark/mllib/regression/LassoSuite.scala | 7 ++++++- 4 files changed, 18 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala index d2d3bb33c7d..1ff32d2e7f6 100644 --- a/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala @@ -26,7 +26,7 @@ object LassoGenerator { val sc = new SparkContext(sparkMaster, "LassoGenerator") val globalRnd = new Random(94720) - val trueWeights = Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() } + val trueWeights = new DoubleMatrix(1, nfeatures+1, Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) @@ -34,7 +34,7 @@ object LassoGenerator { val x = Array.fill[Double](nfeatures) { rnd.nextDouble() * 2.0 - 1.0 } - val y = ((1.0 +: x) zip trueWeights).map{wx => wx._1 * wx._2}.reduceLeft(_+_) + rnd.nextGaussian() * 0.1 + val y = (new DoubleMatrix(1, x.length, x:_*)).dot(trueWeights) + rnd.nextGaussian() * 0.1 (y, x) } diff --git a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala index 029f262660e..ced52093f55 100644 --- a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala @@ -8,6 +8,8 @@ import org.jblas.DoubleMatrix import spark.{RDD, SparkContext} import spark.mllib.util.MLUtils +import org.jblas.DoubleMatrix + object SVMGenerator { def main(args: Array[String]) { @@ -27,7 +29,7 @@ object SVMGenerator { val sc = new SparkContext(sparkMaster, "SVMGenerator") val globalRnd = new Random(94720) - val trueWeights = Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() } + val trueWeights = new DoubleMatrix(1, nfeatures+1, Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) @@ -35,7 +37,7 @@ object SVMGenerator { val x = Array.fill[Double](nfeatures) { rnd.nextDouble() * 2.0 - 1.0 } - val y = signum(((1.0 +: x) zip trueWeights).map{wx => wx._1 * wx._2}.reduceLeft(_+_) + rnd.nextGaussian() * 0.1) + val y = signum((new DoubleMatrix(1, x.length, x:_*)).dot(trueWeights) + rnd.nextGaussian() * 0.1) (y, x) } diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index 91c037e9b16..3f00398a0a9 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -25,6 +25,8 @@ import org.scalatest.FunSuite import spark.SparkContext +import org.jblas.DoubleMatrix + class SVMSuite extends FunSuite with BeforeAndAfterAll { val sc = new SparkContext("local", "test") @@ -38,16 +40,17 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { intercept: Double, weights: Array[Double], nPoints: Int, - seed: Int): Seq[(Double, Array[Double])] = { + seed: Int): Seq[(Int, Array[Double])] = { val rnd = new Random(seed) + val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) val x = Array.fill[Array[Double]](nPoints)(Array.fill[Double](weights.length)(rnd.nextGaussian())) val y = x.map(xi => - signum((xi zip weights).map(xw => xw._1*xw._2).reduce(_+_) + intercept + 0.1 * rnd.nextGaussian()) + signum((new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) + intercept + 0.1 * rnd.nextGaussian()).toInt ) y zip x } - def validatePrediction(predictions: Seq[Double], input: Seq[(Double, Array[Double])]) { + def validatePrediction(predictions: Seq[Int], input: Seq[(Int, Array[Double])]) { 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 diff --git a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala index 33e87dfd9fc..cf2b067d40f 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala @@ -24,6 +24,8 @@ import org.scalatest.FunSuite import spark.SparkContext +import org.jblas.DoubleMatrix + class LassoSuite extends FunSuite with BeforeAndAfterAll { val sc = new SparkContext("local", "test") @@ -40,8 +42,11 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { nPoints: Int, seed: Int): Seq[(Double, Array[Double])] = { val rnd = new Random(seed) + val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) val x = Array.fill[Array[Double]](nPoints)(Array.fill[Double](weights.length)(rnd.nextGaussian())) - val y = x.map(xi => (xi zip weights).map(xw => xw._1*xw._2).reduce(_+_) + intercept + 0.1 * rnd.nextGaussian()) + val y = x.map(xi => + (new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) + intercept + 0.1 * rnd.nextGaussian() + ) y zip x } From b9d6783f36d527f5082bf13a4ee6fd108e97795c Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 28 Jul 2013 23:28:42 -0400 Subject: [PATCH 0505/2521] Optimize Python take() to not compute entire first partition --- .../scala/spark/api/python/PythonRDD.scala | 64 +++++++++++-------- python/pyspark/rdd.py | 15 +++-- 2 files changed, 45 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index af10822dbd2..2dd79f7100e 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -63,34 +63,42 @@ private[spark] class PythonRDD[T: ClassManifest]( // 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 stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) - val dataOut = new DataOutputStream(stream) - val printOut = new PrintWriter(stream) - // Partition index - dataOut.writeInt(split.index) - // sparkFilesDir - PythonRDD.writeAsPickle(SparkFiles.getRootDirectory, dataOut) - // Broadcast variables - dataOut.writeInt(broadcastVars.length) - for (broadcast <- broadcastVars) { - dataOut.writeLong(broadcast.id) - dataOut.writeInt(broadcast.value.length) - dataOut.write(broadcast.value) - } - dataOut.flush() - // Serialized user code - for (elem <- command) { - printOut.println(elem) - } - printOut.flush() - // Data values - for (elem <- parent.iterator(split, context)) { - PythonRDD.writeAsPickle(elem, dataOut) + try { + SparkEnv.set(env) + val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) + val dataOut = new DataOutputStream(stream) + val printOut = new PrintWriter(stream) + // Partition index + dataOut.writeInt(split.index) + // sparkFilesDir + PythonRDD.writeAsPickle(SparkFiles.getRootDirectory, dataOut) + // Broadcast variables + dataOut.writeInt(broadcastVars.length) + for (broadcast <- broadcastVars) { + dataOut.writeLong(broadcast.id) + dataOut.writeInt(broadcast.value.length) + dataOut.write(broadcast.value) + } + dataOut.flush() + // Serialized user code + for (elem <- command) { + printOut.println(elem) + } + printOut.flush() + // Data values + for (elem <- parent.iterator(split, context)) { + PythonRDD.writeAsPickle(elem, dataOut) + } + dataOut.flush() + printOut.flush() + worker.shutdownOutput() + } catch { + case e: IOException => + // This can happen for legitimate reasons if the Python code stops returning data before we are done + // passing elements through, e.g., for take(). Just log a message to say it happened. + logInfo("stdin writer to Python finished early") + logDebug("stdin writer to Python finished early", e) } - dataOut.flush() - printOut.flush() - worker.shutdownOutput() } }.start() @@ -297,7 +305,7 @@ class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int) 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 override def addInPlace(val1: JList[Array[Byte]], val2: JList[Array[Byte]]) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index c6a6b24c5a6..6efa61aa665 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -386,13 +386,16 @@ def take(self, num): >>> sc.parallelize([2, 3, 4, 5, 6]).take(10) [2, 3, 4, 5, 6] """ + def takeUpToNum(iterator): + taken = 0 + while taken < num: + yield next(iterator) + taken += 1 + # Take only up to num elements from each partition we try + mapped = self.mapPartitions(takeUpToNum) items = [] - for partition in range(self._jrdd.splits().size()): - iterator = self.ctx._takePartition(self._jrdd.rdd(), partition) - # Each item in the iterator is a string, Python object, batch of - # Python objects. Regardless, it is sufficient to take `num` - # of these objects in order to collect `num` Python objects: - iterator = iterator.take(num) + for partition in range(mapped._jrdd.splits().size()): + iterator = self.ctx._takePartition(mapped._jrdd.rdd(), partition) items.extend(self._collect_iterator_through_file(iterator)) if len(items) >= num: break From b5ec35562210c8e7ca4fea07a0d46cb255988c0d Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 28 Jul 2013 23:38:56 -0400 Subject: [PATCH 0506/2521] Optimize Python foreach() to not return as many objects --- python/pyspark/rdd.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 6efa61aa665..4aafe35d134 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -267,7 +267,11 @@ def foreach(self, f): >>> def f(x): print x >>> sc.parallelize([1, 2, 3, 4, 5]).foreach(f) """ - self.map(f).collect() # Force evaluation + def processPartition(iterator): + for x in iterator: + f(x) + yield None + self.mapPartitions(processPartition).collect() # Force evaluation def collect(self): """ From 96b50e82dc0db501ab94748a23ae3237f1d82034 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 28 Jul 2013 23:39:27 -0400 Subject: [PATCH 0507/2521] Allow python/run-tests to run from any directory --- python/run-tests | 3 +++ 1 file changed, 3 insertions(+) diff --git a/python/run-tests b/python/run-tests index 1ee947d4145..6643faa2e03 100755 --- a/python/run-tests +++ b/python/run-tests @@ -21,6 +21,9 @@ # Figure out where the Spark framework is installed FWDIR="$(cd `dirname $0`; cd ../; pwd)" +# CD into the python directory to find things on the right path +cd "$FWDIR/python" + FAILED=0 $FWDIR/pyspark pyspark/rdd.py From d75c3086951f603ec30b2527c24559e053ed7f25 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 28 Jul 2013 23:50:38 -0400 Subject: [PATCH 0508/2521] Use None instead of empty string as it's slightly smaller/faster --- python/pyspark/rdd.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 4aafe35d134..8734cacb0ba 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -160,7 +160,7 @@ def distinct(self): >>> sorted(sc.parallelize([1, 1, 2, 3]).distinct().collect()) [1, 2, 3] """ - return self.map(lambda x: (x, "")) \ + return self.map(lambda x: (x, None)) \ .reduceByKey(lambda x, _: x) \ .map(lambda (x, _): x) From feba7ee540fca28872957120e5e39b9e36466953 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 29 Jul 2013 00:09:11 -0400 Subject: [PATCH 0509/2521] SPARK-815. Python parallelize() should split lists before batching One unfortunate consequence of this fix is that we materialize any collections that are given to us as generators, but this seems necessary to get reasonable behavior on small collections. We could add a batchSize parameter later to bypass auto-computation of batch size if this becomes a problem (e.g. if users really want to parallelize big generators nicely) --- python/pyspark/context.py | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 2f741cb3451..c2b49ff37ac 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -141,14 +141,21 @@ def stop(self): def parallelize(self, c, numSlices=None): """ Distribute a local Python collection to form an RDD. + + >>> sc.parallelize(range(5), 5).glom().collect() + [[0], [1], [2], [3], [4]] """ numSlices = numSlices or self.defaultParallelism # Calling the Java parallelize() method with an ArrayList is too slow, # because it sends O(n) Py4J commands. As an alternative, serialized # objects are written to a file and loaded through textFile(). tempFile = NamedTemporaryFile(delete=False, dir=self._temp_dir) - if self.batchSize != 1: - c = batched(c, self.batchSize) + # Make sure we distribute data evenly if it's smaller than self.batchSize + if "__len__" not in dir(c): + c = list(c) # Make it a list so we can compute its length + batchSize = min(len(c) // numSlices, self.batchSize) + if batchSize > 1: + c = batched(c, batchSize) for x in c: write_with_length(dump_pickle(x), tempFile) tempFile.close() From 497f55755f8374dd553f21cc3c5edcc6e3f5acce Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 29 Jul 2013 02:50:48 -0400 Subject: [PATCH 0510/2521] Add docs about ipython --- docs/python-programming-guide.md | 34 +++++++++++++++++++++++++++++--- 1 file changed, 31 insertions(+), 3 deletions(-) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index e8aaac74d03..794bff56474 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -10,6 +10,7 @@ To learn the basics of Spark, we recommend reading through the easy to follow even if you don't know Scala. This guide will show how to use the Spark features described there in Python. + # Key Differences in the Python API There are a few key differences between the Python and Scala APIs: @@ -50,6 +51,7 @@ PySpark will automatically ship these functions to workers, along with any objec Instances of classes will be serialized and shipped to workers by PySpark, but classes themselves cannot be automatically distributed to workers. The [Standalone Use](#standalone-use) section describes how to ship code dependencies to workers. + # Installing and Configuring PySpark PySpark requires Python 2.6 or higher. @@ -81,16 +83,41 @@ The Python shell can be used explore data interactively and is a simple way to l >>> help(pyspark) # Show all pyspark functions {% endhighlight %} -By default, the `pyspark` shell creates SparkContext that runs jobs locally. -To connect to a non-local cluster, set the `MASTER` environment variable. +By default, the `pyspark` shell creates SparkContext that runs jobs locally on a single core. +To connect to a non-local cluster, or use multiple cores, set the `MASTER` environment variable. For example, to use the `pyspark` shell with a [standalone Spark cluster](spark-standalone.html): {% highlight bash %} $ MASTER=spark://IP:PORT ./pyspark {% endhighlight %} +Or, to use four cores on the local machine: + +{% highlight bash %} +$ MASTER=local[4] ./pyspark +{% endhighlight %} + + +## IPython + +It is also possible to launch PySpark in [IPython](http://ipython.org), the enhanced Python interpreter. +To do this, simply set the `IPYTHON` variable to `1` when running `pyspark`: + +{% highlight bash %} +$ IPYTHON=1 ./pyspark +{% endhighlight %} + +Alternatively, you can customize the `ipython` command by setting `IPYTHON_OPTS`. For example, to launch +the [IPython Notebook](http://ipython.org/notebook.html) with PyLab graphing support: -# Standalone Use +{% highlight bash %} +$ IPYTHON_OPTS="notebook --pylab inline" ./pyspark +{% endhighlight %} + +IPython also works on a cluster or on multiple cores if you set the `MASTER` environment variable. + + +# Standalone Programs PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `pyspark`. The Quick Start guide includes a [complete example](quick-start.html#a-standalone-job-in-python) of a standalone Python job. @@ -105,6 +132,7 @@ sc = SparkContext("local", "Job Name", pyFiles=['MyFile.py', 'lib.zip', 'app.egg Files listed here will be added to the `PYTHONPATH` and shipped to remote worker machines. Code dependencies can be added to an existing SparkContext using its `addPyFile()` method. + # Where to Go from Here PySpark includes several sample programs in the [`python/examples` folder](https://github.com/mesos/spark/tree/master/python/examples). From 75f375730025788a5982146d97bf3df9ef69ab23 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Mon, 29 Jul 2013 09:19:56 -0700 Subject: [PATCH 0511/2521] Fix rounding error in LogisticRegression.scala --- .../spark/mllib/classification/LogisticRegression.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index cbc0d03ae1a..bc1c3277292 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -21,6 +21,8 @@ import spark.{Logging, RDD, SparkContext} import spark.mllib.optimization._ import spark.mllib.util.MLUtils +import scala.math.round + import org.jblas.DoubleMatrix /** @@ -42,14 +44,14 @@ class LogisticRegressionModel( val localIntercept = intercept testData.map { x => val margin = new DoubleMatrix(1, x.length, x:_*).mmul(localWeights).get(0) + localIntercept - (1.0/ (1.0 + math.exp(margin * -1))).toInt + round(1.0/ (1.0 + math.exp(margin * -1))).toInt } } override def predict(testData: Array[Double]): Int = { val dataMat = new DoubleMatrix(1, testData.length, testData:_*) val margin = dataMat.mmul(weightsMatrix).get(0) + this.intercept - (1.0/ (1.0 + math.exp(margin * -1))).toInt + round(1.0/ (1.0 + math.exp(margin * -1))).toInt } } From 3a8d07df8ca5bccdbed178991dd12fde74802542 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Mon, 29 Jul 2013 09:20:26 -0700 Subject: [PATCH 0512/2521] Deleting extra LogisticRegressionGenerator and RidgeRegressionGenerator --- .../LogisticRegressionGenerator.scala | 41 -------------- .../regression/RidgeRegressionGenerator.scala | 55 ------------------- 2 files changed, 96 deletions(-) delete mode 100644 mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala delete mode 100644 mllib/src/main/scala/spark/mllib/regression/RidgeRegressionGenerator.scala diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala deleted file mode 100644 index cde1148adf4..00000000000 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegressionGenerator.scala +++ /dev/null @@ -1,41 +0,0 @@ -package spark.mllib.classification - -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 b83f505d8e4..00000000000 --- a/mllib/src/main/scala/spark/mllib/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.saveLabeledData(data, outputPath) - sc.stop() - } -} From 07f17439a52b65d4f5ef8c8d80bc25dadc0182a8 Mon Sep 17 00:00:00 2001 From: Xinghao Date: Mon, 29 Jul 2013 09:22:31 -0700 Subject: [PATCH 0513/2521] Fix validatePrediction functions for Classification models Classifiers return categorical (Int) values that should be compared directly --- .../spark/mllib/classification/LogisticRegressionSuite.scala | 3 +-- mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala index 3aa9fe6d121..d3fe58a382f 100644 --- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala @@ -62,8 +62,7 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll { def validatePrediction(predictions: Seq[Int], input: Seq[(Int, Array[Double])]) { 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.toDouble - expected.toDouble) > 0.5 + (prediction != expected) }.size // At least 80% of the predictions should be on. assert(numOffPredictions < input.length / 5) diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index 3f00398a0a9..d546e0729ee 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -52,8 +52,7 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { def validatePrediction(predictions: Seq[Int], input: Seq[(Int, Array[Double])]) { 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 + (prediction != expected) }.size // At least 80% of the predictions should be on. assert(numOffPredictions < input.length / 5) From 2b2630ba3c621f4121da8e76fe6fcfa69d3eb74c Mon Sep 17 00:00:00 2001 From: Xinghao Date: Mon, 29 Jul 2013 09:22:49 -0700 Subject: [PATCH 0514/2521] Style fix Lines shortened to < 100 characters --- mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala | 3 ++- mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala index 1ff32d2e7f6..ef4f42a494e 100644 --- a/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala @@ -26,7 +26,8 @@ object LassoGenerator { val sc = new SparkContext(sparkMaster, "LassoGenerator") val globalRnd = new Random(94720) - val trueWeights = new DoubleMatrix(1, nfeatures+1, Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) + val trueWeights = new DoubleMatrix(1, nfeatures+1, + Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) diff --git a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala index ced52093f55..00a54d9a702 100644 --- a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala @@ -29,7 +29,8 @@ object SVMGenerator { val sc = new SparkContext(sparkMaster, "SVMGenerator") val globalRnd = new Random(94720) - val trueWeights = new DoubleMatrix(1, nfeatures+1, Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) + val trueWeights = new DoubleMatrix(1, nfeatures+1, + Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) val data: RDD[(Double, Array[Double])] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) From 43a2cc15c0539c14732dc10220fc8bb2f152a111 Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 29 Jul 2013 13:37:24 -0700 Subject: [PATCH 0515/2521] Use Bootstrap progress bars in web UI --- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index f31af3cda6e..16390708ca2 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -76,16 +76,11 @@ private[spark] class IndexPage(parent: JobProgressUI) { } def makeProgressBar(completed: Int, total: Int): Seq[Node] = { - val width=130 - val height=15 - val completeWidth = (completed.toDouble / total) * width - - - - - + val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) + +
    +
    +
    } From 478a2886d95fe2fcaaad07790c9c422736b539ce Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 29 Jul 2013 14:51:07 -0700 Subject: [PATCH 0516/2521] Added started tasks to progress bar --- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 16390708ca2..f7f5c2fb6e1 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -75,11 +75,13 @@ private[spark] class IndexPage(parent: JobProgressUI) { } } - def makeProgressBar(completed: Int, total: Int): Seq[Node] = { + def makeProgressBar(started: Int, completed: Int, total: Int): Seq[Node] = { val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) + val startWidth = "width: %s%%".format((started.toDouble/total)*100)
    -
    +
    +
    } @@ -96,6 +98,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { case (false, true) => "Write" case _ => "" } + val startedTasks = listener.stageToTasksActive.getOrElse(s.id, Seq[Long]()).size val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) val totalTasks = s.numPartitions @@ -105,7 +108,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { {submissionTime} {getElapsedTime(s.submissionTime, s.completionTime.getOrElse(System.currentTimeMillis()))} - {makeProgressBar(completedTasks, totalTasks)} + {makeProgressBar(startedTasks, completedTasks, totalTasks)} {completedTasks} / {totalTasks} {listener.stageToTasksFailed.getOrElse(s.id, 0) match { case f if f > 0 => "(%s failed)".format(f) From 2d6da9195ab8feeaf3fc478ec09d22568d6aa64b Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 29 Jul 2013 15:50:52 -0700 Subject: [PATCH 0517/2521] Alphabetized imports --- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index c12a4fe2a5f..646ae5ecbc5 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -25,9 +25,9 @@ import scala.Some import scala.xml.{NodeSeq, Node} import spark.scheduler.Stage -import spark.ui.UIUtils._ -import spark.ui.Page._ import spark.storage.StorageLevel +import spark.ui.Page._ +import spark.ui.UIUtils._ import spark.Utils /** Page showing list of all ongoing and recently finished stages */ From 07da72b45190f7db9daa2c6bd33577d28e19e659 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 29 Jul 2013 16:25:17 -0700 Subject: [PATCH 0518/2521] Remove duplicate loss history and clarify why. Also some minor style fixes. --- .../mllib/classification/LogisticRegression.scala | 5 +---- .../mllib/optimization/GradientDescent.scala | 15 +++++++-------- .../scala/spark/mllib/optimization/Updater.scala | 2 +- 3 files changed, 9 insertions(+), 13 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index bc1c3277292..bf3b05dedb9 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -151,7 +151,6 @@ object LogisticRegressionLocalRandomSGD { input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, - miniBatchFraction: Double, initialWeights: Array[Double]) : LogisticRegressionModel = @@ -174,7 +173,6 @@ object LogisticRegressionLocalRandomSGD { input: RDD[(Int, Array[Double])], numIterations: Int, stepSize: Double, - miniBatchFraction: Double) : LogisticRegressionModel = { @@ -195,8 +193,7 @@ object LogisticRegressionLocalRandomSGD { def train( input: RDD[(Int, Array[Double])], numIterations: Int, - stepSize: Double - ) + stepSize: Double) : LogisticRegressionModel = { train(input, numIterations, stepSize, 1.0) diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala index d4b83a14561..19cda26446f 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala @@ -61,7 +61,7 @@ object GradientDescent { // Initialize weights as a column vector var weights = new DoubleMatrix(initialWeights.length, 1, initialWeights:_*) - var reg_val = 0.0 + var regVal = 0.0 for (i <- 1 to numIters) { val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42+i).map { @@ -71,15 +71,14 @@ object GradientDescent { (grad, loss) }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) - stochasticLossHistory.append(lossSum / miniBatchSize + reg_val) + /** + * NOTE(Xinghao): lossSum is computed using the weights from the previous iteration + * and regVal is the regularization value computed in the previous iteration as well. + */ + stochasticLossHistory.append(lossSum / miniBatchSize + regVal) val update = updater.compute(weights, gradientSum.div(miniBatchSize), stepSize, i, regParam) weights = update._1 - reg_val = update._2 - stochasticLossHistory.append(lossSum / miniBatchSize + reg_val) - /* - * NOTE(Xinghao): The loss here is sum of lossSum computed using the weights before applying updater, - * and reg_val using weights after applying updater - */ + regVal = update._2 } (weights.toArray, stochasticLossHistory.toArray) diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index 188fe7d972d..bbf21e5c284 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -76,7 +76,7 @@ class SquaredL2Updater extends Updater { val thisIterStepSize = stepSize / math.sqrt(iter) val normGradient = gradient.mul(thisIterStepSize) val newWeights = weightsOld.sub(normGradient).div(2.0 * thisIterStepSize * regParam + 1.0) - (newWeights, pow(newWeights.norm2,2.0) * regParam) + (newWeights, pow(newWeights.norm2, 2.0) * regParam) } } From 87b821dc3953bac7584592b9b030c2f57203dd0d Mon Sep 17 00:00:00 2001 From: Karen Feng Date: Mon, 29 Jul 2013 16:50:51 -0700 Subject: [PATCH 0519/2521] Fixed continuity of executorToTasksActive, changed color of progress bars --- core/src/main/scala/spark/ui/exec/ExecutorsUI.scala | 2 +- core/src/main/scala/spark/ui/jobs/IndexPage.scala | 7 +++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index b70153fd30d..d6f83157d3a 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -93,7 +93,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, Seq[Long]()).size.toString + val activeTasks = listener.executorToTasksActive.getOrElse(a.toString, HashSet[TaskInfo]()).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 diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index e0a41682904..e2cf84346a0 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -21,9 +21,11 @@ import java.util.Date import javax.servlet.http.HttpServletRequest +import scala.collection.mutable.HashSet import scala.Some import scala.xml.{NodeSeq, Node} +import spark.scheduler.cluster.TaskInfo import spark.scheduler.Stage import spark.storage.StorageLevel import spark.ui.Page._ @@ -110,8 +112,8 @@ private[spark] class IndexPage(parent: JobProgressUI) { val startWidth = "width: %s%%".format((started.toDouble/total)*100)
    -
    -
    +
    +
    } @@ -131,6 +133,7 @@ private[spark] class IndexPage(parent: JobProgressUI) { case b => Utils.memoryBytesToString(b) } + val startedTasks = listener.stageToTasksActive.getOrElse(s.id, HashSet[TaskInfo]()).size val completedTasks = listener.stageToTasksComplete.getOrElse(s.id, 0) val totalTasks = s.numPartitions From 17e62113d446c87d0313daa2ac230867591627be Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Jul 2013 17:14:44 -0700 Subject: [PATCH 0520/2521] Moved DeployMessage's into its own DeployMessages object. Also renamed MasterState to MasterStateResponse and WorkerState to WorkerStateResponse for clarity. --- .../scala/spark/deploy/DeployMessage.scala | 146 +++++++++--------- .../scala/spark/deploy/JsonProtocol.scala | 11 +- .../scala/spark/deploy/client/Client.scala | 18 ++- .../scala/spark/deploy/master/Master.scala | 39 +++-- .../deploy/master/ui/ApplicationPage.scala | 11 +- .../spark/deploy/master/ui/IndexPage.scala | 16 +- .../spark/deploy/worker/ExecutorRunner.scala | 10 +- .../scala/spark/deploy/worker/Worker.scala | 28 ++-- .../spark/deploy/worker/ui/IndexPage.scala | 23 +-- 9 files changed, 160 insertions(+), 142 deletions(-) diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala index e1f8aff6f51..7c37a166157 100644 --- a/core/src/main/scala/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/spark/deploy/DeployMessage.scala @@ -17,109 +17,107 @@ package spark.deploy +import scala.collection.immutable.List + +import spark.Utils import spark.deploy.ExecutorState.ExecutorState import spark.deploy.master.{WorkerInfo, ApplicationInfo} import spark.deploy.worker.ExecutorRunner -import scala.collection.immutable.List -import spark.Utils -private[spark] sealed trait DeployMessage extends Serializable +private[deploy] sealed trait DeployMessage extends Serializable -// Worker to Master +private[deploy] object DeployMessages { -private[spark] -case class RegisterWorker( - id: String, - host: String, - port: Int, - cores: Int, - memory: Int, - webUiPort: Int, - publicAddress: String) - extends DeployMessage { - Utils.checkHost(host, "Required hostname") - assert (port > 0) -} + // Worker to Master -private[spark] -case class ExecutorStateChanged( - appId: String, - execId: Int, - state: ExecutorState, - message: Option[String], - exitStatus: Option[Int]) - extends DeployMessage + case class RegisterWorker( + id: String, + host: String, + port: Int, + cores: Int, + memory: Int, + webUiPort: Int, + publicAddress: String) + extends DeployMessage { + Utils.checkHost(host, "Required hostname") + assert (port > 0) + } -private[spark] case class Heartbeat(workerId: String) extends DeployMessage + case class ExecutorStateChanged( + appId: String, + execId: Int, + state: ExecutorState, + message: Option[String], + exitStatus: Option[Int]) + extends DeployMessage -// Master to Worker + case class Heartbeat(workerId: String) extends DeployMessage -private[spark] case class RegisteredWorker(masterWebUiUrl: String) extends DeployMessage -private[spark] case class RegisterWorkerFailed(message: String) extends DeployMessage -private[spark] case class KillExecutor(appId: String, execId: Int) extends DeployMessage + // Master to Worker -private[spark] case class LaunchExecutor( - appId: String, - execId: Int, - appDesc: ApplicationDescription, - cores: Int, - memory: Int, - sparkHome: String) - extends DeployMessage + case class RegisteredWorker(masterWebUiUrl: String) extends DeployMessage -// Client to Master + case class RegisterWorkerFailed(message: String) extends DeployMessage -private[spark] case class RegisterApplication(appDescription: ApplicationDescription) - extends DeployMessage + case class KillExecutor(appId: String, execId: Int) extends DeployMessage -// Master to Client + case class LaunchExecutor( + appId: String, + execId: Int, + appDesc: ApplicationDescription, + cores: Int, + memory: Int, + sparkHome: String) + extends DeployMessage -private[spark] -case class RegisteredApplication(appId: String) extends DeployMessage + // Client to Master -private[spark] -case class ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) { - Utils.checkHostPort(hostPort, "Required hostport") -} + case class RegisterApplication(appDescription: ApplicationDescription) + extends DeployMessage -private[spark] -case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String], - exitStatus: Option[Int]) + // Master to Client -private[spark] -case class ApplicationRemoved(message: String) + case class RegisteredApplication(appId: String) extends DeployMessage -// Internal message in Client + case class ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) { + Utils.checkHostPort(hostPort, "Required hostport") + } -private[spark] case object StopClient + case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String], + exitStatus: Option[Int]) -// MasterWebUI To Master + case class ApplicationRemoved(message: String) -private[spark] case object RequestMasterState + // Internal message in Client -// Master to MasterWebUI + case object StopClient -private[spark] -case class MasterState(host: String, port: Int, workers: Array[WorkerInfo], - activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo]) { + // MasterWebUI To Master - Utils.checkHost(host, "Required hostname") - assert (port > 0) + case object RequestMasterState - def uri = "spark://" + host + ":" + port -} + // Master to MasterWebUI + + case class MasterStateResponse(host: String, port: Int, workers: Array[WorkerInfo], + activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo]) { + + Utils.checkHost(host, "Required hostname") + assert (port > 0) + + def uri = "spark://" + host + ":" + port + } -// WorkerWebUI to Worker -private[spark] case object RequestWorkerState + // WorkerWebUI to Worker + case object RequestWorkerState -// Worker to WorkerWebUI + // Worker to WorkerWebUI -private[spark] -case class WorkerState(host: String, port: Int, workerId: String, executors: List[ExecutorRunner], - finishedExecutors: List[ExecutorRunner], masterUrl: String, cores: Int, memory: Int, - coresUsed: Int, memoryUsed: Int, masterWebUiUrl: String) { + case class WorkerStateResponse(host: String, port: Int, workerId: String, + executors: List[ExecutorRunner], finishedExecutors: List[ExecutorRunner], masterUrl: String, + cores: Int, memory: Int, coresUsed: Int, memoryUsed: Int, masterWebUiUrl: String) { - Utils.checkHost(host, "Required hostname") - assert (port > 0) + Utils.checkHost(host, "Required hostname") + assert (port > 0) + } } diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index 64f89623e14..bd1db7c2940 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -17,9 +17,12 @@ package spark.deploy -import master.{ApplicationInfo, WorkerInfo} import net.liftweb.json.JsonDSL._ -import worker.ExecutorRunner + +import spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} +import spark.deploy.master.{ApplicationInfo, WorkerInfo} +import spark.deploy.worker.ExecutorRunner + private[spark] object JsonProtocol { def writeWorkerInfo(obj: WorkerInfo) = { @@ -57,7 +60,7 @@ private[spark] object JsonProtocol { ("appdesc" -> writeApplicationDescription(obj.appDesc)) } - def writeMasterState(obj: MasterState) = { + def writeMasterState(obj: MasterStateResponse) = { ("url" -> ("spark://" + obj.uri)) ~ ("workers" -> obj.workers.toList.map(writeWorkerInfo)) ~ ("cores" -> obj.workers.map(_.cores).sum) ~ @@ -68,7 +71,7 @@ private[spark] object JsonProtocol { ("completedapps" -> obj.completedApps.toList.map(writeApplicationInfo)) } - def writeWorkerState(obj: WorkerState) = { + def writeWorkerState(obj: WorkerStateResponse) = { ("id" -> obj.workerId) ~ ("masterurl" -> obj.masterUrl) ~ ("masterwebuiurl" -> obj.masterWebUiUrl) ~ diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala index 29e494f4959..9d5ba8a7960 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/spark/deploy/client/Client.scala @@ -17,21 +17,23 @@ package spark.deploy.client -import spark.deploy._ +import java.util.concurrent.TimeoutException + import akka.actor._ +import akka.actor.Terminated import akka.pattern.ask import akka.util.Duration -import akka.util.duration._ -import java.util.concurrent.TimeoutException -import spark.{SparkException, Logging} +import akka.remote.RemoteClientDisconnected import akka.remote.RemoteClientLifeCycleEvent import akka.remote.RemoteClientShutdown -import spark.deploy.RegisterApplication -import spark.deploy.master.Master -import akka.remote.RemoteClientDisconnected -import akka.actor.Terminated import akka.dispatch.Await +import spark.Logging +import spark.deploy.{ApplicationDescription, ExecutorState} +import spark.deploy.DeployMessages._ +import spark.deploy.master.Master + + /** * The main class used to talk to a Spark deploy cluster. Takes a master URL, an app description, * and a listener for cluster events, and calls back the listener when various events occur. diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 9692af52953..202d5bcdb7f 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -17,21 +17,22 @@ package spark.deploy.master -import akka.actor._ -import akka.actor.Terminated -import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown} -import akka.util.duration._ - import java.text.SimpleDateFormat import java.util.Date import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import spark.deploy._ +import akka.actor._ +import akka.actor.Terminated +import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown} +import akka.util.duration._ + import spark.{Logging, SparkException, Utils} +import spark.deploy.{ApplicationDescription, ExecutorState} +import spark.deploy.DeployMessages._ +import spark.deploy.master.ui.MasterWebUI 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 { @@ -168,7 +169,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } case RequestMasterState => { - sender ! MasterState(host, port, workers.toArray, apps.toArray, completedApps.toArray) + sender ! MasterStateResponse(host, port, workers.toArray, apps.toArray, completedApps.toArray) } } @@ -233,20 +234,27 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act def launchExecutor(worker: WorkerInfo, exec: ExecutorInfo, sparkHome: String) { logInfo("Launching executor " + exec.fullId + " on worker " + worker.id) worker.addExecutor(exec) - worker.actor ! LaunchExecutor(exec.application.id, exec.id, exec.application.desc, exec.cores, exec.memory, sparkHome) - exec.application.driver ! ExecutorAdded(exec.id, worker.id, worker.hostPort, exec.cores, exec.memory) + worker.actor ! LaunchExecutor( + exec.application.id, exec.id, exec.application.desc, exec.cores, exec.memory, sparkHome) + exec.application.driver ! ExecutorAdded( + exec.id, worker.id, worker.hostPort, exec.cores, exec.memory) } def addWorker(id: String, host: String, port: Int, cores: Int, memory: Int, webUiPort: Int, publicAddress: String): WorkerInfo = { - // There may be one or more refs to dead workers on this same node (w/ different ID's), remove them. - workers.filter(w => (w.host == host && w.port == port) && (w.state == WorkerState.DEAD)).foreach(workers -= _) + // There may be one or more refs to dead workers on this same node (w/ different ID's), + // remove them. + workers.filter { w => + (w.host == host && w.port == port) && (w.state == WorkerState.DEAD) + }.foreach { w => + workers -= w + } val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort, publicAddress) workers += worker idToWorker(worker.id) = worker actorToWorker(sender) = worker addressToWorker(sender.path.address) = worker - return worker + worker } def removeWorker(worker: WorkerInfo) { @@ -257,7 +265,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act addressToWorker -= worker.actor.path.address for (exec <- worker.executors.values) { logInfo("Telling app of lost executor: " + exec.id) - exec.application.driver ! ExecutorUpdated(exec.id, ExecutorState.LOST, Some("worker lost"), None) + exec.application.driver ! ExecutorUpdated( + exec.id, ExecutorState.LOST, Some("worker lost"), None) exec.application.removeExecutor(exec) } } @@ -277,7 +286,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act if (workersAlive.size > 0 && !workersAlive.exists(_.memoryFree >= desc.memoryPerSlave)) { logWarning("Could not find any workers with enough memory for " + firstApp.get.id) } - return app + app } def finishApplication(app: ApplicationInfo) { 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 32264af3934..b4c62bc224c 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -17,6 +17,8 @@ package spark.deploy.master.ui +import scala.xml.Node + import akka.dispatch.Await import akka.pattern.ask import akka.util.duration._ @@ -25,9 +27,8 @@ import javax.servlet.http.HttpServletRequest import net.liftweb.json.JsonAST.JValue -import scala.xml.Node - -import spark.deploy.{RequestMasterState, JsonProtocol, MasterState} +import spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} +import spark.deploy.JsonProtocol import spark.deploy.master.ExecutorInfo import spark.ui.UIUtils @@ -38,7 +39,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { /** Executor details for a particular application */ def renderJson(request: HttpServletRequest): JValue = { val appId = request.getParameter("appId") - val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, 30 seconds) val app = state.activeApps.find(_.id == appId).getOrElse({ state.completedApps.find(_.id == appId).getOrElse(null) @@ -49,7 +50,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { /** 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 stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, 30 seconds) val app = state.activeApps.find(_.id == appId).getOrElse({ state.completedApps.find(_.id == appId).getOrElse(null) 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 b05197c1b97..557df89b41f 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -17,18 +17,20 @@ 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 akka.dispatch.Await +import akka.pattern.ask +import akka.util.duration._ + import spark.Utils -import spark.ui.UIUtils +import spark.deploy.DeployWebUI +import spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import spark.deploy.master.{ApplicationInfo, WorkerInfo} +import spark.ui.UIUtils + private[spark] class IndexPage(parent: MasterWebUI) { val master = parent.master @@ -36,7 +38,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { /** Index view listing applications and executors */ def render(request: HttpServletRequest): Seq[Node] = { - val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterState] + val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, 30 seconds) val workerHeaders = Seq("Id", "Address", "State", "Cores", "Memory") diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 47d3390928b..345dfe879cf 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -19,14 +19,12 @@ 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} -import java.net.{URI, URL} -import org.apache.hadoop.fs.{Path, FileSystem} -import org.apache.hadoop.conf.Configuration -import scala.Some -import spark.deploy.ExecutorStateChanged +import spark.deploy.{ExecutorState, ApplicationDescription} +import spark.deploy.DeployMessages.ExecutorStateChanged /** * Manages the execution of one executor process. diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 8fa0d12b828..0e46fa281ec 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -17,22 +17,24 @@ package spark.deploy.worker -import scala.collection.mutable.{ArrayBuffer, HashMap} +import java.text.SimpleDateFormat +import java.util.Date +import java.io.File + +import scala.collection.mutable.HashMap + import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} +import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} 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 -import spark.deploy.RegisterWorker -import spark.deploy.LaunchExecutor -import spark.deploy.RegisterWorkerFailed +import spark.deploy.ExecutorState +import spark.deploy.DeployMessages._ import spark.deploy.master.Master -import java.io.File -import ui.WorkerWebUI +import spark.deploy.worker.ui.WorkerWebUI +import spark.metrics.MetricsSystem +import spark.util.AkkaUtils + private[spark] class Worker( host: String, @@ -164,7 +166,7 @@ private[spark] class Worker( masterDisconnected() case RequestWorkerState => { - sender ! WorkerState(host, port, workerId, executors.values.toList, + sender ! WorkerStateResponse(host, port, workerId, executors.values.toList, finishedExecutors.values.toList, masterUrl, cores, memory, coresUsed, memoryUsed, masterWebUiUrl) } 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 7548a26c2ef..1619c6a4c25 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -17,34 +17,36 @@ package spark.deploy.worker.ui +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + 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.{RequestWorkerState, JsonProtocol, WorkerState} -import spark.deploy.worker.ExecutorRunner import spark.Utils +import spark.deploy.JsonProtocol +import spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} +import spark.deploy.worker.ExecutorRunner 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 = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerState] + val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] val workerState = Await.result(stateFuture, 30 seconds) JsonProtocol.writeWorkerState(workerState) } def render(request: HttpServletRequest): Seq[Node] = { - val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerState] + val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] val workerState = Await.result(stateFuture, 30 seconds) val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") @@ -69,7 +71,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
    -
    +
    @@ -88,7 +90,8 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
    ; - UIUtils.basicSparkPage(content, "Spark Worker on %s:%s".format(workerState.host, workerState.port)) + UIUtils.basicSparkPage(content, "Spark Worker on %s:%s".format( + workerState.host, workerState.port)) } def executorRow(executor: ExecutorRunner): Seq[Node] = { From 207548b67bc6ce208b60215d40a03904d8ac0cbc Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 29 Jul 2013 17:19:33 -0700 Subject: [PATCH 0521/2521] Open up Job UI ports (33000-33010) on EC2 clusters --- ec2/spark_ec2.py | 39 ++++++++++++++++++++------------------- 1 file changed, 20 insertions(+), 19 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 2ec3c007fba..740ec08542f 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -9,9 +9,9 @@ # to you under the Apache License, Version 2.0 (the # "License"); you may not use this file except in compliance # with the License. You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, software # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -53,7 +53,7 @@ def parse_args(): help="Seconds to wait for nodes to start (default: 120)") parser.add_option("-k", "--key-pair", help="Key pair to use on instances") - parser.add_option("-i", "--identity-file", + parser.add_option("-i", "--identity-file", help="SSH private key file to use for logging into instances") parser.add_option("-t", "--instance-type", default="m1.large", help="Type of instance to launch (default: m1.large). " + @@ -69,7 +69,7 @@ def parse_args(): parser.add_option("-a", "--ami", default="latest", help="Amazon Machine Image ID to use, or 'latest' to use latest " + "available AMI (default: latest)") - parser.add_option("-D", metavar="[ADDRESS:]PORT", dest="proxy_port", + parser.add_option("-D", metavar="[ADDRESS:]PORT", dest="proxy_port", help="Use SSH dynamic port forwarding to create a SOCKS proxy at " + "the given local address (for use with login)") parser.add_option("--resume", action="store_true", default=False, @@ -99,7 +99,7 @@ def parse_args(): help="The SSH user you want to connect as (default: root)") parser.add_option("--delete-groups", action="store_true", default=False, help="When destroying a cluster, delete the security groups that were created") - + (opts, args) = parser.parse_args() if len(args) != 2: parser.print_help() @@ -112,7 +112,7 @@ def parse_args(): if opts.cluster_type not in ["mesos", "standalone"] and action == "launch": print >> stderr, ("ERROR: Invalid cluster type: " + opts.cluster_type) sys.exit(1) - + # Boto config check # http://boto.cloudhackers.com/en/latest/boto_config_tut.html home_dir = os.getenv('HOME') @@ -178,6 +178,7 @@ def launch_cluster(conn, opts, cluster_name): master_group.authorize('tcp', 50030, 50030, '0.0.0.0/0') master_group.authorize('tcp', 50070, 50070, '0.0.0.0/0') master_group.authorize('tcp', 60070, 60070, '0.0.0.0/0') + master_group.authorize('tcp', 33000, 33010, '0.0.0.0/0') if opts.cluster_type == "mesos": master_group.authorize('tcp', 38090, 38090, '0.0.0.0/0') if opts.ganglia: @@ -257,7 +258,7 @@ def launch_cluster(conn, opts, cluster_name): block_device_map = block_map) my_req_ids += [req.id for req in slave_reqs] i += 1 - + print "Waiting for spot instances to be granted..." try: while True: @@ -413,7 +414,7 @@ def setup_standalone_cluster(master, slave_nodes, opts): slave_ips = '\n'.join([i.public_dns_name for i in slave_nodes]) ssh(master, opts, "echo \"%s\" > spark/conf/slaves" % (slave_ips)) ssh(master, opts, "/root/spark/bin/start-all.sh") - + def setup_spark_cluster(master, opts): ssh(master, opts, "chmod u+x spark-ec2/setup.sh") ssh(master, opts, "spark-ec2/setup.sh") @@ -528,7 +529,7 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, zoo_nodes, dest.write(text) dest.close() # rsync the whole directory over to the master machine - command = (("rsync -rv -e 'ssh -o StrictHostKeyChecking=no -i %s' " + + command = (("rsync -rv -e 'ssh -o StrictHostKeyChecking=no -i %s' " + "'%s/' '%s@%s:/'") % (opts.identity_file, tmp_dir, opts.user, active_master)) subprocess.check_call(command, shell=True) # Remove the temp directory we created above @@ -557,9 +558,9 @@ def ssh(host, opts, command): print "Error connecting to host {0}, sleeping 30".format(e) time.sleep(30) tries = tries + 1 - - - + + + # Gets a list of zones to launch instances in @@ -618,12 +619,12 @@ def main(): print "Terminating zoo..." for inst in zoo_nodes: inst.terminate() - + # Delete security groups as well if opts.delete_groups: print "Deleting security groups (this will take some time)..." group_names = [cluster_name + "-master", cluster_name + "-slaves", cluster_name + "-zoo"] - + attempt = 1; while attempt <= 3: print "Attempt %d" % attempt @@ -639,7 +640,7 @@ def main(): from_port=rule.from_port, to_port=rule.to_port, src_group=grant) - + # Sleep for AWS eventual-consistency to catch up, and for instances # to terminate time.sleep(30) # Yes, it does have to be this long :-( @@ -650,13 +651,13 @@ def main(): except boto.exception.EC2ResponseError: success = False; print "Failed to delete security group " + group.name - + # Unfortunately, group.revoke() returns True even if a rule was not # deleted, so this needs to be rerun if something fails if success: break; - + attempt += 1 - + if not success: print "Failed to delete all security groups after 3 tries." print "Try re-running in a few minutes." @@ -679,7 +680,7 @@ def main(): elif action == "stop": response = raw_input("Are you sure you want to stop the cluster " + cluster_name + "?\nDATA ON EPHEMERAL DISKS WILL BE LOST, " + - "BUT THE CLUSTER WILL KEEP USING SPACE ON\n" + + "BUT THE CLUSTER WILL KEEP USING SPACE ON\n" + "AMAZON EBS IF IT IS EBS-BACKED!!\n" + "Stop cluster " + cluster_name + " (y/N): ") if response == "y": From 105f4d22e93af23ff3bf117bcfd976d412f494e6 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Jul 2013 17:30:38 -0700 Subject: [PATCH 0522/2521] Removed Cache and SoftReferenceCache since they are no longer used. --- core/src/main/scala/spark/Cache.scala | 80 ------------------- .../main/scala/spark/SoftReferenceCache.scala | 35 -------- 2 files changed, 115 deletions(-) delete mode 100644 core/src/main/scala/spark/Cache.scala delete mode 100644 core/src/main/scala/spark/SoftReferenceCache.scala diff --git a/core/src/main/scala/spark/Cache.scala b/core/src/main/scala/spark/Cache.scala deleted file mode 100644 index b0c83ce59d1..00000000000 --- a/core/src/main/scala/spark/Cache.scala +++ /dev/null @@ -1,80 +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 - -import java.util.concurrent.atomic.AtomicInteger - -private[spark] sealed trait CachePutResponse -private[spark] case class CachePutSuccess(size: Long) extends CachePutResponse -private[spark] case class CachePutFailure() extends CachePutResponse - -/** - * An interface for caches in Spark, to allow for multiple implementations. Caches are used to store - * both partitions of cached RDDs and broadcast variables on Spark executors. Caches are also aware - * of which entries are part of the same dataset (for example, partitions in the same RDD). The key - * for each value in a cache is a (datasetID, partition) pair. - * - * A single Cache instance gets created on each machine and is shared by all caches (i.e. both the - * RDD split cache and the broadcast variable cache), to enable global replacement policies. - * However, because these several independent modules all perform caching, it is important to give - * them separate key namespaces, so that an RDD and a broadcast variable (for example) do not use - * the same key. For this purpose, Cache has the notion of KeySpaces. Each client module must first - * ask for a KeySpace, and then call get() and put() on that space using its own keys. - * - * This abstract class handles the creation of key spaces, so that subclasses need only deal with - * keys that are unique across modules. - */ -private[spark] abstract class Cache { - private val nextKeySpaceId = new AtomicInteger(0) - private def newKeySpaceId() = nextKeySpaceId.getAndIncrement() - - def newKeySpace() = new KeySpace(this, newKeySpaceId()) - - /** - * Get the value for a given (datasetId, partition), or null if it is not - * found. - */ - def get(datasetId: Any, partition: Int): Any - - /** - * Attempt to put a value in the cache; returns CachePutFailure if this was - * not successful (e.g. because the cache replacement policy forbids it), and - * CachePutSuccess if successful. If size estimation is available, the cache - * implementation should set the size field in CachePutSuccess. - */ - def put(datasetId: Any, partition: Int, value: Any): CachePutResponse - - /** - * Report the capacity of the cache partition. By default this just reports - * zero. Specific implementations can choose to provide the capacity number. - */ - def getCapacity: Long = 0L -} - -/** - * A key namespace in a Cache. - */ -private[spark] class KeySpace(cache: Cache, val keySpaceId: Int) { - def get(datasetId: Any, partition: Int): Any = - cache.get((keySpaceId, datasetId), partition) - - def put(datasetId: Any, partition: Int, value: Any): CachePutResponse = - cache.put((keySpaceId, datasetId), partition, value) - - def getCapacity: Long = cache.getCapacity -} diff --git a/core/src/main/scala/spark/SoftReferenceCache.scala b/core/src/main/scala/spark/SoftReferenceCache.scala deleted file mode 100644 index f41a379582a..00000000000 --- a/core/src/main/scala/spark/SoftReferenceCache.scala +++ /dev/null @@ -1,35 +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 - -import com.google.common.collect.MapMaker - -/** - * An implementation of Cache that uses soft references. - */ -private[spark] class SoftReferenceCache extends Cache { - val map = new MapMaker().softValues().makeMap[Any, Any]() - - override def get(datasetId: Any, partition: Int): Any = - map.get((datasetId, partition)) - - override def put(datasetId: Any, partition: Int, value: Any): CachePutResponse = { - map.put((datasetId, partition), value) - return CachePutSuccess(0) - } -} From 23b5da14ed6413b0dcb4c0bfdb80c98c433f3c9d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Jul 2013 17:42:05 -0700 Subject: [PATCH 0523/2521] Moved block manager messages into BlockManagerMessages object. --- .../spark/storage/BlockManagerMaster.scala | 1 + .../storage/BlockManagerMasterActor.scala | 2 + .../spark/storage/BlockManagerMessages.scala | 163 ++++++++---------- .../storage/BlockManagerSlaveActor.scala | 2 +- .../spark/storage/BlockManagerSource.scala | 2 +- .../scala/spark/storage/BlockMessage.scala | 1 - .../spark/storage/BlockMessageArray.scala | 5 +- .../spark/storage/BlockObjectWriter.scala | 2 - 8 files changed, 82 insertions(+), 96 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala index 3186f7c85b1..76128e8cfff 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala @@ -23,6 +23,7 @@ import akka.pattern.ask import akka.util.Duration import spark.{Logging, SparkException} +import spark.storage.BlockManagerMessages._ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging { diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index 244000d9527..011bb6b83d7 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -29,6 +29,8 @@ import akka.util.Duration import akka.util.duration._ import spark.{Logging, Utils, SparkException} +import spark.storage.BlockManagerMessages._ + /** * BlockManagerMasterActor is an actor on the master node to track statuses of diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/spark/storage/BlockManagerMessages.scala index 01de4ccb8f4..9375a9ca548 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMessages.scala @@ -22,102 +22,89 @@ import java.io.{Externalizable, ObjectInput, ObjectOutput} import akka.actor.ActorRef -////////////////////////////////////////////////////////////////////////////////// -// Messages from the master to slaves. -////////////////////////////////////////////////////////////////////////////////// -private[spark] -sealed trait ToBlockManagerSlave - -// Remove a block from the slaves that have it. This can only be used to remove -// blocks that the master knows about. -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 - - -////////////////////////////////////////////////////////////////////////////////// -// Messages from slaves to the master. -////////////////////////////////////////////////////////////////////////////////// -private[spark] -sealed trait ToBlockManagerMaster - -private[spark] -case class RegisterBlockManager( - blockManagerId: BlockManagerId, - maxMemSize: Long, - sender: ActorRef) - extends ToBlockManagerMaster - -private[spark] -case class HeartBeat(blockManagerId: BlockManagerId) extends ToBlockManagerMaster - -private[spark] -class UpdateBlockInfo( - var blockManagerId: BlockManagerId, - var blockId: String, - var storageLevel: StorageLevel, - var memSize: Long, - var diskSize: Long) - extends ToBlockManagerMaster - with Externalizable { - - def this() = this(null, null, null, 0, 0) // For deserialization only - - override def writeExternal(out: ObjectOutput) { - blockManagerId.writeExternal(out) - out.writeUTF(blockId) - storageLevel.writeExternal(out) - out.writeLong(memSize) - out.writeLong(diskSize) +private[storage] object BlockManagerMessages { + ////////////////////////////////////////////////////////////////////////////////// + // Messages from the master to slaves. + ////////////////////////////////////////////////////////////////////////////////// + sealed trait ToBlockManagerSlave + + // Remove a block from the slaves that have it. This can only be used to remove + // blocks that the master knows about. + case class RemoveBlock(blockId: String) extends ToBlockManagerSlave + + // Remove all blocks belonging to a specific RDD. + case class RemoveRdd(rddId: Int) extends ToBlockManagerSlave + + + ////////////////////////////////////////////////////////////////////////////////// + // Messages from slaves to the master. + ////////////////////////////////////////////////////////////////////////////////// + sealed trait ToBlockManagerMaster + + case class RegisterBlockManager( + blockManagerId: BlockManagerId, + maxMemSize: Long, + sender: ActorRef) + extends ToBlockManagerMaster + + case class HeartBeat(blockManagerId: BlockManagerId) extends ToBlockManagerMaster + + class UpdateBlockInfo( + var blockManagerId: BlockManagerId, + var blockId: String, + var storageLevel: StorageLevel, + var memSize: Long, + var diskSize: Long) + extends ToBlockManagerMaster + with Externalizable { + + def this() = this(null, null, null, 0, 0) // For deserialization only + + override def writeExternal(out: ObjectOutput) { + blockManagerId.writeExternal(out) + out.writeUTF(blockId) + storageLevel.writeExternal(out) + out.writeLong(memSize) + out.writeLong(diskSize) + } + + override def readExternal(in: ObjectInput) { + blockManagerId = BlockManagerId(in) + blockId = in.readUTF() + storageLevel = StorageLevel(in) + memSize = in.readLong() + diskSize = in.readLong() + } } - override def readExternal(in: ObjectInput) { - blockManagerId = BlockManagerId(in) - blockId = in.readUTF() - storageLevel = StorageLevel(in) - memSize = in.readLong() - diskSize = in.readLong() + object UpdateBlockInfo { + def apply(blockManagerId: BlockManagerId, + blockId: String, + storageLevel: StorageLevel, + memSize: Long, + diskSize: Long): UpdateBlockInfo = { + new UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize) + } + + // For pattern-matching + def unapply(h: UpdateBlockInfo): Option[(BlockManagerId, String, StorageLevel, Long, Long)] = { + Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize)) + } } -} -private[spark] -object UpdateBlockInfo { - def apply(blockManagerId: BlockManagerId, - blockId: String, - storageLevel: StorageLevel, - memSize: Long, - diskSize: Long): UpdateBlockInfo = { - new UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize) - } + case class GetLocations(blockId: String) extends ToBlockManagerMaster - // For pattern-matching - def unapply(h: UpdateBlockInfo): Option[(BlockManagerId, String, StorageLevel, Long, Long)] = { - Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize)) - } -} + case class GetLocationsMultipleBlockIds(blockIds: Array[String]) extends ToBlockManagerMaster -private[spark] -case class GetLocations(blockId: String) extends ToBlockManagerMaster + case class GetPeers(blockManagerId: BlockManagerId, size: Int) extends ToBlockManagerMaster -private[spark] -case class GetLocationsMultipleBlockIds(blockIds: Array[String]) extends ToBlockManagerMaster + case class RemoveExecutor(execId: String) extends ToBlockManagerMaster -private[spark] -case class GetPeers(blockManagerId: BlockManagerId, size: Int) extends ToBlockManagerMaster + case object StopBlockManagerMaster extends ToBlockManagerMaster -private[spark] -case class RemoveExecutor(execId: String) extends ToBlockManagerMaster + case object GetMemoryStatus extends ToBlockManagerMaster -private[spark] -case object StopBlockManagerMaster extends ToBlockManagerMaster + case object ExpireDeadHosts extends ToBlockManagerMaster -private[spark] -case object GetMemoryStatus extends ToBlockManagerMaster - -private[spark] -case object ExpireDeadHosts extends ToBlockManagerMaster - -private[spark] -case object GetStorageStatus extends ToBlockManagerMaster + case object GetStorageStatus extends ToBlockManagerMaster +} diff --git a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala index 45cffad810e..6e5fb437329 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala @@ -19,7 +19,7 @@ package spark.storage import akka.actor.Actor -import spark.{Logging, SparkException, Utils} +import spark.storage.BlockManagerMessages._ /** diff --git a/core/src/main/scala/spark/storage/BlockManagerSource.scala b/core/src/main/scala/spark/storage/BlockManagerSource.scala index 4faa715c947..2aecd1ea716 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/spark/storage/BlockManagerSource.scala @@ -3,7 +3,7 @@ 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() diff --git a/core/src/main/scala/spark/storage/BlockMessage.scala b/core/src/main/scala/spark/storage/BlockMessage.scala index ab72dbb62ba..bcce26b7c14 100644 --- a/core/src/main/scala/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/spark/storage/BlockMessage.scala @@ -22,7 +22,6 @@ import java.nio.ByteBuffer import scala.collection.mutable.StringBuilder import scala.collection.mutable.ArrayBuffer -import spark._ import spark.network._ private[spark] case class GetBlock(id: String) diff --git a/core/src/main/scala/spark/storage/BlockMessageArray.scala b/core/src/main/scala/spark/storage/BlockMessageArray.scala index b0229d6124d..ee2fc167d5b 100644 --- a/core/src/main/scala/spark/storage/BlockMessageArray.scala +++ b/core/src/main/scala/spark/storage/BlockMessageArray.scala @@ -19,7 +19,6 @@ package spark.storage import java.nio.ByteBuffer -import scala.collection.mutable.StringBuilder import scala.collection.mutable.ArrayBuffer import spark._ @@ -113,7 +112,7 @@ private[spark] object BlockMessageArray { def main(args: Array[String]) { val blockMessages = - (0 until 10).map(i => { + (0 until 10).map { i => if (i % 2 == 0) { val buffer = ByteBuffer.allocate(100) buffer.clear @@ -121,7 +120,7 @@ private[spark] object BlockMessageArray { } else { BlockMessage.fromGetBlock(GetBlock(i.toString)) } - }) + } val blockMessageArray = new BlockMessageArray(blockMessages) println("Block message array created") diff --git a/core/src/main/scala/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/spark/storage/BlockObjectWriter.scala index 01ed6e8c1fa..3812009ca16 100644 --- a/core/src/main/scala/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/spark/storage/BlockObjectWriter.scala @@ -17,8 +17,6 @@ package spark.storage -import java.nio.ByteBuffer - /** * An interface for writing JVM objects to some underlying storage. This interface allows From 81720e13fc9e1f475dd1333babfa08f3f806a5d0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Jul 2013 17:53:01 -0700 Subject: [PATCH 0524/2521] Moved all StandaloneClusterMessage's into StandaloneClusterMessages object. --- .../executor/StandaloneExecutorBackend.scala | 17 +++--- .../cluster/StandaloneClusterMessage.scala | 61 ++++++++++--------- .../cluster/StandaloneSchedulerBackend.scala | 9 +-- 3 files changed, 43 insertions(+), 44 deletions(-) diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index f4003da7329..e47fe500216 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -18,19 +18,16 @@ package spark.executor import java.nio.ByteBuffer -import spark.Logging -import spark.TaskState.TaskState -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.scheduler.cluster._ -import spark.scheduler.cluster.RegisteredExecutor -import spark.scheduler.cluster.LaunchTask -import spark.scheduler.cluster.RegisterExecutorFailed -import spark.scheduler.cluster.RegisterExecutor -import spark.Utils + +import spark.{Logging, Utils} +import spark.TaskState.TaskState import spark.deploy.SparkHadoopUtil +import spark.scheduler.cluster.StandaloneClusterMessages._ +import spark.util.AkkaUtils + private[spark] class StandaloneExecutorBackend( driverUrl: String, diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala index ac9e5ef94d3..05c29eb72f6 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala @@ -17,46 +17,47 @@ package spark.scheduler.cluster -import spark.TaskState.TaskState import java.nio.ByteBuffer -import spark.util.SerializableBuffer + +import spark.TaskState.TaskState import spark.Utils +import spark.util.SerializableBuffer + private[spark] sealed trait StandaloneClusterMessage extends Serializable -// Driver to executors -private[spark] -case class LaunchTask(task: TaskDescription) extends StandaloneClusterMessage +private[spark] object StandaloneClusterMessages { -private[spark] -case class RegisteredExecutor(sparkProperties: Seq[(String, String)]) - extends StandaloneClusterMessage + // Driver to executors + case class LaunchTask(task: TaskDescription) extends StandaloneClusterMessage -private[spark] -case class RegisterExecutorFailed(message: String) extends StandaloneClusterMessage + case class RegisteredExecutor(sparkProperties: Seq[(String, String)]) + extends StandaloneClusterMessage -// Executors to driver -private[spark] -case class RegisterExecutor(executorId: String, hostPort: String, cores: Int) - extends StandaloneClusterMessage { - Utils.checkHostPort(hostPort, "Expected host port") -} + case class RegisterExecutorFailed(message: String) extends StandaloneClusterMessage -private[spark] -case class StatusUpdate(executorId: String, taskId: Long, state: TaskState, data: SerializableBuffer) - extends StandaloneClusterMessage + // Executors to driver + case class RegisterExecutor(executorId: String, hostPort: String, cores: Int) + extends StandaloneClusterMessage { + Utils.checkHostPort(hostPort, "Expected host port") + } + + case class StatusUpdate(executorId: String, taskId: Long, state: TaskState, + data: SerializableBuffer) extends StandaloneClusterMessage -private[spark] -object StatusUpdate { - /** Alternate factory method that takes a ByteBuffer directly for the data field */ - def apply(executorId: String, taskId: Long, state: TaskState, data: ByteBuffer): StatusUpdate = { - StatusUpdate(executorId, taskId, state, new SerializableBuffer(data)) + object StatusUpdate { + /** Alternate factory method that takes a ByteBuffer directly for the data field */ + def apply(executorId: String, taskId: Long, state: TaskState, data: ByteBuffer) + : StatusUpdate = { + StatusUpdate(executorId, taskId, state, new SerializableBuffer(data)) + } } -} -// Internal messages in driver -private[spark] case object ReviveOffers extends StandaloneClusterMessage -private[spark] case object StopDriver extends StandaloneClusterMessage + // Internal messages in driver + case object ReviveOffers extends StandaloneClusterMessage -private[spark] case class RemoveExecutor(executorId: String, reason: String) - extends StandaloneClusterMessage + case object StopDriver extends StandaloneClusterMessage + + case class RemoveExecutor(executorId: String, reason: String) extends StandaloneClusterMessage + +} diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 03a64e01921..075a7cbf7e2 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -17,17 +17,18 @@ package spark.scheduler.cluster +import java.util.concurrent.atomic.AtomicInteger + import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import akka.actor._ -import akka.util.duration._ +import akka.dispatch.Await import akka.pattern.ask +import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent} import akka.util.Duration import spark.{Utils, SparkException, Logging, TaskState} -import akka.dispatch.Await -import java.util.concurrent.atomic.AtomicInteger -import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent} +import spark.scheduler.cluster.StandaloneClusterMessages._ /** * A standalone scheduler backend, which waits for standalone executors to connect to it through From 3ca9faa341dcddb54f8b2e26b582c08901ea875f Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 29 Jul 2013 18:37:28 -0700 Subject: [PATCH 0525/2521] Clarify how regVal is computed in Updater docs --- .../spark/mllib/optimization/Updater.scala | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala index bbf21e5c284..e916a92c332 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Updater.scala @@ -23,6 +23,7 @@ import org.jblas.DoubleMatrix abstract class Updater extends Serializable { /** * Compute an updated value for weights given the gradient, stepSize and iteration number. + * Also returns the regularization value computed using the *updated* weights. * * @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. @@ -31,7 +32,7 @@ abstract class Updater extends Serializable { * @param regParam - Regularization parameter * * @return A tuple of 2 elements. The first element is a column matrix containing updated weights, - * and the second element is the regularization value. + * and the second element is the regularization value computed using updated weights. */ def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) @@ -46,13 +47,13 @@ class SimpleUpdater extends Updater { } /** -* L1 regularization -- corresponding proximal operator is the soft-thresholding function -* That is, each weight component is shrunk towards 0 by shrinkageVal -* If w > shrinkageVal, set weight component to w-shrinkageVal. -* If w < -shrinkageVal, set weight component to w+shrinkageVal. -* If -shrinkageVal < w < shrinkageVal, set weight component to 0. -* Equivalently, set weight component to signum(w) * max(0.0, abs(w) - shrinkageVal) -**/ + * L1 regularization -- corresponding proximal operator is the soft-thresholding function + * That is, each weight component is shrunk towards 0 by shrinkageVal + * If w > shrinkageVal, set weight component to w-shrinkageVal. + * If w < -shrinkageVal, set weight component to w+shrinkageVal. + * If -shrinkageVal < w < shrinkageVal, set weight component to 0. + * Equivalently, set weight component to signum(w) * max(0.0, abs(w) - shrinkageVal) + */ class L1Updater extends Updater { override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { From 8e5cd041bbf7f802794b8f6e960f702fb59e5863 Mon Sep 17 00:00:00 2001 From: Dmitriy Lyubimov Date: Mon, 29 Jul 2013 18:25:33 -0700 Subject: [PATCH 0526/2521] initial externalization of ParallelCollectionRDD's split --- .../spark/rdd/ParallelCollectionRDD.scala | 70 +++++++++++++++---- 1 file changed, 55 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala index 16ba0c26f8e..104257ac071 100644 --- a/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala @@ -20,13 +20,15 @@ package spark.rdd import scala.collection.immutable.NumericRange import scala.collection.mutable.ArrayBuffer import scala.collection.Map -import spark.{RDD, TaskContext, SparkContext, Partition} +import spark._ +import java.io.{ObjectInput, ObjectOutput, Externalizable} +import java.nio.ByteBuffer private[spark] class ParallelCollectionPartition[T: ClassManifest]( - val rddId: Long, - val slice: Int, - values: Seq[T]) - extends Partition with Serializable { + var rddId: Long, + var slice: Int, + var values: Seq[T]) + extends Partition with Externalizable { def iterator: Iterator[T] = values.iterator @@ -37,14 +39,51 @@ private[spark] class ParallelCollectionPartition[T: ClassManifest]( case _ => false } - override val index: Int = slice + override def index: Int = slice + + override def writeExternal(out: ObjectOutput) { + out.writeLong(rddId) + out.writeInt(slice) + out.writeInt(values.size) + val ser = SparkEnv.get.serializer.newInstance() + values.foreach(x => { + val bb = ser.serialize(x) + out.writeInt(bb.remaining()) + if (bb.hasArray) { + out.write(bb.array(), bb.arrayOffset() + bb.position(), bb.remaining()) + } else { + val b = new Array[Byte](bb.remaining()) + bb.get(b) + out.write(b) + } + }) + } + + override def readExternal(in: ObjectInput) { + rddId = in.readLong() + slice = in.readInt() + val s = in.readInt() + val ser = SparkEnv.get.serializer.newInstance() + var bb = ByteBuffer.allocate(1024) + values = (0 until s).map({ + val s = in.readInt() + if (bb.capacity() < s) { + bb = ByteBuffer.allocate(s) + } else { + bb.clear() + } + in.readFully(bb.array()) + bb.limit(s) + ser.deserialize(bb) + }).toSeq + } } private[spark] class ParallelCollectionRDD[T: ClassManifest]( - @transient sc: SparkContext, - @transient data: Seq[T], - numSlices: Int, - locationPrefs: Map[Int,Seq[String]]) + @transient sc: SparkContext, + @transient data: Seq[T], + numSlices: Int, + locationPrefs: Map[Int, Seq[String]]) extends RDD[T](sc, Nil) { // TODO: Right now, each split sends along its full data, even if later down the RDD chain it gets // cached. It might be worthwhile to write the data to a file in the DFS and read it in the split @@ -82,16 +121,17 @@ private object ParallelCollectionRDD { 1 } slice(new Range( - r.start, r.end + sign, r.step).asInstanceOf[Seq[T]], numSlices) + r.start, r.end + sign, r.step).asInstanceOf[Seq[T]], numSlices) } case r: Range => { (0 until numSlices).map(i => { val start = ((i * r.length.toLong) / numSlices).toInt - val end = (((i+1) * r.length.toLong) / numSlices).toInt + val end = (((i + 1) * r.length.toLong) / numSlices).toInt new Range(r.start + start * r.step, r.start + end * r.step, r.step) }).asInstanceOf[Seq[Seq[T]]] } - case nr: NumericRange[_] => { // For ranges of Long, Double, BigInteger, etc + case nr: NumericRange[_] => { + // For ranges of Long, Double, BigInteger, etc val slices = new ArrayBuffer[Seq[T]](numSlices) val sliceSize = (nr.size + numSlices - 1) / numSlices // Round up to catch everything var r = nr @@ -102,10 +142,10 @@ private object ParallelCollectionRDD { slices } case _ => { - val array = seq.toArray // To prevent O(n^2) operations for List etc + val array = seq.toArray // To prevent O(n^2) operations for List etc (0 until numSlices).map(i => { val start = ((i * array.length.toLong) / numSlices).toInt - val end = (((i+1) * array.length.toLong) / numSlices).toInt + val end = (((i + 1) * array.length.toLong) / numSlices).toInt array.slice(start, end).toSeq }) } From 01f94931d5fe3121bb6413e6efba3473df975136 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 29 Jul 2013 19:23:41 -0700 Subject: [PATCH 0527/2521] Update the Python logistic regression example to read from a file and batch input records for more efficient NumPy computations --- python/examples/logistic_regression.py | 53 +++++++++++++------------- 1 file changed, 26 insertions(+), 27 deletions(-) diff --git a/python/examples/logistic_regression.py b/python/examples/logistic_regression.py index 3ac1bae4e9a..1e1b6d6e5b0 100755 --- a/python/examples/logistic_regression.py +++ b/python/examples/logistic_regression.py @@ -16,7 +16,8 @@ # """ -This example requires numpy (http://www.numpy.org/) +A logistic regression implementation that uses NumPy (http://www.numpy.org) to act on batches +of input data using efficient matrix operations. """ from collections import namedtuple from math import exp @@ -27,47 +28,45 @@ from pyspark import SparkContext -N = 100000 # Number of data points D = 10 # Number of dimensions -R = 0.7 # Scaling factor -ITERATIONS = 5 -np.random.seed(42) -DataPoint = namedtuple("DataPoint", ['x', 'y']) -from logistic_regression import DataPoint # So that DataPoint is properly serialized - - -def generateData(): - def generatePoint(i): - y = -1 if i % 2 == 0 else 1 - x = np.random.normal(size=D) + (y * R) - return DataPoint(x, y) - return [generatePoint(i) for i in range(N)] - +# Read a batch of points from the input file into a NumPy matrix object. We operate on batches to +# make further computations faster. +# The data file contains lines of the form

    Back to Master

  • Spark Java/Scala (Scaladoc)
  • Spark Python (Epydoc)
  • Spark Streaming Java/Scala (Scaladoc)
  • +
  • Spark ML Library (Scaladoc)
  • From cf08bb7a3e0596fe502fa618fdd6958772ebbeb5 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 12 Aug 2013 18:55:02 -0700 Subject: [PATCH 0695/2521] Fix import organization. --- core/src/main/scala/spark/api/java/JavaPairRDD.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index 6e00ef955a4..ff12e8b76cf 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -23,6 +23,7 @@ import java.util.Comparator import scala.Tuple2 import scala.collection.JavaConversions._ +import com.google.common.base.Optional import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.OutputFormat @@ -41,8 +42,6 @@ import spark.Partitioner._ import spark.RDD import spark.SparkContext.rddToPairRDDFunctions -import com.google.common.base.Optional - class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManifest[K], implicit val vManifest: ClassManifest[V]) extends JavaRDDLike[(K, V), JavaPairRDD[K, V]] { From 8fd5c7bc00b1104e4282959ec95b699955ded976 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Mon, 12 Aug 2013 18:00:35 -0700 Subject: [PATCH 0696/2521] Implementing SPARK-865: Add the equivalent of ADD_JARS to PySpark Now ADD_FILES uses a comma as file name separator. --- python/pyspark/shell.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 54ff1bf8e7c..c8297b662e3 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -7,10 +7,15 @@ import pyspark from pyspark.context import SparkContext +# this is the equivalent of ADD_JARS +add_files = os.environ.get("ADD_FILES").split(',') if os.environ.get("ADD_FILES") != None else None -sc = SparkContext(os.environ.get("MASTER", "local"), "PySparkShell") +sc = SparkContext(os.environ.get("MASTER", "local"), "PySparkShell", pyFiles=add_files) print "Spark context avaiable as sc." +if add_files != None: + print "Adding files: [%s]" % ", ".join(add_files) + # The ./pyspark script stores the old PYTHONSTARTUP value in OLD_PYTHONSTARTUP, # which allows us to execute the user's PYTHONSTARTUP file: _pythonstartup = os.environ.get('OLD_PYTHONSTARTUP') From 705c9ace2a893168aadfca7d80749f3597d9a24a Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 12 Aug 2013 22:08:36 -0700 Subject: [PATCH 0697/2521] Use less instances of the random class during ALS setup --- .../spark/mllib/recommendation/ALS.scala | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index 6c71dc1f320..974046d2609 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -124,9 +124,18 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l 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))) + var users = userOutLinks.mapPartitions(itr => { + val rand = new Random() + itr.map({case (x,y) => + (x,y.elementIds.map(u => randomFactor(rank, rand))) + }) + }) + var products = productOutLinks.mapPartitions(itr => { + val rand = new Random() + itr.map({case (x,y) => + (x,y.elementIds.map(u => randomFactor(rank, rand))) + }) + }) for (iter <- 0 until iterations) { // perform ALS update @@ -213,11 +222,9 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l } /** - * Make a random factor vector with the given seed. - * TODO: Initialize things using mapPartitionsWithIndex to make it faster? + * Make a random factor vector with the given random. */ - private def randomFactor(rank: Int, seed: Int): Array[Double] = { - val rand = new Random(seed) + private def randomFactor(rank: Int, rand: Random): Array[Double] = { Array.fill(rank)(rand.nextDouble) } From d145da818ee14c01ff08b29a546fb3c87016aa51 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 12 Aug 2013 22:13:08 -0700 Subject: [PATCH 0698/2521] Code review feedback :) --- .../scala/spark/mllib/recommendation/ALS.scala | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index 974046d2609..aeacbca5108 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -124,16 +124,16 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) // Initialize user and product factors randomly - var users = userOutLinks.mapPartitions(itr => { + var users = userOutLinks.mapPartitions {itr => val rand = new Random() - itr.map({case (x,y) => - (x,y.elementIds.map(u => randomFactor(rank, rand))) + itr.map({case (x, y) => + (x, y.elementIds.map(u => randomFactor(rank, rand))) }) - }) - var products = productOutLinks.mapPartitions(itr => { + } + var products = productOutLinks.mapPartitions {itr => val rand = new Random() - itr.map({case (x,y) => - (x,y.elementIds.map(u => randomFactor(rank, rand))) + itr.map({case (x, y) => + (x, y.elementIds.map(u => randomFactor(rank, rand))) }) }) From ed6a1646e62cf874e77df7a86ecd3e06bdcbee3e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 13 Aug 2013 09:27:07 -0700 Subject: [PATCH 0699/2521] Slight change to pr-784 --- core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala | 5 ++--- core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- core/src/main/scala/spark/metrics/MetricsSystem.scala | 5 ++++- core/src/main/scala/spark/ui/SparkUI.scala | 3 +-- core/src/test/scala/spark/metrics/MetricsSystemSuite.scala | 4 ++-- 5 files changed, 10 insertions(+), 9 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 f0a6ffe047a..c91e1db9f24 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala @@ -59,9 +59,8 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { } } - val metricsHandlers = master.masterMetricsSystem.metricsServlet.map(_.getHandlers) - .getOrElse(Array()) ++ master.applicationMetricsSystem.metricsServlet.map(_.getHandlers) - .getOrElse(Array()) + val metricsHandlers = master.masterMetricsSystem.getServletHandlers ++ + master.applicationMetricsSystem.getServletHandlers val handlers = metricsHandlers ++ Array[(String, Handler)]( ("/static", createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR)), 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 b408c63a02d..22295069dc7 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -48,7 +48,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val indexPage = new IndexPage(this) - val metricsHandlers = worker.metricsSystem.metricsServlet.map(_.getHandlers).getOrElse(Array()) + val metricsHandlers = worker.metricsSystem.getServletHandlers val handlers = metricsHandlers ++ Array[(String, Handler)]( ("/static", createStaticHandler(WorkerWebUI.STATIC_RESOURCE_DIR)), diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/spark/metrics/MetricsSystem.scala index 04c750b17e9..4e6c6b26c86 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/spark/metrics/MetricsSystem.scala @@ -73,7 +73,10 @@ private[spark] class MetricsSystem private (val instance: String) extends Loggin val registry = new MetricRegistry() // Treat MetricsServlet as a special sink as it should be exposed to add handlers to web ui - var metricsServlet: Option[MetricsServlet] = None + private var metricsServlet: Option[MetricsServlet] = None + + /** Get any UI handlers used by this metrics system. */ + def getServletHandlers = metricsServlet.map(_.getHandlers).getOrElse(Array()) metricsConfig.initialize() registerSources() diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 4bcfdeb62bc..1fd5a0989e1 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -45,8 +45,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { val exec = new ExecutorsUI(sc) // Add MetricsServlet handlers by default - val metricsServletHandlers = SparkEnv.get.metricsSystem.metricsServlet.map(_.getHandlers) - .getOrElse(Array()) + val metricsServletHandlers = SparkEnv.get.metricsSystem.getServletHandlers val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ exec.getHandlers ++ metricsServletHandlers ++ handlers diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala index 35c2ae41e9f..dc65ac6994b 100644 --- a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala @@ -34,7 +34,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { assert(sources.length === 0) assert(sinks.length === 0) - assert(metricsSystem.metricsServlet != None) + assert(!metricsSystem.getServletHandlers.isEmpty) } test("MetricsSystem with sources add") { @@ -44,7 +44,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { assert(sources.length === 0) assert(sinks.length === 1) - assert(metricsSystem.metricsServlet != None) + assert(!metricsSystem.getServletHandlers.isEmpty) val source = new spark.deploy.master.MasterSource(null) metricsSystem.registerSource(source) From 654087194d232221dfb64ba646c8a8e12649f961 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Tue, 13 Aug 2013 11:43:49 -0700 Subject: [PATCH 0700/2521] Change SVM to use {0,1} labels. Also add a data validation check to make sure classification labels are always 0 or 1 and add an appropriate test case. --- .../classification/LogisticRegression.scala | 16 ++++--- .../spark/mllib/classification/SVM.scala | 21 +++++++--- .../spark/mllib/optimization/Gradient.scala | 12 ++++-- .../GeneralizedLinearAlgorithm.scala | 9 +++- .../spark/mllib/util/DataValidators.scala | 42 +++++++++++++++++++ .../spark/mllib/util/SVMDataGenerator.scala | 9 ++-- .../spark/mllib/classification/SVMSuite.scala | 33 ++++++++++++--- 7 files changed, 116 insertions(+), 26 deletions(-) create mode 100644 mllib/src/main/scala/spark/mllib/util/DataValidators.scala diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index 30ee0ab0ff8..24f9f4e76bc 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -17,12 +17,13 @@ package spark.mllib.classification +import scala.math.round + import spark.{Logging, RDD, SparkContext} import spark.mllib.optimization._ import spark.mllib.regression._ import spark.mllib.util.MLUtils - -import scala.math.round +import spark.mllib.util.DataValidators import org.jblas.DoubleMatrix @@ -59,10 +60,13 @@ class LogisticRegressionWithSGD private ( val gradient = new LogisticGradient() val updater = new SimpleUpdater() - val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) - .setNumIterations(numIterations) - .setRegParam(regParam) - .setMiniBatchFraction(miniBatchFraction) + override val optimizer = new GradientDescent(gradient, updater) + .setStepSize(stepSize) + .setNumIterations(numIterations) + .setRegParam(regParam) + .setMiniBatchFraction(miniBatchFraction) + override val validateFuncs = List(DataValidators.classificationLabels) + /** * Construct a LogisticRegression object with default parameters */ diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index f799cb28298..d2b50f49876 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -18,10 +18,12 @@ package spark.mllib.classification import scala.math.signum + import spark.{Logging, RDD, SparkContext} import spark.mllib.optimization._ import spark.mllib.regression._ import spark.mllib.util.MLUtils +import spark.mllib.util.DataValidators import org.jblas.DoubleMatrix @@ -45,6 +47,7 @@ class SVMModel( /** * Train an SVM using Stochastic Gradient Descent. + * NOTE: Labels used in SVM should be {0, 1} */ class SVMWithSGD private ( var stepSize: Double, @@ -56,10 +59,14 @@ class SVMWithSGD private ( val gradient = new HingeGradient() val updater = new SquaredL2Updater() - val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) - .setNumIterations(numIterations) - .setRegParam(regParam) - .setMiniBatchFraction(miniBatchFraction) + override val optimizer = new GradientDescent(gradient, updater) + .setStepSize(stepSize) + .setNumIterations(numIterations) + .setRegParam(regParam) + .setMiniBatchFraction(miniBatchFraction) + + override val validateFuncs = List(DataValidators.classificationLabels) + /** * Construct a SVM object with default parameters */ @@ -71,7 +78,7 @@ class SVMWithSGD private ( } /** - * Top-level methods for calling SVM. + * Top-level methods for calling SVM. NOTE: Labels used in SVM should be {0, 1} */ object SVMWithSGD { @@ -80,6 +87,7 @@ object SVMWithSGD { * of iterations of gradient descent using the specified step size. Each iteration uses * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in * gradient descent are initialized using the initial weights provided. + * NOTE: Labels used in SVM should be {0, 1} * * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. @@ -106,6 +114,7 @@ object SVMWithSGD { * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number * of iterations of gradient descent using the specified step size. Each iteration uses * `miniBatchFraction` fraction of the data to calculate the gradient. + * NOTE: Labels used in SVM should be {0, 1} * * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. @@ -128,6 +137,7 @@ object SVMWithSGD { * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number * of iterations of gradient descent using the specified step size. We use the entire data set to * update the gradient in each iteration. + * NOTE: Labels used in SVM should be {0, 1} * * @param input RDD of (label, array of features) pairs. * @param stepSize Step size to be used for each iteration of Gradient Descent. @@ -149,6 +159,7 @@ object SVMWithSGD { * Train a SVM model given an RDD of (label, features) pairs. We run a fixed number * of iterations of gradient descent using a step size of 1.0. We use the entire data set to * update the gradient in each iteration. + * NOTE: Labels used in SVM should be {0, 1} * * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala index e72b8b3a92e..58bfe3f37b4 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala @@ -77,16 +77,20 @@ class SquaredGradient extends Gradient { /** * Compute gradient and loss for a Hinge loss function. + * NOTE: This assumes that the labels are {0,1} */ class HingeGradient extends Gradient { - override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): + override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): (DoubleMatrix, Double) = { val dotProduct = data.dot(weights) + val labelScaled = 2*label - 1.0 - if (1.0 > label * dotProduct) - (data.mul(-label), 1.0 - label * dotProduct) + // Our loss function with {0, 1} labels is max(0, 1 - (2y – 1) (f_w(x))) + // Therefore the gradient is -(2y - 1)*x + if (1.0 > labelScaled * dotProduct) + (data.mul(-labelScaled), 1.0 - labelScaled * dotProduct) else - (DoubleMatrix.zeros(1,weights.length), 0.0) + (DoubleMatrix.zeros(1, weights.length), 0.0) } } diff --git a/mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 4ecafff08b3..55edb3def51 100644 --- a/mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -17,7 +17,7 @@ package spark.mllib.regression -import spark.{Logging, RDD} +import spark.{Logging, RDD, SparkException} import spark.mllib.optimization._ import org.jblas.DoubleMatrix @@ -83,6 +83,8 @@ abstract class GeneralizedLinearModel(val weights: Array[Double], val intercept: abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] extends Logging with Serializable { + protected val validateFuncs: Seq[RDD[LabeledPoint] => Boolean] = List() + val optimizer: Optimizer /** @@ -116,6 +118,11 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] */ def run(input: RDD[LabeledPoint], initialWeights: Array[Double]) : M = { + // Check the data properties before running the optimizer + if (!validateFuncs.forall(func => func(input))) { + throw new SparkException("Input validation failed.") + } + // Add a extra variable consisting of all 1.0's for the intercept. val data = if (addIntercept) { input.map(labeledPoint => (labeledPoint.label, Array(1.0, labeledPoint.features:_*))) diff --git a/mllib/src/main/scala/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/spark/mllib/util/DataValidators.scala new file mode 100644 index 00000000000..57553accf1f --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/util/DataValidators.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.mllib.util + +import spark.{RDD, Logging} +import spark.mllib.regression.LabeledPoint + +/** + * A collection of methods used to validate data before applying ML algorithms. + */ +object DataValidators extends Logging { + + /** + * Function to check if labels used for classification are either zero or one. + * + * @param data - input data set that needs to be checked + * + * @return True if labels are all zero or one, false otherwise. + */ + val classificationLabels: RDD[LabeledPoint] => Boolean = { data => + val numInvalid = data.filter(x => x.label != 1.0 && x.label != 0.0).count() + if (numInvalid != 0) { + logError("Classification labels should be 0 or 1. Found " + numInvalid + " invalid labels") + } + numInvalid == 0 + } +} diff --git a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala index e02bd190f6c..eff456cad63 100644 --- a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala @@ -1,7 +1,6 @@ package spark.mllib.util import scala.util.Random -import scala.math.signum import spark.{RDD, SparkContext} @@ -30,8 +29,8 @@ object SVMDataGenerator { val sc = new SparkContext(sparkMaster, "SVMGenerator") val globalRnd = new Random(94720) - val trueWeights = new DoubleMatrix(1, nfeatures+1, - Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) + val trueWeights = new DoubleMatrix(1, nfeatures + 1, + Array.fill[Double](nfeatures + 1)(globalRnd.nextGaussian()):_*) val data: RDD[LabeledPoint] = sc.parallelize(0 until nexamples, parts).map { idx => val rnd = new Random(42 + idx) @@ -39,11 +38,13 @@ object SVMDataGenerator { val x = Array.fill[Double](nfeatures) { rnd.nextDouble() * 2.0 - 1.0 } - val y = signum((new DoubleMatrix(1, x.length, x:_*)).dot(trueWeights) + rnd.nextGaussian() * 0.1) + val yD = (new DoubleMatrix(1, x.length, x:_*)).dot(trueWeights) + rnd.nextGaussian() * 0.1 + val y = if (yD < 0) 0.0 else 1.0 LabeledPoint(y, x) } MLUtils.saveLabeledData(data, outputPath) + sc.stop() } } diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index 04f631d80f8..f392efa405d 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -50,11 +50,9 @@ object SVMSuite { val x = Array.fill[Array[Double]](nPoints)( Array.fill[Double](weights.length)(rnd.nextGaussian())) val y = x.map { xi => - signum( - (new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) + - intercept + - 0.1 * rnd.nextGaussian() - ).toInt + val yD = (new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) + + intercept + 0.01 * rnd.nextGaussian() + if (yD < 0) 0.0 else 1.0 } y.zip(x).map(p => LabeledPoint(p._1, p._2)) } @@ -100,7 +98,7 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { val model = svm.run(testRDD) val validationData = SVMSuite.generateSVMInput(A, Array[Double](B,C), nPoints, 17) - val validationRDD = sc.parallelize(validationData,2) + val validationRDD = sc.parallelize(validationData, 2) // Test prediction on RDD. validatePrediction(model.predict(validationRDD.map(_.features)).collect(), validationData) @@ -139,4 +137,27 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { // Test prediction on Array. validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } + + test("SVM with invalid labels") { + val nPoints = 10000 + + val A = 2.0 + val B = -1.5 + val C = 1.0 + + val testData = SVMSuite.generateSVMInput(A, Array[Double](B,C), nPoints, 42) + val testRDD = sc.parallelize(testData, 2) + + val testRDDInvalid = testRDD.map { lp => + if (lp.label == 0.0) { + LabeledPoint(-1.0, lp.features) + } else { + lp + } + } + + intercept[spark.SparkException] { + val model = SVMWithSGD.train(testRDDInvalid, 100) + } + } } From c92dd627ca4c0ce580b8be54010827440da77505 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 13 Aug 2013 12:39:50 -0700 Subject: [PATCH 0701/2521] Properly account for killed tasks. The TaskState class's isFinished() method didn't return true for KILLED tasks, which means some resources are never reclaimed for tasks that are killed. This also made it inconsistent with the isFinished() method used by CoarseMesosSchedulerBackend. --- core/src/main/scala/spark/TaskState.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/TaskState.scala b/core/src/main/scala/spark/TaskState.scala index 9df7d8277b8..0955a91c6f0 100644 --- a/core/src/main/scala/spark/TaskState.scala +++ b/core/src/main/scala/spark/TaskState.scala @@ -26,7 +26,7 @@ private[spark] object TaskState type TaskState = Value - def isFinished(state: TaskState) = Seq(FINISHED, FAILED, LOST).contains(state) + def isFinished(state: TaskState) = Seq(FINISHED, FAILED, KILLED, LOST).contains(state) def toMesos(state: TaskState): MesosTaskState = state match { case LAUNCHING => MesosTaskState.TASK_STARTING From 0ab6ff4c3252e7cb9ea573e09d9188da1fcb87cc Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Tue, 13 Aug 2013 13:57:06 -0700 Subject: [PATCH 0702/2521] Fix SVM model and unit test to work with {0,1}. Also rename validateFuncs to validators. --- .../mllib/classification/LogisticRegression.scala | 2 +- .../main/scala/spark/mllib/classification/SVM.scala | 5 +++-- .../scala/spark/mllib/optimization/Gradient.scala | 8 +++++--- .../mllib/regression/GeneralizedLinearAlgorithm.scala | 4 ++-- .../scala/spark/mllib/classification/SVMSuite.scala | 11 +++++++---- 5 files changed, 18 insertions(+), 12 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index 24f9f4e76bc..7f0b1ba841a 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -65,7 +65,7 @@ class LogisticRegressionWithSGD private ( .setNumIterations(numIterations) .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) - override val validateFuncs = List(DataValidators.classificationLabels) + override val validators = List(DataValidators.classificationLabels) /** * Construct a LogisticRegression object with default parameters diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index d2b50f49876..b680d81e86c 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -41,7 +41,8 @@ class SVMModel( override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, intercept: Double) = { - signum(dataMatrix.dot(weightMatrix) + intercept) + val margin = dataMatrix.dot(weightMatrix) + intercept + if (margin < 0) 0.0 else 1.0 } } @@ -65,7 +66,7 @@ class SVMWithSGD private ( .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) - override val validateFuncs = List(DataValidators.classificationLabels) + override val validators = List(DataValidators.classificationLabels) /** * Construct a SVM object with default parameters diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala index 58bfe3f37b4..05568f55af0 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala @@ -84,13 +84,15 @@ class HingeGradient extends Gradient { (DoubleMatrix, Double) = { val dotProduct = data.dot(weights) - val labelScaled = 2*label - 1.0 // Our loss function with {0, 1} labels is max(0, 1 - (2y – 1) (f_w(x))) // Therefore the gradient is -(2y - 1)*x - if (1.0 > labelScaled * dotProduct) + val labelScaled = 2 * label - 1.0 + + if (1.0 > labelScaled * dotProduct) { (data.mul(-labelScaled), 1.0 - labelScaled * dotProduct) - else + } else { (DoubleMatrix.zeros(1, weights.length), 0.0) + } } } diff --git a/mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 55edb3def51..03f991df399 100644 --- a/mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -83,7 +83,7 @@ abstract class GeneralizedLinearModel(val weights: Array[Double], val intercept: abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] extends Logging with Serializable { - protected val validateFuncs: Seq[RDD[LabeledPoint] => Boolean] = List() + protected val validators: Seq[RDD[LabeledPoint] => Boolean] = List() val optimizer: Optimizer @@ -119,7 +119,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] def run(input: RDD[LabeledPoint], initialWeights: Array[Double]) : M = { // Check the data properties before running the optimizer - if (!validateFuncs.forall(func => func(input))) { + if (!validators.forall(func => func(input))) { throw new SparkException("Input validation failed.") } diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index f392efa405d..8fa9e4639b4 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -48,7 +48,7 @@ object SVMSuite { val rnd = new Random(seed) val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) val x = Array.fill[Array[Double]](nPoints)( - Array.fill[Double](weights.length)(rnd.nextGaussian())) + Array.fill[Double](weights.length)(rnd.nextDouble() * 2.0 - 1.0)) val y = x.map { xi => val yD = (new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) + intercept + 0.01 * rnd.nextGaussian() @@ -83,7 +83,8 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { test("SVM using local random SGD") { val nPoints = 10000 - val A = 2.0 + // NOTE: Intercept should be small for generating equal 0s and 1s + val A = 0.01 val B = -1.5 val C = 1.0 @@ -110,7 +111,8 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { test("SVM local random SGD with initial weights") { val nPoints = 10000 - val A = 2.0 + // NOTE: Intercept should be small for generating equal 0s and 1s + val A = 0.01 val B = -1.5 val C = 1.0 @@ -141,7 +143,8 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { test("SVM with invalid labels") { val nPoints = 10000 - val A = 2.0 + // NOTE: Intercept should be small for generating equal 0s and 1s + val A = 0.01 val B = -1.5 val C = 1.0 From f2b8dd3929ce2a8ccfea8bcf05b3c72a7acd63fd Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 13 Aug 2013 14:21:49 -0700 Subject: [PATCH 0703/2521] second indexedrdd design --- .../main/scala/spark/PairRDDFunctions.scala | 4 + .../src/main/scala/spark/rdd/IndexedRDD.scala | 132 ++++++++++++++++++ 2 files changed, 136 insertions(+) create mode 100644 core/src/main/scala/spark/rdd/IndexedRDD.scala diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 6b0cc2fbf15..28b46990f85 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -715,6 +715,10 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( */ def values: RDD[V] = self.map(_._2) + + def index(existingIndex: RDD[JHashMap[K,Int]] = null): IndexedRDD[K,V] = + IndexedRDD(self, existingIndex) + private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure private[spark] def getValueClass() = implicitly[ClassManifest[V]].erasure diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala new file mode 100644 index 00000000000..55ca353618e --- /dev/null +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.rdd + +import java.util.{HashMap => JHashMap, BitSet => JBitSet, HashSet => JHashSet} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer + + +import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} + +import spark.PairRDDFunctions +import spark.SparkContext._ + + +// import java.io.{ObjectOutputStream, IOException} + + +/** + * An IndexedRDD is an RDD[(K,V)] where each K is unique. + * + * The IndexedRDD contains an index datastructure that can + * be used to accelerate join and aggregation operations. + */ +class IndexedRDD[K: ClassManifest, V: ClassManifest]( + sc: SparkContext, + val indexRDD: RDD[ JHashMap[K, Int] ], + val valuesRDD: RDD[ Array[Seq[V]] ]) + extends RDD[(K, V)](sc, + List(new OneToOneDependency(indexRDD), new OneToOneDependency(valuesRDD)) ) { + + + + val tuples = new ZippedRDD[ JHashMap[K, Int], Array[Seq[V]] ](sc, indexRDD, valuesRDD) + + + override val partitioner = indexRDD.partitioner + override def getPartitions: Array[Partition] = tuples.getPartitions + override def getPreferredLocations(s: Partition): Seq[String] = tuples.getPreferredLocations(s) + + + + + + /** + * Provide the RDD[(K,V)] equivalent output. + */ + override def compute(part: Partition, context: TaskContext): Iterator[(K, V)] = { + tuples.compute(part, context).flatMap { case (indexMap, values) => + // Walk the index to construct the key, value pairs + indexMap.iterator + // Extract rows with key value pairs and indicators + .map{ case (k, ind) => (k, values(ind)) } + // Remove tuples that aren't actually present in the array + .filter{ case (_, valar) => valar != null } + // Extract the pair (removing the indicator from the tuple) + .flatMap{ case (k, valar) => valar.map(v => (k,v))} + } + } +} + +object IndexedRDD { + def apply[K: ClassManifest, V: ClassManifest]( + tbl: RDD[(K,V)], + existingIndex: RDD[JHashMap[K,Int]] = null ): IndexedRDD[K, V] = { + + if(existingIndex == null) { + // build th index + val groups = tbl.groupByKey().mapPartitions( iter => { + val indexMap = new JHashMap[K, Int]() + val values = new ArrayBuffer[Seq[V]]() + for((k,ar) <- iter){ + val ind = values.size + indexMap.put(k, ind) + values.append(ar) + } + List((indexMap, values.toArray)).iterator + }, true).cache + // extract the index and the values + val index = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) + val values = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) + new IndexedRDD[K,V](tbl.context, index, values) + } else { + val index = existingIndex + // Shuffle the table according to the index (if necessary) + val shuffledTbl = + if (tbl.partitioner == Some(index.partitioner)) { + tbl + } else { + new ShuffledRDD[K,V](tbl, index.partitioner.get()) + } + + // Use the index to build the new values table + val values = index.zipPartitions[ (K, Seq[V]), Array[Seq[V]] ]( + (indexIter, tblIter) => { + // There is only one map + val index: JHashMap[K,Int] = iter.next() + assert(!iter.hasNext()) + val values = new Array[Seq[V]](index.size) + for((k,a) <- tblIter) { + assert(index.contains(k)) + values(index.get(k)) = a + } + values + }, shuffleTbl) + + new IndexedRDD[K,V](index, values) + } + } + +} + + + + + From 1beb843a6f94644387dba36e92f4850e57de9194 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 13 Aug 2013 14:27:40 -0700 Subject: [PATCH 0704/2521] Reuse the set of failed states rather than creating a new object each time --- core/src/main/scala/spark/TaskState.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/TaskState.scala b/core/src/main/scala/spark/TaskState.scala index 0955a91c6f0..bf757530565 100644 --- a/core/src/main/scala/spark/TaskState.scala +++ b/core/src/main/scala/spark/TaskState.scala @@ -24,9 +24,11 @@ private[spark] object TaskState val LAUNCHING, RUNNING, FINISHED, FAILED, KILLED, LOST = Value + val FINISHED_STATES = Set(FINISHED, FAILED, KILLED, LOST) + type TaskState = Value - def isFinished(state: TaskState) = Seq(FINISHED, FAILED, KILLED, LOST).contains(state) + def isFinished(state: TaskState) = FINISHED_STATES.contains(state) def toMesos(state: TaskState): MesosTaskState = state match { case LAUNCHING => MesosTaskState.TASK_STARTING From a7feb69ae872b39a24aaec3ffde214367240ca0a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 13 Aug 2013 15:07:03 -0700 Subject: [PATCH 0705/2521] Print run command to stderr rather than stdout --- core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 7423eca11e0..a5ea4fe8392 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -169,10 +169,10 @@ private[spark] class ExecutorRunner( // Redirect its stdout and stderr to files val stdout = new File(executorDir, "stdout") - Files.write(header, stdout, Charsets.UTF_8) redirectStream(process.getInputStream, stdout) val stderr = new File(executorDir, "stderr") + Files.write(header, stderr, Charsets.UTF_8) redirectStream(process.getErrorStream, stderr) // Wait for it to exit; this is actually a bad thing if it happens, because we expect to run From f0382007dc7268c603db3f0227e2ca01dc5e7b71 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 Aug 2013 14:45:19 -0700 Subject: [PATCH 0706/2521] Bug fix for display of shuffle read/write metrics. This fixes an error where empty cells are missing if a given task has no shuffle read/write. --- .../main/scala/spark/ui/jobs/StagePage.scala | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 1b071a91e55..884c065deec 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -87,7 +87,7 @@ private[spark] class StagePage(parent: JobProgressUI) { {if (hasShuffleWrite) Seq("Shuffle Write") else Nil} ++ Seq("Details") - val taskTable = listingTable(taskHeaders, taskRow, tasks) + val taskTable = listingTable(taskHeaders, taskRow(hasShuffleRead, hasShuffleWrite), tasks) // Excludes tasks which failed and have incomplete metrics val validTasks = tasks.filter(t => t._1.status == "SUCCESS" && (t._2.isDefined)) @@ -135,7 +135,8 @@ private[spark] class StagePage(parent: JobProgressUI) { } - def taskRow(taskData: (TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])): Seq[Node] = { + def taskRow(shuffleRead: Boolean, shuffleWrite: Boolean) + (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 @@ -154,10 +155,14 @@ private[spark] class StagePage(parent: JobProgressUI) { {info.taskLocality} {info.hostPort} {dateFmt.format(new Date(info.launchTime))} - {metrics.flatMap{m => m.shuffleReadMetrics}.map{s => - {Utils.memoryBytesToString(s.remoteBytesRead)}}.getOrElse("")} - {metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => - {Utils.memoryBytesToString(s.shuffleBytesWritten)}}.getOrElse("")} + {if (shuffleRead) { + {metrics.flatMap{m => m.shuffleReadMetrics}.map{s => + Utils.memoryBytesToString(s.remoteBytesRead)}.getOrElse("")} + }} + {if (shuffleWrite) { + {metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => + Utils.memoryBytesToString(s.shuffleBytesWritten)}.getOrElse("")} + }} {exception.map(e => {e.className} ({e.description})
    From 4e9f0c2df6508db8fbe00901aa78fb5809e46010 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 Aug 2013 15:08:39 -0700 Subject: [PATCH 0707/2521] Capturing GC detials in TaskMetrics --- .../main/scala/spark/executor/Executor.scala | 22 ++++++++++++------- .../scala/spark/executor/TaskMetrics.scala | 7 +++++- .../scheduler/local/LocalScheduler.scala | 13 ++++++++++- .../main/scala/spark/ui/jobs/StagePage.scala | 5 +++++ 4 files changed, 37 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 8a74a8d853a..6bfb1320233 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -17,18 +17,17 @@ package spark.executor -import java.io.{File, FileOutputStream} -import java.net.{URI, URL, URLClassLoader} +import java.io.{File} +import java.lang.management.ManagementFactory +import java.nio.ByteBuffer import java.util.concurrent._ -import org.apache.hadoop.fs.FileUtil - -import scala.collection.mutable.{ArrayBuffer, Map, HashMap} +import scala.collection.JavaConversions._ +import scala.collection.mutable.HashMap -import spark.broadcast._ import spark.scheduler._ import spark._ -import java.nio.ByteBuffer + /** * The Mesos executor for Spark. @@ -116,6 +115,9 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert context.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER) var attemptedTask: Option[Task[Any]] = None var taskStart: Long = 0 + def getTotalGCTime = ManagementFactory.getGarbageCollectorMXBeans.map(g => g.getCollectionTime).sum + val startGCTime = getTotalGCTime + try { SparkEnv.set(env) Accumulators.clear() @@ -132,6 +134,7 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert m.hostname = Utils.localHostName m.executorDeserializeTime = (taskStart - startTime).toInt m.executorRunTime = (taskFinish - taskStart).toInt + m.jvmGCTime = getTotalGCTime - startGCTime } //TODO I'd also like to track the time it takes to serialize the task results, but that is huge headache, b/c // we need to serialize the task metrics first. If TaskMetrics had a custom serialized format, we could @@ -155,7 +158,10 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert case t: Throwable => { val serviceTime = (System.currentTimeMillis() - taskStart).toInt val metrics = attemptedTask.flatMap(t => t.metrics) - metrics.foreach{m => m.executorRunTime = serviceTime} + metrics.foreach{m => + m.executorRunTime = serviceTime + m.jvmGCTime = getTotalGCTime - startGCTime + } val reason = ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace, metrics) context.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index 31516278398..47b8890bee9 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -31,13 +31,18 @@ class TaskMetrics extends Serializable { /** * Time the executor spends actually running the task (including fetching shuffle data) */ - var executorRunTime:Int = _ + var executorRunTime: Int = _ /** * The number of bytes this task transmitted back to the driver as the TaskResult */ var resultSize: Long = _ + /** + * Amount of time the JVM spent in garbage collection while executing this task + */ + var jvmGCTime: Long = _ + /** * If this task reads from shuffle output, metrics on getting shuffle data will be collected here */ diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index f274b1a7679..322c32543f7 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -18,8 +18,11 @@ package spark.scheduler.local import java.io.File +import java.lang.management.ManagementFactory import java.util.concurrent.atomic.AtomicInteger import java.nio.ByteBuffer + +import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet @@ -31,6 +34,7 @@ import spark.scheduler._ import spark.scheduler.cluster._ import spark.scheduler.cluster.SchedulingMode.SchedulingMode import akka.actor._ +import management.ManagementFactory /** * A FIFO or Fair TaskScheduler implementation that runs tasks locally in a thread pool. Optionally @@ -173,6 +177,9 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: var attemptedTask: Option[Task[_]] = None val start = System.currentTimeMillis() var taskStart: Long = 0 + def getTotalGCTime = ManagementFactory.getGarbageCollectorMXBeans.map(g => g.getCollectionTime).sum + val startGCTime = getTotalGCTime + try { Accumulators.clear() Thread.currentThread().setContextClassLoader(classLoader) @@ -202,6 +209,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: val serviceTime = System.currentTimeMillis() - taskStart logInfo("Finished " + taskId) deserializedTask.metrics.get.executorRunTime = serviceTime.toInt + deserializedTask.metrics.get.jvmGCTime = getTotalGCTime - startGCTime deserializedTask.metrics.get.executorDeserializeTime = deserTime.toInt val taskResult = new TaskResult(result, accumUpdates, deserializedTask.metrics.getOrElse(null)) val serializedResult = ser.serialize(taskResult) @@ -210,7 +218,10 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: case t: Throwable => { val serviceTime = System.currentTimeMillis() - taskStart val metrics = attemptedTask.flatMap(t => t.metrics) - metrics.foreach{m => m.executorRunTime = serviceTime.toInt} + metrics.foreach{ m => + m.executorRunTime = serviceTime.toInt + m.jvmGCTime = getTotalGCTime - startGCTime + } 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/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 884c065deec..ce01f6c17ee 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -85,6 +85,7 @@ private[spark] class StagePage(parent: JobProgressUI) { Seq("Task ID", "Status", "Duration", "Locality Level", "Worker", "Launch Time") ++ {if (hasShuffleRead) Seq("Shuffle Read") else Nil} ++ {if (hasShuffleWrite) Seq("Shuffle Write") else Nil} ++ + Seq("GC Time") ++ Seq("Details") val taskTable = listingTable(taskHeaders, taskRow(hasShuffleRead, hasShuffleWrite), tasks) @@ -145,6 +146,7 @@ private[spark] class StagePage(parent: JobProgressUI) { 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("") + val gcTime = metrics.map(m => m.jvmGCTime).getOrElse(0L) {info.taskId} @@ -163,6 +165,9 @@ private[spark] class StagePage(parent: JobProgressUI) { {metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => Utils.memoryBytesToString(s.shuffleBytesWritten)}.getOrElse("")} }} + + {if (gcTime > 0) {parent.formatDuration(gcTime)} else ""} + {exception.map(e => {e.className} ({e.description})
    From 024e5c5ce103d7b45ed5b455747fd784cbde17c9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 Aug 2013 16:14:38 -0700 Subject: [PATCH 0708/2521] Correct sorting order for stages --- core/src/main/scala/spark/ui/jobs/StagePage.scala | 2 +- core/src/main/scala/spark/ui/jobs/StageTable.scala | 14 +++++--------- 2 files changed, 6 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index ce01f6c17ee..28a6d7b179b 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -165,7 +165,7 @@ private[spark] class StagePage(parent: JobProgressUI) { {metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => Utils.memoryBytesToString(s.shuffleBytesWritten)}.getOrElse("")} }} - + {if (gcTime > 0) {parent.formatDuration(gcTime)} else ""} {exception.map(e => diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/spark/ui/jobs/StageTable.scala index 5068a025fa2..19b07ccedac 100644 --- a/core/src/main/scala/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/spark/ui/jobs/StageTable.scala @@ -49,13 +49,6 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU } - private def getElapsedTime(submitted: Option[Long], completed: Long): String = { - submitted match { - case Some(t) => parent.formatDuration(completed - t) - case _ => "Unknown" - } - } - private def makeProgressBar(started: Int, completed: Int, failed: String, total: Int): Seq[Node] = { val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) val startWidth = "width: %s%%".format((started.toDouble/total)*100) @@ -98,6 +91,8 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU val nameLink = {s.name} val description = listener.stageToDescription.get(s) .map(d =>
    {d}
    {nameLink}
    ).getOrElse(nameLink) + val finishTime = s.completionTime.getOrElse(System.currentTimeMillis()) + val duration = s.submissionTime.map(t => finishTime - t) {s.id} @@ -106,8 +101,9 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU } {description} {submissionTime} - {getElapsedTime(s.submissionTime, - s.completionTime.getOrElse(System.currentTimeMillis()))} + + {duration.map(d => parent.formatDuration(d)).getOrElse("Unknown")} + {makeProgressBar(startedTasks, completedTasks, failedTasks, totalTasks)} From fab5cee111767a7edbbdc9020f4c1d456b264184 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 12 Aug 2013 17:52:48 -0700 Subject: [PATCH 0709/2521] Correcting terminology in RDD page --- core/src/main/scala/spark/ui/storage/RDDPage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 4c3ee12c987..40f94b42a6f 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -123,7 +123,7 @@ private[spark] class RDDPage(parent: BlockManagerUI) { {status.blockManagerId.host + ":" + status.blockManagerId.port} {Utils.memoryBytesToString(status.memUsed(prefix))} - ({Utils.memoryBytesToString(status.memRemaining)} Total Available) + ({Utils.memoryBytesToString(status.memRemaining)} Remaining) {Utils.memoryBytesToString(status.diskUsed(prefix))} From c874625354de7117da9586cfbbe919bb6801a932 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Tue, 13 Aug 2013 16:55:53 -0700 Subject: [PATCH 0710/2521] Specify label format in LogisticRegression. --- .../spark/mllib/classification/LogisticRegression.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index 7f0b1ba841a..474ca6e97c9 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -48,6 +48,7 @@ class LogisticRegressionModel( /** * Train a classification model for Logistic Regression using Stochastic Gradient Descent. + * NOTE: Labels used in Logistic Regression should be {0, 1} */ class LogisticRegressionWithSGD private ( var stepSize: Double, @@ -79,6 +80,7 @@ class LogisticRegressionWithSGD private ( /** * Top-level methods for calling Logistic Regression. + * NOTE: Labels used in Logistic Regression should be {0, 1} */ object LogisticRegressionWithSGD { // NOTE(shivaram): We use multiple train methods instead of default arguments to support @@ -89,6 +91,7 @@ object LogisticRegressionWithSGD { * number of iterations of gradient descent using the specified step size. Each iteration uses * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in * gradient descent are initialized using the initial weights provided. + * NOTE: Labels used in Logistic Regression should be {0, 1} * * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. @@ -113,6 +116,7 @@ object LogisticRegressionWithSGD { * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed * number of iterations of gradient descent using the specified step size. Each iteration uses * `miniBatchFraction` fraction of the data to calculate the gradient. + * NOTE: Labels used in Logistic Regression should be {0, 1} * * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. @@ -135,6 +139,7 @@ object LogisticRegressionWithSGD { * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed * number of iterations of gradient descent using the specified step size. We use the entire data * set to update the gradient in each iteration. + * NOTE: Labels used in Logistic Regression should be {0, 1} * * @param input RDD of (label, array of features) pairs. * @param stepSize Step size to be used for each iteration of Gradient Descent. @@ -155,6 +160,7 @@ object LogisticRegressionWithSGD { * Train a logistic regression model given an RDD of (label, features) pairs. We run a fixed * number of iterations of gradient descent using a step size of 1.0. We use the entire data set * to update the gradient in each iteration. + * NOTE: Labels used in Logistic Regression should be {0, 1} * * @param input RDD of (label, array of features) pairs. * @param numIterations Number of iterations of gradient descent to run. From c2231763885540a2268b45c53ebd06f9baaf06aa Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 13 Aug 2013 16:56:37 -0700 Subject: [PATCH 0711/2521] Small style clean-up --- core/src/main/scala/spark/executor/Executor.scala | 2 +- core/src/main/scala/spark/scheduler/local/LocalScheduler.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 6bfb1320233..c5202d94b02 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -158,7 +158,7 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert case t: Throwable => { val serviceTime = (System.currentTimeMillis() - taskStart).toInt val metrics = attemptedTask.flatMap(t => t.metrics) - metrics.foreach{m => + metrics.foreach {m => m.executorRunTime = serviceTime m.jvmGCTime = getTotalGCTime - startGCTime } diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 322c32543f7..33e7a10ea47 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -218,7 +218,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: case t: Throwable => { val serviceTime = System.currentTimeMillis() - taskStart val metrics = attemptedTask.flatMap(t => t.metrics) - metrics.foreach{ m => + metrics.foreach {m => m.executorRunTime = serviceTime.toInt m.jvmGCTime = getTotalGCTime - startGCTime } From d9588183fafaeba1f282c3458132a4e2291518ad Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 13 Aug 2013 18:50:20 -0700 Subject: [PATCH 0712/2521] Update to Mesos 0.12.1 --- pom.xml | 2 +- project/SparkBuild.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index b434e0e3030..7e6d38df9f9 100644 --- a/pom.xml +++ b/pom.xml @@ -70,7 +70,7 @@ 1.5 2.9.3 - 0.9.0-incubating + 0.12.1 2.0.3 1.7.2 4.1.2 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index e5c8e8d230d..f6519c82874 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -181,7 +181,7 @@ object SparkBuild extends Build { "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "net.liftweb" % "lift-json_2.9.2" % "2.5", - "org.apache.mesos" % "mesos" % "0.9.0-incubating", + "org.apache.mesos" % "mesos" % "0.12.1", "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", From a1227708e91946c34fcd5ff074dbca7ea5a93660 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Tue, 13 Aug 2013 20:06:47 -0700 Subject: [PATCH 0713/2521] Set SPARK_CLASSPATH for maven repl tests --- repl/pom.xml | 30 ++++++++++++++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/repl/pom.xml b/repl/pom.xml index 7d8da03254b..862595b9f94 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -73,6 +73,35 @@ target/scala-${scala.version}/classes target/scala-${scala.version}/test-classes + + org.apache.maven.plugins + maven-antrun-plugin + + + test + + run + + + true + + + + + + + + + + + + + + + + + + org.scalatest scalatest-maven-plugin @@ -80,6 +109,7 @@ ${basedir}/.. 1 + ${spark.classpath} From a88aa5e6ed98d212b25a534566e417401da9cc7d Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 13 Aug 2013 23:44:58 -0700 Subject: [PATCH 0714/2521] Fixed 2 bugs in executor UI. 1) UI crashed if the executor UI was loaded before any tasks started. 2) The total tasks was incorrectly reported due to using string (rather than int) arithmetic. --- .../scala/spark/ui/exec/ExecutorsUI.scala | 22 +++++++++---------- 1 file changed, 10 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 6ec48f70a45..43e0c20b19a 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -38,10 +38,9 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { def render(request: HttpServletRequest): Seq[Node] = { val storageStatusList = sc.getExecutorStorageStatus - val maxMem = storageStatusList.map(_.maxMem).reduce(_+_) - val memUsed = storageStatusList.map(_.memUsed()).reduce(_+_) - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)) - .reduceOption(_+_).getOrElse(0L) + val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_+_) + val memUsed = storageStatusList.map(_.memUsed()).fold(0L)(_+_) + val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).fold(0L)(_+_) val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used", "Active tasks", "Failed tasks", "Complete tasks", "Total tasks") @@ -93,10 +92,9 @@ 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.get(a.toString).map(l => l.size) - .getOrElse(0).toString - val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0).toString - val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0).toString + val activeTasks = listener.executorToTasksActive.get(a.toString).map(l => l.size).getOrElse(0) + val failedTasks = listener.executorToTasksFailed.getOrElse(a.toString, 0) + val completedTasks = listener.executorToTasksComplete.getOrElse(a.toString, 0) val totalTasks = activeTasks + failedTasks + completedTasks Seq( @@ -106,10 +104,10 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { memUsed, maxMem, diskUsed, - activeTasks, - failedTasks, - completedTasks, - totalTasks + activeTasks.toString, + failedTasks.toString, + completedTasks.toString, + totalTasks.toString ) } From 04ad78b09d195e52d2747c18fe2e3a4640abf838 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 14 Aug 2013 14:55:24 -0700 Subject: [PATCH 0715/2521] Style cleanup based on Matei feedback --- core/src/main/scala/spark/executor/Executor.scala | 4 ++-- .../src/main/scala/spark/scheduler/local/LocalScheduler.scala | 3 +-- core/src/main/scala/spark/ui/jobs/StagePage.scala | 2 +- 3 files changed, 4 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index c5202d94b02..05a960d7c50 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -130,7 +130,7 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert taskStart = System.currentTimeMillis() val value = task.run(taskId.toInt) val taskFinish = System.currentTimeMillis() - task.metrics.foreach{ m => + for (m <- task.metrics) { m.hostname = Utils.localHostName m.executorDeserializeTime = (taskStart - startTime).toInt m.executorRunTime = (taskFinish - taskStart).toInt @@ -158,7 +158,7 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert case t: Throwable => { val serviceTime = (System.currentTimeMillis() - taskStart).toInt val metrics = attemptedTask.flatMap(t => t.metrics) - metrics.foreach {m => + for (m <- metrics) { m.executorRunTime = serviceTime m.jvmGCTime = getTotalGCTime - startGCTime } diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 33e7a10ea47..6c43928bc85 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -34,7 +34,6 @@ import spark.scheduler._ import spark.scheduler.cluster._ import spark.scheduler.cluster.SchedulingMode.SchedulingMode import akka.actor._ -import management.ManagementFactory /** * A FIFO or Fair TaskScheduler implementation that runs tasks locally in a thread pool. Optionally @@ -218,7 +217,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: case t: Throwable => { val serviceTime = System.currentTimeMillis() - taskStart val metrics = attemptedTask.flatMap(t => t.metrics) - metrics.foreach {m => + for (m <- metrics) { m.executorRunTime = serviceTime.toInt m.jvmGCTime = getTotalGCTime - startGCTime } diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 28a6d7b179b..f91a415e370 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -166,7 +166,7 @@ private[spark] class StagePage(parent: JobProgressUI) { Utils.memoryBytesToString(s.shuffleBytesWritten)}.getOrElse("")} }} - {if (gcTime > 0) {parent.formatDuration(gcTime)} else ""} + {if (gcTime > 0) parent.formatDuration(gcTime) else ""} {exception.map(e => From 7a9abb9ddc8461e101771a14ac2e5e07f1741699 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 14 Aug 2013 15:12:12 -0700 Subject: [PATCH 0716/2521] Fix PySpark unit tests on Python 2.6. --- python/pyspark/tests.py | 13 ++++++++----- python/run-tests | 26 ++++++++++++-------------- 2 files changed, 20 insertions(+), 19 deletions(-) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index dfd841b10a0..f75215a7813 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -64,7 +64,7 @@ def test_basic_checkpointing(self): flatMappedRDD = parCollection.flatMap(lambda x: range(1, x + 1)) self.assertFalse(flatMappedRDD.isCheckpointed()) - self.assertIsNone(flatMappedRDD.getCheckpointFile()) + self.assertTrue(flatMappedRDD.getCheckpointFile() is None) flatMappedRDD.checkpoint() result = flatMappedRDD.collect() @@ -79,13 +79,13 @@ def test_checkpoint_and_restore(self): flatMappedRDD = parCollection.flatMap(lambda x: [x]) self.assertFalse(flatMappedRDD.isCheckpointed()) - self.assertIsNone(flatMappedRDD.getCheckpointFile()) + self.assertTrue(flatMappedRDD.getCheckpointFile() is None) flatMappedRDD.checkpoint() flatMappedRDD.count() # forces a checkpoint to be computed time.sleep(1) # 1 second - self.assertIsNotNone(flatMappedRDD.getCheckpointFile()) + self.assertTrue(flatMappedRDD.getCheckpointFile() is not None) recovered = self.sc._checkpointFile(flatMappedRDD.getCheckpointFile()) self.assertEquals([1, 2, 3, 4], recovered.collect()) @@ -164,9 +164,12 @@ def do_termination_test(self, terminator): time.sleep(1) # daemon should no longer accept connections - with self.assertRaises(EnvironmentError) as trap: + try: self.connect(port) - self.assertEqual(trap.exception.errno, ECONNREFUSED) + except EnvironmentError as exception: + self.assertEqual(exception.errno, ECONNREFUSED) + else: + self.fail("Expected EnvironmentError to be raised") def test_termination_stdin(self): """Ensure that daemon and workers terminate when stdin is closed.""" diff --git a/python/run-tests b/python/run-tests index 6643faa2e03..cbc554ea9db 100755 --- a/python/run-tests +++ b/python/run-tests @@ -26,20 +26,18 @@ cd "$FWDIR/python" FAILED=0 -$FWDIR/pyspark pyspark/rdd.py -FAILED=$(($?||$FAILED)) - -$FWDIR/pyspark pyspark/context.py -FAILED=$(($?||$FAILED)) - -$FWDIR/pyspark -m doctest pyspark/broadcast.py -FAILED=$(($?||$FAILED)) - -$FWDIR/pyspark -m doctest pyspark/accumulators.py -FAILED=$(($?||$FAILED)) - -$FWDIR/pyspark -m unittest pyspark.tests -FAILED=$(($?||$FAILED)) +rm -f unit-tests.log + +function run_test() { + $FWDIR/pyspark $1 2>&1 | tee -a unit-tests.log + FAILED=$((PIPESTATUS[0]||$FAILED)) +} + +run_test "pyspark/rdd.py" +run_test "pyspark/context.py" +run_test "-m doctest pyspark/broadcast.py" +run_test "-m doctest pyspark/accumulators.py" +run_test "pyspark/tests.py" if [[ $FAILED != 0 ]]; then echo -en "\033[31m" # Red From b71d4febbc4ed4760a3e2ddeecf9e677b5742635 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 14 Aug 2013 15:25:56 -0700 Subject: [PATCH 0717/2521] Finished early prototype of IndexedRDD --- .../src/main/scala/spark/rdd/IndexedRDD.scala | 272 ++++++++++++++++-- 1 file changed, 251 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index 55ca353618e..0f6e29ad589 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -27,11 +27,12 @@ import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskConte import spark.PairRDDFunctions import spark.SparkContext._ +import spark.SparkException +import spark.Partitioner // import java.io.{ObjectOutputStream, IOException} - /** * An IndexedRDD is an RDD[(K,V)] where each K is unique. * @@ -39,22 +40,238 @@ import spark.SparkContext._ * be used to accelerate join and aggregation operations. */ class IndexedRDD[K: ClassManifest, V: ClassManifest]( - sc: SparkContext, - val indexRDD: RDD[ JHashMap[K, Int] ], - val valuesRDD: RDD[ Array[Seq[V]] ]) - extends RDD[(K, V)](sc, - List(new OneToOneDependency(indexRDD), new OneToOneDependency(valuesRDD)) ) { + val index: RDD[ JHashMap[K, Int] ], + val valuesRDD: RDD[ Seq[Seq[V]] ]) + extends RDD[(K, V)](index.context, + List(new OneToOneDependency(index), new OneToOneDependency(valuesRDD)) ) { - val tuples = new ZippedRDD[ JHashMap[K, Int], Array[Seq[V]] ](sc, indexRDD, valuesRDD) + val tuples = new ZippedRDD[JHashMap[K, Int], Seq[Seq[V]]](index.context, index, valuesRDD) - override val partitioner = indexRDD.partitioner + override val partitioner = index.partitioner override def getPartitions: Array[Partition] = tuples.getPartitions override def getPreferredLocations(s: Partition): Seq[String] = tuples.getPreferredLocations(s) + /** + * Construct a new IndexedRDD that is indexed by only the keys in the RDD + */ + def reindex(): IndexedRDD[K,V] = IndexedRDD(this) + + + /** + * Pass each value in the key-value pair RDD through a map function without changing the keys; + * this also retains the original RDD's partitioning. + */ + def mapValues[U: ClassManifest](f: V => U): IndexedRDD[K, U] = { + val cleanF = index.context.clean(f) + val newValues = valuesRDD.mapPartitions(_.map{ values => + values.map{_.map(x => f(x))} + }, true) + new IndexedRDD[K,U](index, newValues) + } + + + /** + * Pass each value in the key-value pair RDD through a flatMap function without changing the + * keys; this also retains the original RDD's partitioning. + */ + def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): IndexedRDD[K,U] = { + val cleanF = index.context.clean(f) + val newValues = valuesRDD.mapPartitions(_.map{ values => + values.map{_.flatMap(x => f(x))} + }, true) + new IndexedRDD[K,U](index, newValues) + } + + + /** + * Generic function to combine the elements for each key using a custom set of aggregation + * functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C + * Note that V and C can be different -- for example, one might group an RDD of type + * (Int, Int) into an RDD of type (Int, Seq[Int]). Users provide three functions: + * + * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) + * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) + * - `mergeCombiners`, to combine two C's into a single one. + * + * In addition, users can control the partitioning of the output RDD, and whether to perform + * map-side aggregation (if a mapper can produce multiple items with the same key). + */ + def combineByKey[C: ClassManifest](createCombiner: V => C, + mergeValue: (C, V) => C, + mergeCombiners: (C, C) => C, + partitioner: Partitioner, + mapSideCombine: Boolean = true, + serializerClass: String = null): IndexedRDD[K, C] = { + val newValues = valuesRDD.mapPartitions( + _.map{ groups: Seq[Seq[V]] => + groups.map{ group: Seq[V] => + if(group != null && !group.isEmpty) { + val c: C = createCombiner(group.head) + val sum: C = group.tail.foldLeft(c)(mergeValue) + Seq(sum) + } else { + null + } + } + }, true) + new IndexedRDD[K,C](index, newValues) + } + + + + /** + * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the + * list of values for that key in `this` as well as `other`. + */ + def cogroup[W: ClassManifest](other: RDD[(K, W)], partitionerUnused: Partitioner): + IndexedRDD[K, (Seq[V], Seq[W])] = { + other match { + case other: IndexedRDD[_, _] if other.index == index => { + // if both RDDs share exactly the same index and therefore the same super set of keys + // then we simply merge the value RDDs. + // However it is possible that both RDDs are missing a value for a given key in + // which case the returned RDD should have a null value + val newValues = + valuesRDD.zipPartitions[ Seq[Seq[W]], Seq[Seq[(Seq[V], Seq[W])]] ]( + (thisIter, otherIter) => { + val thisValues: Seq[Seq[V]] = thisIter.next() + assert(!thisIter.hasNext()) + val otherValues: Seq[Seq[W]] = otherIter.next() + assert(!otherIter.hasNext()) + // Zip the values and if both arrays are null then the key is not present and + // so the resulting value must be null (not a tuple of empty sequences) + val tmp: Seq[Seq[(Seq[V], Seq[W])]] = thisValues.view.zip(otherValues).map{ + case (null, null) => null // The key is not present in either RDD + case (a, null) => Seq((a, Seq.empty[W])) + case (null, b) => Seq((Seq.empty[V], b)) + case (a, b) => Seq((a,b)) + }.toSeq + List(tmp).iterator + }, other.valuesRDD) + new IndexedRDD[K, (Seq[V], Seq[W])](index, newValues) + } + case other: IndexedRDD[_, _] if other.index.partitioner == index.partitioner => { + // If both RDDs are indexed using different indices but with the same partitioners + // then we we need to first merge the indicies and then use the merged index to + // merge the values. + val newIndex = + index.zipPartitions[JHashMap[K,Int], JHashMap[K,Int]]( (thisIter, otherIter) => { + val thisIndex = thisIter.next() + assert(!thisIter.hasNext()) + val otherIndex = otherIter.next() + assert(!otherIter.hasNext()) + val newIndex = new JHashMap[K, Int]() + // @todo Merge only the keys that correspond to non-null values + // Merge the keys + newIndex.putAll(thisIndex) + newIndex.putAll(otherIndex) + // We need to rekey the index + var ctr = 0 + for(e <- newIndex.entrySet) { + e.setValue(ctr) + ctr += 1 + } + List(newIndex).iterator + }, other.index).cache() + // Use the new index along with the this and the other indices to merge the values + val newValues = + newIndex.zipPartitions[ + (JHashMap[K, Int], Seq[Seq[V]]), + (JHashMap[K, Int], Seq[Seq[W]]), + Seq[Seq[(Seq[V],Seq[W])]] ]( + (newIndexIter, thisTuplesIter, otherTuplesIter) => { + // Get the new index for this partition + val newIndex = newIndexIter.next() + assert(!newIndexIter.hasNext()) + // Get the corresponding indicies and values for this and the other IndexedRDD + val (thisIndex, thisValues) = thisTuplesIter.next() + assert(!thisTuplesIter.hasNext()) + val (otherIndex, otherValues) = otherTuplesIter.next() + assert(!otherTuplesIter.hasNext()) + // Preallocate the new Values array + val newValues = new Array[Seq[(Seq[V],Seq[W])]](newIndex.size) + // Lookup the sequences in both submaps + for((k,ind) <- newIndex) { + val thisSeq = if(thisIndex.contains(k)) thisValues(thisIndex.get(k)) else null + val otherSeq = if(otherIndex.contains(k)) otherValues(otherIndex.get(k)) else null + // if either of the sequences is not null then the key was in one of the two tables + // and so the value should appear in the returned table + newValues(ind) = (thisSeq, otherSeq) match { + case (null, null) => null + case (a, null) => Seq( (a, Seq.empty[W]) ) + case (null, b) => Seq( (Seq.empty[V], b) ) + case (a, b) => Seq( (a,b) ) + } + } + List(newValues.toSeq).iterator + }, tuples, other.tuples) + new IndexedRDD(newIndex, newValues) + } + case _ => { + // Get the partitioner from the index + val partitioner = index.partitioner match { + case Some(p) => p + case None => throw new SparkException("An index must have a partitioner.") + } + // Shuffle the other RDD using the partitioner for this index + val otherShuffled = + if (other.partitioner == Some(partitioner)) { + other + } else { + new ShuffledRDD[K,W](other, partitioner) + } + // Join the other RDD with this RDD building a new valueset and new index on the fly + val groups = tuples.zipPartitions[(K, W), (JHashMap[K, Int], Seq[Seq[(Seq[V],Seq[W])]]) ]( + (thisTuplesIter, otherTuplesIter) => { + // Get the corresponding indicies and values for this IndexedRDD + val (thisIndex, thisValues) = thisTuplesIter.next() + assert(!thisTuplesIter.hasNext()) + // Construct a new index + val newIndex = thisIndex.clone().asInstanceOf[JHashMap[K, Int]] + // Construct a new array Buffer to store the values + val newValues = new ArrayBuffer[(Seq[V], ArrayBuffer[W])](thisValues.size) + // populate the newValues with the values in this IndexedRDD + for((k,i) <- thisIndex) { + if(thisValues(i) != null) { + newValues(i) = (thisValues(i), new ArrayBuffer[W]()) + } + } + // Now iterate through the other tuples updating the map + for((k,w) <- otherTuplesIter){ + if(!newIndex.contains(k)) { + // update the index + val ind = newIndex.size + newIndex.put(k, ind) + // Create the buffer for w + val wbuffer = new ArrayBuffer[W]() + wbuffer.append(w) + // Update the values + newValues.append( (Seq.empty[V], wbuffer) ) + } else { + val ind = newIndex.get(k) + newValues(ind)._2.append(w) + } + } + // Finalize the new values array + val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = + newValues.view.map{ case (s, ab) => Seq((s, ab.toSeq)) }.toSeq + List( (newIndex, newValuesArray) ).iterator + }, otherShuffled).cache() + + // Extract the index and values from the above RDD + val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) + val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) + + new IndexedRDD[K, (Seq[V], Seq[W])](newIndex, newValues) + + } + } + } + @@ -68,12 +285,16 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // Extract rows with key value pairs and indicators .map{ case (k, ind) => (k, values(ind)) } // Remove tuples that aren't actually present in the array - .filter{ case (_, valar) => valar != null } + .filter{ case (_, valar) => valar != null && !valar.isEmpty()} // Extract the pair (removing the indicator from the tuple) .flatMap{ case (k, valar) => valar.map(v => (k,v))} } } -} + +} // End of IndexedRDD + + + object IndexedRDD { def apply[K: ClassManifest, V: ClassManifest]( @@ -90,35 +311,44 @@ object IndexedRDD { indexMap.put(k, ind) values.append(ar) } - List((indexMap, values.toArray)).iterator + List((indexMap, values.toSeq)).iterator }, true).cache // extract the index and the values val index = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) val values = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - new IndexedRDD[K,V](tbl.context, index, values) + new IndexedRDD[K,V](index, values) } else { val index = existingIndex + val partitioner = index.partitioner match { + case Some(p) => p + case None => throw new SparkException("An index must have a partitioner.") + } + // Shuffle the table according to the index (if necessary) val shuffledTbl = - if (tbl.partitioner == Some(index.partitioner)) { + if (tbl.partitioner == Some(partitioner)) { tbl } else { - new ShuffledRDD[K,V](tbl, index.partitioner.get()) + new ShuffledRDD[K,V](tbl, partitioner) } // Use the index to build the new values table - val values = index.zipPartitions[ (K, Seq[V]), Array[Seq[V]] ]( + val values = index.zipPartitions[ (K, V), Seq[Seq[V]] ]( (indexIter, tblIter) => { // There is only one map - val index: JHashMap[K,Int] = iter.next() - assert(!iter.hasNext()) + val index: JHashMap[K,Int] = indexIter.next() + assert(!indexIter.hasNext()) val values = new Array[Seq[V]](index.size) - for((k,a) <- tblIter) { + for((k,v) <- tblIter) { assert(index.contains(k)) - values(index.get(k)) = a + val ind = index(k) + if(values(ind) == null){ + values(ind) = new ArrayBuffer[V]() + } + values(ind).asInstanceOf[ArrayBuffer[V]].append(v) } - values - }, shuffleTbl) + List(values.toSeq).iterator + }, shuffledTbl) new IndexedRDD[K,V](index, values) } From 54b54903c316096b0ef1cda48670d32adeb746e1 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 14 Aug 2013 16:35:20 -0700 Subject: [PATCH 0718/2521] Adding testing code for indexedrdd --- .../src/main/scala/spark/rdd/IndexedRDD.scala | 24 +- .../test/scala/spark/IndexedRDDSuite.scala | 354 ++++++++++++++++++ 2 files changed, 372 insertions(+), 6 deletions(-) create mode 100644 core/src/test/scala/spark/IndexedRDDSuite.scala diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index 0f6e29ad589..a6852f3f8a6 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -44,6 +44,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val valuesRDD: RDD[ Seq[Seq[V]] ]) extends RDD[(K, V)](index.context, List(new OneToOneDependency(index), new OneToOneDependency(valuesRDD)) ) { + //with PairRDDFunctions[K,V] { @@ -129,6 +130,8 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( */ def cogroup[W: ClassManifest](other: RDD[(K, W)], partitionerUnused: Partitioner): IndexedRDD[K, (Seq[V], Seq[W])] = { + //RDD[(K, (Seq[V], Seq[W]))] = { + assert(false) other match { case other: IndexedRDD[_, _] if other.index == index => { // if both RDDs share exactly the same index and therefore the same super set of keys @@ -302,14 +305,23 @@ object IndexedRDD { existingIndex: RDD[JHashMap[K,Int]] = null ): IndexedRDD[K, V] = { if(existingIndex == null) { - // build th index - val groups = tbl.groupByKey().mapPartitions( iter => { + // Shuffle the table (if necessary) + val shuffledTbl = + if (tbl.partitioner.isEmpty) { + new ShuffledRDD[K,V](tbl, Partitioner.defaultPartitioner(tbl)) + } else { tbl } + + val groups = shuffledTbl.mapPartitions( iter => { val indexMap = new JHashMap[K, Int]() val values = new ArrayBuffer[Seq[V]]() - for((k,ar) <- iter){ - val ind = values.size - indexMap.put(k, ind) - values.append(ar) + for((k,v) <- iter){ + if(!indexMap.contains(k)) { + val ind = indexMap.size + indexMap.put(k, ind) + values.append(new ArrayBuffer[V]()) + } + val ind = indexMap.get(k) + values(ind).asInstanceOf[ArrayBuffer[V]].append(v) } List((indexMap, values.toSeq)).iterator }, true).cache diff --git a/core/src/test/scala/spark/IndexedRDDSuite.scala b/core/src/test/scala/spark/IndexedRDDSuite.scala new file mode 100644 index 00000000000..aacb6423ee1 --- /dev/null +++ b/core/src/test/scala/spark/IndexedRDDSuite.scala @@ -0,0 +1,354 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR 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 org.scalatest.prop.Checkers +import org.scalacheck.Arbitrary._ +import org.scalacheck.Gen +import org.scalacheck.Prop._ + +import com.google.common.io.Files + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashSet + +import spark.rdd.ShuffledRDD +import spark.SparkContext._ + +class IndexedRDDSuite extends FunSuite with SharedSparkContext { + + def lineage(rdd: RDD[_]): collection.mutable.HashSet[RDD[_]] = { + val set = new collection.mutable.HashSet[RDD[_]] + def visit(rdd: RDD[_]) { + for (dep <- rdd.dependencies) { + set += dep.rdd + visit(dep.rdd) + } + } + visit(rdd) + set + } + + test("groupByKey") { + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).index() + 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))).index() + 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))).index() + 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))).index() + 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))).index() + 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))).index() + 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))).index() + 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).index() + 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 = lineage(sums) + + assert(deps.filter(_.isInstanceOf[ShuffledRDD[_,_]]).size === 1) // 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("joinIndexVsPair") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() + 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("joinIndexVsIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index() + 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("joinSharedIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4,-4), (4, 4) )).index() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index(rdd1.index) + val joined = rdd1.join(rdd2).collect() + assert(joined.toSet === Set( + (1, (1, 'x')), + (1, (2, 'x')), + (2, (1, 'y')), + (2, (1, 'z')), + (4, (-4, 'w')), + (4, (4, 'w')) + )) + } + + + 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)))) + } +} From 3886b5493341b2abdf12d4d8052145399a6e590a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 14 Aug 2013 17:19:42 -0700 Subject: [PATCH 0719/2521] A few small scheduler / job description changes. 1. Renamed SparkContext.addLocalProperty to setLocalProperty. And allow this function to unset a property. 2. Renamed SparkContext.setDescription to setCurrentJobDescription. 3. Throw an exception if the fair scheduler allocation file is invalid. --- core/src/main/scala/spark/SparkContext.scala | 14 ++-- .../cluster/SchedulableBuilder.scala | 78 +++++++++---------- .../scala/spark/ui/UIWorkloadGenerator.scala | 4 +- .../spark/scheduler/LocalSchedulerSuite.scala | 48 ++++++------ 4 files changed, 74 insertions(+), 70 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 1069e27513a..a9851c17226 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -267,16 +267,20 @@ class SparkContext( localProperties.value = new Properties() } - def addLocalProperty(key: String, value: String) { - if(localProperties.value == null) { + def setLocalProperty(key: String, value: String) { + if (localProperties.value == null) { localProperties.value = new Properties() } - localProperties.value.setProperty(key,value) + if (value == null) { + localProperties.value.remove(key) + } else { + localProperties.value.setProperty(key, value) + } } /** Set a human readable description of the current job. */ - def setDescription(value: String) { - addLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, value) + def setCurrentJobDescription(value: String) { + setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, value) } // Post init diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala index b2d089f31d9..2fc8a76a057 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala @@ -17,19 +17,14 @@ package spark.scheduler.cluster -import java.io.{File, FileInputStream, FileOutputStream} +import java.io.{File, FileInputStream, FileOutputStream, FileNotFoundException} +import java.util.Properties -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 scala.xml.XML import spark.Logging import spark.scheduler.cluster.SchedulingMode.SchedulingMode -import java.util.Properties /** * An interface to build Schedulable tree @@ -56,7 +51,7 @@ private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) private[spark] class FairSchedulableBuilder(val rootPool: Pool) extends SchedulableBuilder with Logging { - val schedulerAllocFile = System.getProperty("spark.fairscheduler.allocation.file","unspecified") + val schedulerAllocFile = System.getProperty("spark.fairscheduler.allocation.file") val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.cluster.fair.pool" val DEFAULT_POOL_NAME = "default" val MINIMUM_SHARES_PROPERTY = "minShare" @@ -69,39 +64,44 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool) val DEFAULT_WEIGHT = 1 override def buildPools() { + if (schedulerAllocFile != null) { 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") + 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 xmlMinShare = (poolNode \ MINIMUM_SHARES_PROPERTY).text + if (xmlMinShare != "") { + minShare = xmlMinShare.toInt + } - val xmlWeight = (poolNode \ WEIGHT_PROPERTY).text - if (xmlWeight != "") { - weight = xmlWeight.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)) + val pool = new Pool(poolName, schedulingMode, minShare, weight) + rootPool.addSchedulable(pool) + logInfo("Created pool %s, schedulingMode: %s, minShare: %d, weight: %d".format( + poolName, schedulingMode, minShare, weight)) + } + } else { + throw new java.io.FileNotFoundException( + "Fair scheduler allocation file not found: " + schedulerAllocFile) } } @@ -110,7 +110,7 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool) 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( + logInfo("Created default pool %s, schedulingMode: %s, minShare: %d, weight: %d".format( DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT)) } } @@ -127,7 +127,7 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool) 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( + logInfo("Created pool %s, schedulingMode: %s, minShare: %d, weight: %d".format( poolName, DEFAULT_SCHEDULING_MODE, DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT)) } } diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala index 97ea644021f..0dfb1a064cc 100644 --- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala @@ -48,9 +48,9 @@ private[spark] object UIWorkloadGenerator { def setProperties(s: String) = { if(schedulingMode == SchedulingMode.FAIR) { - sc.addLocalProperty("spark.scheduler.cluster.fair.pool", s) + sc.setLocalProperty("spark.scheduler.cluster.fair.pool", s) } - sc.addLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, s) + sc.setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, s) } val baseData = sc.makeRDD(1 to NUM_PARTITIONS * 10, NUM_PARTITIONS) diff --git a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala index 66fd59e8bbe..a79b8bf2560 100644 --- a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala @@ -57,23 +57,23 @@ object TaskThreadInfo { * 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, + * 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) { - + 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.addLocalProperty("spark.scheduler.cluster.fair.pool", poolName) + sc.setLocalProperty("spark.scheduler.cluster.fair.pool", poolName) } override def run() { val ans = nums.map(number => { @@ -88,7 +88,7 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { } }.start() } - + test("Local FIFO scheduler end-to-end test") { System.setProperty("spark.cluster.schedulingmode", "FIFO") sc = new SparkContext("local[4]", "test") @@ -103,8 +103,8 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { 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() + // 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? @@ -112,24 +112,24 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { 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) assert(TaskThreadInfo.threadToRunning(4) === true) assert(TaskThreadInfo.threadToRunning(5) === false) assert(TaskThreadInfo.threadToRunning(6) === false) - + TaskThreadInfo.threadToLock(1).jobFinished() TaskThreadInfo.threadToStarted(5).await() - + 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() TaskThreadInfo.threadToStarted(6).await() @@ -139,7 +139,7 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { 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() @@ -160,18 +160,18 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { 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) @@ -185,19 +185,19 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { assert(TaskThreadInfo.threadToRunning(12) === true) assert(TaskThreadInfo.threadToRunning(22) === true) assert(TaskThreadInfo.threadToRunning(32) === false) - + TaskThreadInfo.threadToLock(10).jobFinished() 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 + //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() TaskThreadInfo.threadToStarted(23).await() @@ -206,7 +206,7 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { TaskThreadInfo.threadToLock(12).jobFinished() TaskThreadInfo.threadToStarted(33).await() - + assert(TaskThreadInfo.threadToRunning(33) === true) TaskThreadInfo.threadToLock(20).jobFinished() @@ -217,7 +217,7 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext { TaskThreadInfo.threadToLock(31).jobFinished() TaskThreadInfo.threadToLock(32).jobFinished() TaskThreadInfo.threadToLock(33).jobFinished() - - sem.acquire(11) + + sem.acquire(11) } } From 290e3e6e65f04f865334224718c3ac7ed650b101 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 14 Aug 2013 18:40:53 -0700 Subject: [PATCH 0720/2521] Renamed setCurrentJobDescription to setJobDescription. --- 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 a9851c17226..80c65dfebd2 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -279,7 +279,7 @@ class SparkContext( } /** Set a human readable description of the current job. */ - def setCurrentJobDescription(value: String) { + def setJobDescription(value: String) { setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, value) } From 2d2a556bdf8049d0b567533c26611db2388621d5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 14 Aug 2013 23:23:09 -0700 Subject: [PATCH 0721/2521] Various UI improvements. --- .../main/resources/spark/ui/static/webui.css | 4 --- .../spark/deploy/master/ui/IndexPage.scala | 2 +- core/src/main/scala/spark/ui/JettyUtils.scala | 10 +++--- .../scala/spark/ui/UIWorkloadGenerator.scala | 3 +- .../scala/spark/ui/env/EnvironmentUI.scala | 27 ++++++++------- .../scala/spark/ui/exec/ExecutorsUI.scala | 26 ++++++-------- .../main/scala/spark/ui/jobs/IndexPage.scala | 13 +++---- .../main/scala/spark/ui/jobs/PoolPage.scala | 7 ++-- .../main/scala/spark/ui/jobs/PoolTable.scala | 11 +++--- .../main/scala/spark/ui/jobs/StagePage.scala | 34 +++++++++++-------- .../main/scala/spark/ui/jobs/StageTable.scala | 21 ++++-------- .../main/scala/spark/ui/storage/RDDPage.scala | 13 +++---- 12 files changed, 83 insertions(+), 88 deletions(-) diff --git a/core/src/main/resources/spark/ui/static/webui.css b/core/src/main/resources/spark/ui/static/webui.css index b83f4109c03..70fd0ba31eb 100644 --- a/core/src/main/resources/spark/ui/static/webui.css +++ b/core/src/main/resources/spark/ui/static/webui.css @@ -5,10 +5,6 @@ padding: 0; } -body { - font-size: 15px !important; -} - .version { line-height: 30px; vertical-align: bottom; 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 4443d880560..60fbcbfad63 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -53,7 +53,7 @@ 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", + val appHeaders = Seq("ID", "Name", "Cores", "Memory per Node", "Launch Time", "User", "State", "Duration") val activeApps = state.activeApps.sortBy(_.startTime).reverse val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps) diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index 1cc85124d3a..f66fe399052 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -17,21 +17,21 @@ package spark.ui -import annotation.tailrec - import javax.servlet.http.{HttpServletResponse, HttpServletRequest} +import scala.annotation.tailrec +import scala.util.{Try, Success, Failure} +import scala.xml.Node + 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.Logging + /** 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/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala index 0dfb1a064cc..f96419520fb 100644 --- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala @@ -22,7 +22,8 @@ import scala.util.Random import spark.SparkContext import spark.SparkContext._ import spark.scheduler.cluster.SchedulingMode -import spark.scheduler.cluster.SchedulingMode.SchedulingMode + + /** * Continuously generates jobs that expose various features of the WebUI (internal testing tool). * diff --git a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala index dc39b91648e..6ee58cda2d7 100644 --- a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala @@ -19,18 +19,17 @@ package spark.ui.env import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.Handler - import scala.collection.JavaConversions._ import scala.util.Properties +import scala.xml.Node + +import org.eclipse.jetty.server.Handler import spark.ui.JettyUtils._ -import spark.ui.UIUtils.headerSparkPage +import spark.ui.UIUtils import spark.ui.Page.Environment import spark.SparkContext -import spark.ui.UIUtils -import scala.xml.Node private[spark] class EnvironmentUI(sc: SparkContext) { @@ -49,10 +48,9 @@ private[spark] class EnvironmentUI(sc: SparkContext) { 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 classPathProperty = properties.find { case (k, v) => + k.contains("java.class.path") + }.getOrElse(("", "")) val sparkProperties = properties.filter(_._1.startsWith("spark")).sorted val otherProperties = properties.diff(sparkProperties :+ classPathProperty).sorted @@ -76,11 +74,14 @@ private[spark] class EnvironmentUI(sc: SparkContext) { val content =

    Runtime Information

    {jvmTable} -

    Spark Properties

    {sparkPropertyTable} -

    System Properties

    {otherPropertyTable} -

    Classpath Entries

    {classPathTable} +
    +

    {sparkProperties.size} Spark Properties

    {sparkPropertyTable} +
    +

    {otherProperties.size} System Properties

    {otherPropertyTable} +
    +

    {classPath.size} Classpath Entries

    {classPathTable}
    - headerSparkPage(content, sc, "Environment", Environment) + UIUtils.headerSparkPage(content, sc, "Environment", Environment) } } diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 43e0c20b19a..28f6b3211c6 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -1,25 +1,20 @@ package spark.ui.exec - import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.Handler +import scala.collection.mutable.{HashMap, HashSet} +import scala.xml.Node -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.util.Properties +import org.eclipse.jetty.server.Handler -import spark.{ExceptionFailure, Logging, SparkContext, Success, Utils} +import spark.{ExceptionFailure, Logging, Utils, SparkContext} import spark.executor.TaskMetrics import spark.scheduler.cluster.TaskInfo -import spark.scheduler._ -import spark.SparkContext -import spark.storage.{StorageStatus, StorageUtils} +import spark.scheduler.{SparkListenerTaskStart, SparkListenerTaskEnd, SparkListener} import spark.ui.JettyUtils._ import spark.ui.Page.Executors -import spark.ui.UIUtils.headerSparkPage import spark.ui.UIUtils -import scala.xml.{Node, XML} private[spark] class ExecutorsUI(val sc: SparkContext) { @@ -44,7 +39,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used", "Active tasks", "Failed tasks", "Complete tasks", "Total tasks") - def execRow(kv: Seq[String]) = + + def execRow(kv: Seq[String]) = { {kv(0)} {kv(1)} @@ -60,9 +56,9 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { {kv(8)} {kv(9)} - val execInfo = - for (b <- 0 until storageStatusList.size) - yield getExecInfo(b) + } + + val execInfo = for (b <- 0 until storageStatusList.size) yield getExecInfo(b) val execTable = UIUtils.listingTable(execHead, execRow, execInfo) val content = @@ -82,7 +78,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { ; - headerSparkPage(content, sc, "Executors", Executors) + UIUtils.headerSparkPage(content, sc, execInfo.size + " Executors", Executors) } def getExecInfo(a: Int): Seq[String] = { diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index 9724671a032..cda6addd22a 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -24,7 +24,7 @@ import scala.xml.{NodeSeq, Node} import spark.scheduler.cluster.SchedulingMode import spark.ui.Page._ import spark.ui.UIUtils._ -import spark.Utils + /** Page showing list of all ongoing and recently finished stages and pools*/ private[spark] class IndexPage(parent: JobProgressUI) { @@ -46,7 +46,8 @@ private[spark] class IndexPage(parent: JobProgressUI) { val completedStagesTable = new StageTable(completedStages.sortBy(_.submissionTime).reverse, parent) val failedStagesTable = new StageTable(failedStages.sortBy(_.submissionTime).reverse, parent) - val poolTable = new PoolTable(listener.sc.getAllPools, listener) + val pools = listener.sc.getAllPools + val poolTable = new PoolTable(pools, listener) val summary: NodeSeq =
      @@ -76,15 +77,15 @@ private[spark] class IndexPage(parent: JobProgressUI) { val content = summary ++ {if (listener.sc.getSchedulingMode == SchedulingMode.FAIR) { -

      Pools

      ++ poolTable.toNodeSeq +

      {pools.size} Fair Scheduler Pools

      ++ poolTable.toNodeSeq } else { Seq() }} ++ -

      Active Stages: {activeStages.size}

      ++ +

      {activeStages.size} Active Stages

      ++ activeStagesTable.toNodeSeq++ -

      Completed Stages: {completedStages.size}

      ++ +

      {completedStages.size} Completed Stages

      ++ completedStagesTable.toNodeSeq++ -

      Failed Stages: {failedStages.size}

      ++ +

      {failedStages.size} Failed Stages

      ++ failedStagesTable.toNodeSeq headerSparkPage(content, parent.sc, "Spark Stages", Jobs) diff --git a/core/src/main/scala/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/spark/ui/jobs/PoolPage.scala index 04ef35c8008..e8f80ebfce7 100644 --- a/core/src/main/scala/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/spark/ui/jobs/PoolPage.scala @@ -23,10 +23,11 @@ private[spark] class PoolPage(parent: JobProgressUI) { val pool = listener.sc.getPoolForName(poolName).get val poolTable = new PoolTable(Seq(pool), listener) - val content =

      Pool

      ++ poolTable.toNodeSeq() ++ -

      Active Stages : {activeStages.size}

      ++ activeStagesTable.toNodeSeq() + val content =

      Summary

      ++ poolTable.toNodeSeq() ++ +
      +

      {activeStages.size} Active Stages

      ++ activeStagesTable.toNodeSeq() - headerSparkPage(content, parent.sc, "Spark Pool Details", Jobs) + headerSparkPage(content, parent.sc, "Fair Scheduler Pool: " + poolName, Jobs) } } } diff --git a/core/src/main/scala/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/spark/ui/jobs/PoolTable.scala index 21ebcef63aa..12fb5f0b89d 100644 --- a/core/src/main/scala/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/spark/ui/jobs/PoolTable.scala @@ -1,8 +1,8 @@ package spark.ui.jobs -import scala.xml.Node import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet +import scala.xml.Node import spark.scheduler.Stage import spark.scheduler.cluster.Schedulable @@ -26,9 +26,9 @@ private[spark] class PoolTable(pools: Seq[Schedulable], listener: JobProgressLis Pool Name Minimum Share Pool Weight - Active Stages - Running Tasks - SchedulingMode + Active Stages + Running Tasks + SchedulingMode {rows.map(r => makeRow(r, poolToActiveStages))} @@ -36,7 +36,8 @@ private[spark] class PoolTable(pools: Seq[Schedulable], listener: JobProgressLis } - private def poolRow(p: Schedulable, poolToActiveStages: HashMap[String, HashSet[Stage]]): Seq[Node] = { + private def poolRow(p: Schedulable, poolToActiveStages: HashMap[String, HashSet[Stage]]) + : Seq[Node] = { val activeStages = poolToActiveStages.get(p.name) match { case Some(stages) => stages.size case None => 0 diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index f91a415e370..8e2458f94ba 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -46,11 +46,12 @@ private[spark] class StagePage(parent: JobProgressUI) {

      Summary Metrics

      No tasks have started yet

      Tasks

      No tasks have started yet
    - return headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId), Jobs) + return headerSparkPage(content, parent.sc, "Details for Stage %s".format(stageId), Jobs) } val tasks = listener.stageToTaskInfos(stageId).toSeq.sortBy(_._1.launchTime) + val numCompleted = tasks.count(_._1.finished) val shuffleReadBytes = listener.stageToShuffleRead.getOrElse(stageId, 0L) val hasShuffleRead = shuffleReadBytes > 0 val shuffleWriteBytes = listener.stageToShuffleWrite.getOrElse(stageId, 0L) @@ -82,11 +83,11 @@ private[spark] class StagePage(parent: JobProgressUI) { val taskHeaders: Seq[String] = - Seq("Task ID", "Status", "Duration", "Locality Level", "Worker", "Launch Time") ++ - {if (hasShuffleRead) Seq("Shuffle Read") else Nil} ++ - {if (hasShuffleWrite) Seq("Shuffle Write") else Nil} ++ + Seq("Task ID", "Status", "Locality Level", "Executor", "Launch Time", "Duration") ++ Seq("GC Time") ++ - Seq("Details") + {if (hasShuffleRead) Seq("Shuffle Read") else Nil} ++ + {if (hasShuffleWrite) Seq("Shuffle Write") else Nil} ++ + Seq("Errors") val taskTable = listingTable(taskHeaders, taskRow(hasShuffleRead, hasShuffleWrite), tasks) @@ -122,16 +123,19 @@ private[spark] class StagePage(parent: JobProgressUI) { if (hasShuffleRead) shuffleReadQuantiles else Nil, if (hasShuffleWrite) shuffleWriteQuantiles else Nil) - val quantileHeaders = Seq("Metric", "Min", "25%", "50%", "75%", "Max") + val quantileHeaders = Seq("Metric", "Min (0th percentitle)", "25th percentile", + "50th percentile", "75th percentile", "Max (100th percentile)") def quantileRow(data: Seq[String]): Seq[Node] = {data.map(d => {d})} Some(listingTable(quantileHeaders, quantileRow, listings)) } val content = - summary ++

    Summary Metrics

    ++ summaryTable.getOrElse(Nil) ++ -

    Tasks

    ++ taskTable; + summary ++ +

    Summary Metrics for {numCompleted} Completed Tasks

    ++ +
    {summaryTable.getOrElse("No tasks have reported their execution metrics yet.")}
    ++ +

    Tasks

    ++ taskTable; - headerSparkPage(content, parent.sc, "Stage Details: %s".format(stageId), Jobs) + headerSparkPage(content, parent.sc, "Details for Stage %d".format(stageId), Jobs) } } @@ -151,12 +155,15 @@ private[spark] class StagePage(parent: JobProgressUI) { {info.taskId} {info.status} - - {formatDuration} - {info.taskLocality} {info.hostPort} {dateFmt.format(new Date(info.launchTime))} + + {formatDuration} + + + {if (gcTime > 0) parent.formatDuration(gcTime) else ""} + {if (shuffleRead) { {metrics.flatMap{m => m.shuffleReadMetrics}.map{s => Utils.memoryBytesToString(s.remoteBytesRead)}.getOrElse("")} @@ -165,9 +172,6 @@ private[spark] class StagePage(parent: JobProgressUI) { {metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => Utils.memoryBytesToString(s.shuffleBytesWritten)}.getOrElse("")} }} - - {if (gcTime > 0) parent.formatDuration(gcTime) else ""} - {exception.map(e => {e.className} ({e.description})
    diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/spark/ui/jobs/StageTable.scala index 19b07ccedac..96bcc62480c 100644 --- a/core/src/main/scala/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/spark/ui/jobs/StageTable.scala @@ -1,21 +1,14 @@ package spark.ui.jobs import java.util.Date -import java.text.SimpleDateFormat -import javax.servlet.http.HttpServletRequest - -import scala.Some -import scala.xml.{NodeSeq, Node} -import scala.collection.mutable.HashMap +import scala.xml.Node import scala.collection.mutable.HashSet +import spark.Utils import spark.scheduler.cluster.{SchedulingMode, TaskInfo} import spark.scheduler.Stage -import spark.ui.UIUtils._ -import spark.ui.Page._ -import spark.Utils -import spark.storage.StorageLevel + /** Page showing list of all ongoing and recently finished stages */ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressUI) { @@ -38,10 +31,10 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU {if (isFairScheduler) {Pool Name} else {}} Description Submitted - Duration - Tasks: Succeeded/Total - Shuffle Read - Shuffle Write + Duration + Tasks: Succeeded/Total + Shuffle Read + Shuffle Write {rows.map(r => makeRow(r))} diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 40f94b42a6f..5fce1ea59be 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -21,12 +21,13 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import spark.storage.{StorageStatus, StorageUtils} -import spark.ui.UIUtils._ import spark.Utils +import spark.storage.{StorageStatus, StorageUtils} import spark.storage.BlockManagerMasterActor.BlockStatus +import spark.ui.UIUtils._ import spark.ui.Page._ + /** Page showing storage details for a given RDD */ private[spark] class RDDPage(parent: BlockManagerUI) { val sc = parent.sc @@ -44,7 +45,7 @@ private[spark] class RDDPage(parent: BlockManagerUI) { val workerTable = listingTable(workerHeaders, workerRow, workers) val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk", - "Locations") + "Executors") val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1) val blockLocations = StorageUtils.blockLocationsFromStorageStatus(filteredStorageStatusList) @@ -83,19 +84,19 @@ private[spark] class RDDPage(parent: BlockManagerUI) {
    -

    Data Distribution Summary

    +

    Data Distribution on {workers.size} Executors

    {workerTable}

    -

    Partitions

    +

    {blocks.size} Partitions

    {blockTable}
    ; - headerSparkPage(content, parent.sc, "RDD Info: " + rddInfo.name, Storage) + headerSparkPage(content, parent.sc, "RDD Storage Info for " + rddInfo.name, Storage) } def blockRow(row: (String, BlockStatus, Seq[String])): Seq[Node] = { From ad2e8b512654314d68f4b7c149950756943992be Mon Sep 17 00:00:00 2001 From: Daemoen Date: Thu, 15 Aug 2013 12:19:14 -0700 Subject: [PATCH 0722/2521] Updated json output to allow for display of worker state Ops teams need to ensure that the cluster is functional and performant. Having to scrape the html source for worker state won't work reliably, and will be slow. By exposing the state in the json output, ops teams are able to ensure a fully functional environment by querying for the json output and parsing for dead nodes. --- core/src/main/scala/spark/deploy/JsonProtocol.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index bd1db7c2940..9901bb8a8cf 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -33,7 +33,8 @@ private[spark] object JsonProtocol { ("cores" -> obj.cores) ~ ("coresused" -> obj.coresUsed) ~ ("memory" -> obj.memory) ~ - ("memoryused" -> obj.memoryUsed) + ("memoryused" -> obj.memoryUsed) ~ + ("state" -> obj.getState) } def writeApplicationInfo(obj: ApplicationInfo) = { From 61281756f225902386e8f41d12a3c58f06da3977 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 15 Aug 2013 14:20:59 -0700 Subject: [PATCH 0723/2521] IndexedRDD passes all PairRDD Function tests --- .../src/main/scala/spark/rdd/IndexedRDD.scala | 337 ++++++++++++++++-- .../test/scala/spark/IndexedRDDSuite.scala | 40 ++- 2 files changed, 338 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index a6852f3f8a6..b65efa4447b 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -17,18 +17,21 @@ package spark.rdd +import java.nio.ByteBuffer + + import java.util.{HashMap => JHashMap, BitSet => JBitSet, HashSet => JHashSet} import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer -import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} - -import spark.PairRDDFunctions +import spark._ +import spark.rdd._ import spark.SparkContext._ -import spark.SparkException -import spark.Partitioner +import spark.Partitioner._ + + // import java.io.{ObjectOutputStream, IOException} @@ -110,7 +113,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val newValues = valuesRDD.mapPartitions( _.map{ groups: Seq[Seq[V]] => groups.map{ group: Seq[V] => - if(group != null && !group.isEmpty) { + if (group != null && !group.isEmpty) { val c: C = createCombiner(group.head) val sum: C = group.tail.foldLeft(c)(mergeValue) Seq(sum) @@ -123,15 +126,129 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } + /** + * Simplified version of combineByKey that hash-partitions the output RDD. + */ + def combineByKey[C: ClassManifest](createCombiner: V => C, + mergeValue: (C, V) => C, + mergeCombiners: (C, C) => C, + numPartitions: Int): IndexedRDD[K, C] = { + combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions)) + } + + + /** + * Simplified version of combineByKey that hash-partitions the resulting RDD using the + * existing partitioner/parallelism level. + */ + def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, + mergeCombiners: (C, C) => C) + : IndexedRDD[K, C] = { + combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(index)) + } + + + /** + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). + */ + def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): IndexedRDD[K, V] = { + // 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) + } + + /** + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). + */ + def foldByKey(zeroValue: V, numPartitions: Int)(func: (V, V) => V): IndexedRDD[K, V] = { + foldByKey(zeroValue, new HashPartitioner(numPartitions))(func) + } + + + /** + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). + */ + def foldByKey(zeroValue: V)(func: (V, V) => V): IndexedRDD[K, V] = { + foldByKey(zeroValue, defaultPartitioner(index))(func) + } + + + /** + * Merge the values for each key using an associative reduce function. This will also perform + * the merging locally on each mapper before sending results to a reducer, similarly to a + * "combiner" in MapReduce. + */ + def reduceByKey(partitioner: Partitioner, func: (V, V) => V): IndexedRDD[K, V] = { + combineByKey[V]((v: V) => v, func, func, partitioner) + } + + /** + * Merge the values for each key using an associative reduce function. This will also perform + * the merging locally on each mapper before sending results to a reducer, similarly to a + * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions. + */ + def reduceByKey(func: (V, V) => V, numPartitions: Int): IndexedRDD[K, V] = { + reduceByKey(new HashPartitioner(numPartitions), func) + } + + /** + * Merge the values for each key using an associative reduce function. This will also perform + * the merging locally on each mapper before sending results to a reducer, similarly to a + * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/ + * parallelism level. + */ + def reduceByKey(func: (V, V) => V): IndexedRDD[K, V] = { + reduceByKey(defaultPartitioner(index), func) + } + + + /** + * Group the values for each key in the RDD into a single sequence. Allows controlling the + * partitioning of the resulting key-value pair RDD by passing a Partitioner. + */ + def groupByKey(partitioner: Partitioner): IndexedRDD[K, Seq[V]] = { + val newValues = valuesRDD.mapPartitions(_.map{ar => ar.map{s => Seq(s)} }, true) + new IndexedRDD[K, Seq[V]](index, newValues) + } + + + /** + * Group the values for each key in the RDD into a single sequence. Hash-partitions the + * resulting RDD with into `numPartitions` partitions. + */ + def groupByKey(numPartitions: Int): IndexedRDD[K, Seq[V]] = { + groupByKey(new HashPartitioner(numPartitions)) + } + + + /** + * Group the values for each key in the RDD into a single sequence. Hash-partitions the + * resulting RDD with the existing partitioner/parallelism level. + */ + def groupByKey(): IndexedRDD[K, Seq[V]] = { + groupByKey(defaultPartitioner(index)) + } + /** * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W: ClassManifest](other: RDD[(K, W)], partitionerUnused: Partitioner): - IndexedRDD[K, (Seq[V], Seq[W])] = { + def cogroup[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): + IndexedRDD[K, (Seq[V], Seq[W])] = { //RDD[(K, (Seq[V], Seq[W]))] = { - assert(false) other match { case other: IndexedRDD[_, _] if other.index == index => { // if both RDDs share exactly the same index and therefore the same super set of keys @@ -174,7 +291,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( newIndex.putAll(otherIndex) // We need to rekey the index var ctr = 0 - for(e <- newIndex.entrySet) { + for (e <- newIndex.entrySet) { e.setValue(ctr) ctr += 1 } @@ -198,9 +315,9 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // Preallocate the new Values array val newValues = new Array[Seq[(Seq[V],Seq[W])]](newIndex.size) // Lookup the sequences in both submaps - for((k,ind) <- newIndex) { - val thisSeq = if(thisIndex.contains(k)) thisValues(thisIndex.get(k)) else null - val otherSeq = if(otherIndex.contains(k)) otherValues(otherIndex.get(k)) else null + for ((k,ind) <- newIndex) { + val thisSeq = if (thisIndex.contains(k)) thisValues(thisIndex.get(k)) else null + val otherSeq = if (otherIndex.contains(k)) otherValues(otherIndex.get(k)) else null // if either of the sequences is not null then the key was in one of the two tables // and so the value should appear in the returned table newValues(ind) = (thisSeq, otherSeq) match { @@ -233,27 +350,27 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // Get the corresponding indicies and values for this IndexedRDD val (thisIndex, thisValues) = thisTuplesIter.next() assert(!thisTuplesIter.hasNext()) + assert(thisIndex.size == thisValues.size) // Construct a new index val newIndex = thisIndex.clone().asInstanceOf[JHashMap[K, Int]] + assert(thisIndex.size == newIndex.size) // Construct a new array Buffer to store the values - val newValues = new ArrayBuffer[(Seq[V], ArrayBuffer[W])](thisValues.size) + val newValues = ArrayBuffer.fill[(Seq[V], ArrayBuffer[W])](thisValues.size)(null) // populate the newValues with the values in this IndexedRDD - for((k,i) <- thisIndex) { - if(thisValues(i) != null) { - newValues(i) = (thisValues(i), new ArrayBuffer[W]()) + for ((k,i) <- thisIndex) { + assert(i < thisValues.size) + if (thisValues(i) != null) { + newValues(i) = (thisValues(i), ArrayBuffer.empty[W]) } } // Now iterate through the other tuples updating the map - for((k,w) <- otherTuplesIter){ - if(!newIndex.contains(k)) { + for ((k,w) <- otherTuplesIter){ + if (!newIndex.contains(k)) { // update the index val ind = newIndex.size newIndex.put(k, ind) - // Create the buffer for w - val wbuffer = new ArrayBuffer[W]() - wbuffer.append(w) // Update the values - newValues.append( (Seq.empty[V], wbuffer) ) + newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) } else { val ind = newIndex.get(k) newValues(ind)._2.append(w) @@ -265,17 +382,167 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( List( (newIndex, newValuesArray) ).iterator }, otherShuffled).cache() - // Extract the index and values from the above RDD - val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) - val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) + // Extract the index and values from the above RDD + val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) + val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - new IndexedRDD[K, (Seq[V], Seq[W])](newIndex, newValues) - - } + new IndexedRDD[K, (Seq[V], Seq[W])](newIndex, newValues) } } + } + /** + * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the + * list of values for that key in `this` as well as `other`. + */ + def cogroup[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { + cogroup(other, defaultPartitioner(this, other)) + } + + // /** + // * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a + // * tuple with the list of values for that key in `this`, `other1` and `other2`. + // */ + // def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) + // : IndexedRDD[K, (Seq[V], Seq[W1], Seq[W2])] = { + // cogroup(other1, other2, defaultPartitioner(this, other1, other2)) + // } + + /** + * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the + * list of values for that key in `this` as well as `other`. + */ + def cogroup[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (Seq[V], Seq[W])] = { + cogroup(other, new HashPartitioner(numPartitions)) + } + + // /** + // * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a + // * tuple with the list of values for that key in `this`, `other1` and `other2`. + // */ + // def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) + // : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + // cogroup(other1, other2, new HashPartitioner(numPartitions)) + // } + + /** Alias for cogroup. */ + def groupWith[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { + cogroup(other, defaultPartitioner(this, other)) + } + + // /** Alias for cogroup. */ + // def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) + // : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + // cogroup(other1, other2, defaultPartitioner(self, other1, other2)) + // } + + /** + * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each + * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and + * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. + */ + def join[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): IndexedRDD[K, (V, W)] = { + cogroup(other, partitioner).flatMapValues { + case (vs, ws) => + for (v <- vs.iterator; w <- ws.iterator) yield (v, w) + } + } + + /** + * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the + * pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to + * partition the output RDD. + */ + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): IndexedRDD[K, (V, Option[W])] = { + cogroup(other, partitioner).flatMapValues { + case (vs, ws) => + if (ws.isEmpty) { + vs.iterator.map(v => (v, None)) + } else { + for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w)) + } + } + } + + /** + * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the + * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the + * pair (k, (None, w)) if no elements in `this` have key k. Uses the given Partitioner to + * partition the output RDD. + */ + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner) + : IndexedRDD[K, (Option[V], W)] = { + cogroup(other, partitioner).flatMapValues { + case (vs, ws) => + if (vs.isEmpty) { + ws.iterator.map(w => (None, w)) + } else { + for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w) + } + } + } + + /** + * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each + * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and + * (k, v2) is in `other`. Performs a hash join across the cluster. + */ + def join[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, W)] = { + join(other, defaultPartitioner(this, other)) + } + + /** + * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each + * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and + * (k, v2) is in `other`. Performs a hash join across the cluster. + */ + def join[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (V, W)] = { + join(other, new HashPartitioner(numPartitions)) + } + + /** + * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the + * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output + * using the existing partitioner/parallelism level. + */ + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, Option[W])] = { + leftOuterJoin(other, defaultPartitioner(this, other)) + } + + /** + * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the + * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output + * into `numPartitions` partitions. + */ + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (V, Option[W])] = { + leftOuterJoin(other, new HashPartitioner(numPartitions)) + } + + /** + * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the + * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the + * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting + * RDD using the existing partitioner/parallelism level. + */ + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Option[V], W)] = { + rightOuterJoin(other, defaultPartitioner(this, other)) + } + + /** + * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the + * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the + * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting + * RDD into the given number of partitions. + */ + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (Option[V], W)] = { + rightOuterJoin(other, new HashPartitioner(numPartitions)) + } + + /** @@ -304,7 +571,7 @@ object IndexedRDD { tbl: RDD[(K,V)], existingIndex: RDD[JHashMap[K,Int]] = null ): IndexedRDD[K, V] = { - if(existingIndex == null) { + if (existingIndex == null) { // Shuffle the table (if necessary) val shuffledTbl = if (tbl.partitioner.isEmpty) { @@ -314,11 +581,11 @@ object IndexedRDD { val groups = shuffledTbl.mapPartitions( iter => { val indexMap = new JHashMap[K, Int]() val values = new ArrayBuffer[Seq[V]]() - for((k,v) <- iter){ + for ((k,v) <- iter){ if(!indexMap.contains(k)) { val ind = indexMap.size indexMap.put(k, ind) - values.append(new ArrayBuffer[V]()) + values.append(ArrayBuffer.empty[V]) } val ind = indexMap.get(k) values(ind).asInstanceOf[ArrayBuffer[V]].append(v) @@ -351,11 +618,11 @@ object IndexedRDD { val index: JHashMap[K,Int] = indexIter.next() assert(!indexIter.hasNext()) val values = new Array[Seq[V]](index.size) - for((k,v) <- tblIter) { + for ((k,v) <- tblIter) { assert(index.contains(k)) val ind = index(k) - if(values(ind) == null){ - values(ind) = new ArrayBuffer[V]() + if (values(ind) == null) { + values(ind) = ArrayBuffer.empty[V] } values(ind).asInstanceOf[ArrayBuffer[V]].append(v) } diff --git a/core/src/test/scala/spark/IndexedRDDSuite.scala b/core/src/test/scala/spark/IndexedRDDSuite.scala index aacb6423ee1..f30e1f57fa1 100644 --- a/core/src/test/scala/spark/IndexedRDDSuite.scala +++ b/core/src/test/scala/spark/IndexedRDDSuite.scala @@ -31,6 +31,8 @@ import scala.collection.mutable.HashSet import spark.rdd.ShuffledRDD import spark.SparkContext._ +import spark._ + class IndexedRDDSuite extends FunSuite with SharedSparkContext { @@ -165,6 +167,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4,-4), (4, 4) )).index() val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index(rdd1.index) val joined = rdd1.join(rdd2).collect() + assert(joined.size === 6) assert(joined.toSet === Set( (1, (1, 'x')), (1, (2, 'x')), @@ -177,8 +180,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { 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 rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))).index() + val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))).index(rdd1.index) val joined = rdd1.join(rdd2).collect() assert(joined.size === 6) assert(joined.toSet === Set( @@ -191,8 +194,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { )) } - test("leftOuterJoin") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + test("leftOuterJoinIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val joined = rdd1.leftOuterJoin(rdd2).collect() assert(joined.size === 5) @@ -205,6 +208,35 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { )) } + test("leftOuterJoinIndextoIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index() + 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("leftOuterJoinIndextoSharedIndex") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4, -4))).index() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index(rdd1.index) + val joined = rdd1.leftOuterJoin(rdd2).collect() + assert(joined.size === 6) + assert(joined.toSet === Set( + (1, (1, Some('x'))), + (1, (2, Some('x'))), + (2, (1, Some('y'))), + (2, (1, Some('z'))), + (4, (-4, Some('w'))), + (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'))) From 3bb6e019d4eee5cc80b6b506951ce9efc4f21ed2 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 15 Aug 2013 14:29:48 -0700 Subject: [PATCH 0724/2521] adding better error handling when indexing an RDD --- core/src/main/scala/spark/rdd/IndexedRDD.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index b65efa4447b..7d8e0d3938e 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -350,15 +350,12 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // Get the corresponding indicies and values for this IndexedRDD val (thisIndex, thisValues) = thisTuplesIter.next() assert(!thisTuplesIter.hasNext()) - assert(thisIndex.size == thisValues.size) // Construct a new index val newIndex = thisIndex.clone().asInstanceOf[JHashMap[K, Int]] - assert(thisIndex.size == newIndex.size) // Construct a new array Buffer to store the values val newValues = ArrayBuffer.fill[(Seq[V], ArrayBuffer[W])](thisValues.size)(null) // populate the newValues with the values in this IndexedRDD for ((k,i) <- thisIndex) { - assert(i < thisValues.size) if (thisValues(i) != null) { newValues(i) = (thisValues(i), ArrayBuffer.empty[W]) } @@ -619,7 +616,10 @@ object IndexedRDD { assert(!indexIter.hasNext()) val values = new Array[Seq[V]](index.size) for ((k,v) <- tblIter) { - assert(index.contains(k)) + if (!index.contains(k)) { + throw new SparkException("Error: Try to bind an external index " + + "to an RDD which contains keys that are not in the index.") + } val ind = index(k) if (values(ind) == null) { values(ind) = ArrayBuffer.empty[V] From 1a51deae8a07d961e250cc9b027fd7d6e66af95b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 15 Aug 2013 14:34:07 -0700 Subject: [PATCH 0725/2521] More minor UI changes including code review feedback. --- .../main/resources/spark/ui/static/webui.css | 4 ++++ .../spark/deploy/master/ui/IndexPage.scala | 2 +- core/src/main/scala/spark/ui/UIUtils.scala | 18 +++++++++++++--- .../scala/spark/ui/env/EnvironmentUI.scala | 21 ++++++++++++------- .../main/scala/spark/ui/jobs/PoolTable.scala | 2 +- .../main/scala/spark/ui/jobs/StagePage.scala | 8 +++---- 6 files changed, 39 insertions(+), 16 deletions(-) diff --git a/core/src/main/resources/spark/ui/static/webui.css b/core/src/main/resources/spark/ui/static/webui.css index 70fd0ba31eb..9914a8ad2a3 100644 --- a/core/src/main/resources/spark/ui/static/webui.css +++ b/core/src/main/resources/spark/ui/static/webui.css @@ -49,6 +49,10 @@ line-height: 15px !important; } +.table-fixed { + table-layout:fixed; +} + .table td { vertical-align: middle !important; } 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 60fbcbfad63..47936e2bad6 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -53,7 +53,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { val workers = state.workers.sortBy(_.id) val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers) - val appHeaders = Seq("ID", "Name", "Cores", "Memory per Node", "Launch Time", "User", + val appHeaders = Seq("ID", "Name", "Cores", "Memory per Node", "Submitted Time", "User", "State", "Duration") val activeApps = state.activeApps.sortBy(_.startTime).reverse val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps) diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala index ee7a8b482ed..fe2afc11299 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -125,9 +125,21 @@ private[spark] object UIUtils { } /** 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 => )} + def listingTable[T]( + headers: Seq[String], + makeRow: T => Seq[Node], + rows: Seq[T], + fixedWidth: Boolean = false): Seq[Node] = { + + val colWidth = 100.toDouble / headers.size + val colWidthAttr = if (fixedWidth) colWidth + "%" else "" + var tableClass = "table table-bordered table-striped table-condensed sortable" + if (fixedWidth) { + tableClass += " table-fixed" + } + +
    {h}
    + {headers.map(h => )} {rows.map(r => makeRow(r))} diff --git a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala index 6ee58cda2d7..b3e28ce317b 100644 --- a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala @@ -45,7 +45,8 @@ private[spark] class EnvironmentUI(sc: SparkContext) { ("Scala Home", Properties.scalaHome) ).sorted def jvmRow(kv: (String, String)) = - def jvmTable = UIUtils.listingTable(Seq("Name", "Value"), jvmRow, jvmInformation) + def jvmTable = + UIUtils.listingTable(Seq("Name", "Value"), jvmRow, jvmInformation, fixedWidth = true) val properties = System.getProperties.iterator.toSeq val classPathProperty = properties.find { case (k, v) => @@ -56,8 +57,10 @@ private[spark] class EnvironmentUI(sc: SparkContext) { val propertyHeaders = Seq("Name", "Value") def propertyRow(kv: (String, String)) = - val sparkPropertyTable = UIUtils.listingTable(propertyHeaders, propertyRow, sparkProperties) - val otherPropertyTable = UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties) + val sparkPropertyTable = + UIUtils.listingTable(propertyHeaders, propertyRow, sparkProperties, fixedWidth = true) + val otherPropertyTable = + UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties, fixedWidth = true) val classPathEntries = classPathProperty._2 .split(System.getProperty("path.separator", ":")) @@ -69,17 +72,21 @@ private[spark] class EnvironmentUI(sc: SparkContext) { val classPathHeaders = Seq("Resource", "Source") def classPathRow(data: (String, String)) = - val classPathTable = UIUtils.listingTable(classPathHeaders, classPathRow, classPath) + val classPathTable = + UIUtils.listingTable(classPathHeaders, classPathRow, classPath, fixedWidth = true) val content =

    Runtime Information

    {jvmTable}
    -

    {sparkProperties.size} Spark Properties

    {sparkPropertyTable} +

    {sparkProperties.size} Spark Properties

    + {sparkPropertyTable}
    -

    {otherProperties.size} System Properties

    {otherPropertyTable} +

    {otherProperties.size} System Properties

    + {otherPropertyTable}
    -

    {classPath.size} Classpath Entries

    {classPathTable} +

    {classPath.size} Classpath Entries

    + {classPathTable}
    UIUtils.headerSparkPage(content, sc, "Environment", Environment) diff --git a/core/src/main/scala/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/spark/ui/jobs/PoolTable.scala index 12fb5f0b89d..621828f9c3e 100644 --- a/core/src/main/scala/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/spark/ui/jobs/PoolTable.scala @@ -21,7 +21,7 @@ private[spark] class PoolTable(pools: Seq[Schedulable], listener: JobProgressLis private def poolTable(makeRow: (Schedulable, HashMap[String, HashSet[Stage]]) => Seq[Node], rows: Seq[Schedulable] ): Seq[Node] = { -
    {h}
    {kv._1}{kv._2}
    {kv._1}{kv._2}
    {data._1}{data._2}
    +
    diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 8e2458f94ba..f2a6f4f3032 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -123,16 +123,16 @@ private[spark] class StagePage(parent: JobProgressUI) { if (hasShuffleRead) shuffleReadQuantiles else Nil, if (hasShuffleWrite) shuffleWriteQuantiles else Nil) - val quantileHeaders = Seq("Metric", "Min (0th percentitle)", "25th percentile", - "50th percentile", "75th percentile", "Max (100th percentile)") + val quantileHeaders = Seq("Metric", "Min", "25th percentile", + "Median", "75th percentile", "Max") def quantileRow(data: Seq[String]): Seq[Node] = {data.map(d => )} - Some(listingTable(quantileHeaders, quantileRow, listings)) + Some(listingTable(quantileHeaders, quantileRow, listings, fixedWidth = true)) } val content = summary ++

    Summary Metrics for {numCompleted} Completed Tasks

    ++ -
    {summaryTable.getOrElse("No tasks have reported their execution metrics yet.")}
    ++ +
    {summaryTable.getOrElse("No tasks have reported metrics yet.")}
    ++

    Tasks

    ++ taskTable; headerSparkPage(content, parent.sc, "Details for Stage %d".format(stageId), Jobs) From 4c6ade1ad5e083a44b1907bcae3a8bdc738c5331 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 15 Aug 2013 15:52:28 -0700 Subject: [PATCH 0726/2521] Rename `memoryBytesToString` and `memoryMegabytesToString` These are used all over the place now and they are not specific to memory at all. memoryBytesToString --> bytesToString memoryMegabytesToString --> megabytesToString --- core/src/main/scala/spark/Utils.scala | 10 +++++----- .../main/scala/spark/deploy/master/Master.scala | 2 +- .../scala/spark/deploy/master/ui/IndexPage.scala | 10 +++++----- .../main/scala/spark/deploy/worker/Worker.scala | 2 +- .../scala/spark/deploy/worker/ui/IndexPage.scala | 6 +++--- .../spark/deploy/worker/ui/WorkerWebUI.scala | 4 ++-- .../scala/spark/scheduler/SparkListener.scala | 2 +- .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../spark/storage/BlockFetcherIterator.scala | 4 ++-- .../spark/storage/BlockManagerMasterActor.scala | 14 +++++++------- .../src/main/scala/spark/storage/DiskStore.scala | 4 ++-- .../main/scala/spark/storage/MemoryStore.scala | 6 +++--- .../main/scala/spark/storage/StorageUtils.scala | 4 ++-- .../main/scala/spark/ui/exec/ExecutorsUI.scala | 10 +++++----- .../src/main/scala/spark/ui/jobs/StagePage.scala | 10 +++++----- .../main/scala/spark/ui/jobs/StageTable.scala | 4 ++-- .../main/scala/spark/ui/storage/IndexPage.scala | 4 ++-- .../main/scala/spark/ui/storage/RDDPage.scala | 14 +++++++------- core/src/test/scala/spark/UtilsSuite.scala | 16 ++++++++-------- 19 files changed, 64 insertions(+), 64 deletions(-) diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 673f9a810dd..885a7391d63 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -521,9 +521,9 @@ private object Utils extends Logging { } /** - * Convert a memory quantity in bytes to a human-readable string such as "4.0 MB". + * Convert a quantity in bytes to a human-readable string such as "4.0 MB". */ - def memoryBytesToString(size: Long): String = { + def bytesToString(size: Long): String = { val TB = 1L << 40 val GB = 1L << 30 val MB = 1L << 20 @@ -566,10 +566,10 @@ private object Utils extends Logging { } /** - * Convert a memory quantity in megabytes to a human-readable string such as "4.0 MB". + * Convert a quantity in megabytes to a human-readable string such as "4.0 MB". */ - def memoryMegabytesToString(megabytes: Long): String = { - memoryBytesToString(megabytes * 1024L * 1024L) + def megabytesToString(megabytes: Long): String = { + bytesToString(megabytes * 1024L * 1024L) } /** diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala index 152cb2887a8..04af5e149c6 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/spark/deploy/master/Master.scala @@ -96,7 +96,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act override def receive = { case RegisterWorker(id, host, workerPort, cores, memory, worker_webUiPort, publicAddress) => { logInfo("Registering worker %s:%d with %d cores, %s RAM".format( - host, workerPort, cores, Utils.memoryMegabytesToString(memory))) + host, workerPort, cores, Utils.megabytesToString(memory))) if (idToWorker.contains(id)) { sender ! RegisterWorkerFailed("Duplicate worker ID") } else { 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 4443d880560..1d293ae11ce 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -70,8 +70,8 @@ private[spark] class IndexPage(parent: MasterWebUI) {
  • 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
  • + {Utils.megabytesToString(state.workers.map(_.memory).sum)} Total, + {Utils.megabytesToString(state.workers.map(_.memoryUsed).sum)} Used
  • Applications: {state.activeApps.size} Running, {state.completedApps.size} Completed
  • @@ -116,8 +116,8 @@ private[spark] class IndexPage(parent: MasterWebUI) {
    } @@ -135,7 +135,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { {app.coresGranted} diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 0b5013b8644..d4b58fc34e8 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -96,7 +96,7 @@ private[spark] class Worker( override def preStart() { logInfo("Starting Spark worker %s:%d with %d cores, %s RAM".format( - host, port, cores, Utils.memoryMegabytesToString(memory))) + host, port, cores, Utils.megabytesToString(memory))) sparkHome = new File(Option(System.getenv("SPARK_HOME")).getOrElse(".")) logInfo("Spark home: " + sparkHome) createWorkDir() 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 700eb22d962..b67059068bf 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -65,8 +65,8 @@ private[spark] class IndexPage(parent: WorkerWebUI) { Master URL: {workerState.masterUrl}
  • Cores: {workerState.cores} ({workerState.coresUsed} Used)
  • -
  • Memory: {Utils.memoryMegabytesToString(workerState.memory)} - ({Utils.memoryMegabytesToString(workerState.memoryUsed)} Used)
  • +
  • Memory: {Utils.megabytesToString(workerState.memory)} + ({Utils.megabytesToString(workerState.memoryUsed)} Used)
  • Back to Master

    @@ -97,7 +97,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { @@ -70,9 +70,9 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
    • Memory: - {Utils.memoryBytesToString(memUsed)} Used - ({Utils.memoryBytesToString(maxMem)} Total)
    • -
    • Disk: {Utils.memoryBytesToString(diskSpaceUsed)} Used
    • + {Utils.bytesToString(memUsed)} Used + ({Utils.bytesToString(maxMem)} Total) +
    • Disk: {Utils.bytesToString(diskSpaceUsed)} Used
    diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index f91a415e370..061fb463da7 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -69,13 +69,13 @@ private[spark] class StagePage(parent: JobProgressUI) { {if (hasShuffleRead)
  • Shuffle read: - {Utils.memoryBytesToString(shuffleReadBytes)} + {Utils.bytesToString(shuffleReadBytes)}
  • } {if (hasShuffleWrite)
  • Shuffle write: - {Utils.memoryBytesToString(shuffleWriteBytes)} + {Utils.bytesToString(shuffleWriteBytes)}
  • } @@ -104,7 +104,7 @@ private[spark] class StagePage(parent: JobProgressUI) { ms => parent.formatDuration(ms.toLong)) def getQuantileCols(data: Seq[Double]) = - Distribution(data).get.getQuantiles().map(d => Utils.memoryBytesToString(d.toLong)) + Distribution(data).get.getQuantiles().map(d => Utils.bytesToString(d.toLong)) val shuffleReadSizes = validTasks.map { case(info, metrics, exception) => @@ -159,11 +159,11 @@ private[spark] class StagePage(parent: JobProgressUI) { {if (shuffleRead) { + Utils.bytesToString(s.remoteBytesRead)}.getOrElse("")} }} {if (shuffleWrite) { + Utils.bytesToString(s.shuffleBytesWritten)}.getOrElse("")} }} - - + + } } diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index 40f94b42a6f..b0071994fe1 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -71,11 +71,11 @@ private[spark] class RDDPage(parent: BlockManagerUI) {
  • Memory Size: - {Utils.memoryBytesToString(rddInfo.memSize)} + {Utils.bytesToString(rddInfo.memSize)}
  • Disk Size: - {Utils.memoryBytesToString(rddInfo.diskSize)} + {Utils.bytesToString(rddInfo.diskSize)}
  • @@ -106,10 +106,10 @@ private[spark] class RDDPage(parent: BlockManagerUI) { {block.storageLevel.description} - + } } diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/spark/UtilsSuite.scala index 31c3b25c502..98a6c1a1c9e 100644 --- a/core/src/test/scala/spark/UtilsSuite.scala +++ b/core/src/test/scala/spark/UtilsSuite.scala @@ -26,14 +26,14 @@ import scala.util.Random class UtilsSuite extends FunSuite { - test("memoryBytesToString") { - assert(Utils.memoryBytesToString(10) === "10.0 B") - assert(Utils.memoryBytesToString(1500) === "1500.0 B") - assert(Utils.memoryBytesToString(2000000) === "1953.1 KB") - assert(Utils.memoryBytesToString(2097152) === "2.0 MB") - assert(Utils.memoryBytesToString(2306867) === "2.2 MB") - assert(Utils.memoryBytesToString(5368709120L) === "5.0 GB") - assert(Utils.memoryBytesToString(5L * 1024L * 1024L * 1024L * 1024L) === "5.0 TB") + test("bytesToString") { + assert(Utils.bytesToString(10) === "10.0 B") + assert(Utils.bytesToString(1500) === "1500.0 B") + assert(Utils.bytesToString(2000000) === "1953.1 KB") + assert(Utils.bytesToString(2097152) === "2.0 MB") + assert(Utils.bytesToString(2306867) === "2.2 MB") + assert(Utils.bytesToString(5368709120L) === "5.0 GB") + assert(Utils.bytesToString(5L * 1024L * 1024L * 1024L * 1024L) === "5.0 TB") } test("copyStream") { From 327a4db9f7546c58d93e0ccd38a6d131aee98c78 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 15 Aug 2013 16:36:26 -0700 Subject: [PATCH 0727/2521] changing caching behavior on indexedrdds --- core/src/main/scala/spark/rdd/IndexedRDD.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index 7d8e0d3938e..6b1cb7608bb 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -58,6 +58,12 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( override def getPartitions: Array[Partition] = tuples.getPartitions override def getPreferredLocations(s: Partition): Seq[String] = tuples.getPreferredLocations(s) + override def cache: IndexedRDD[K,V] = { + index.cache + valuesRDD.cache + return this + } + /** * Construct a new IndexedRDD that is indexed by only the keys in the RDD From b877e20a339872f9a29a35272e6c1f280ac901d5 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 17 Jul 2013 14:53:37 -0700 Subject: [PATCH 0728/2521] move yarn to its own directory --- .../main}/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala | 0 .../scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala | 0 .../src/main}/scala/spark/deploy/SparkHadoopUtil.scala | 0 .../src/main}/scala/spark/deploy/yarn/ApplicationMaster.scala | 0 .../scala/spark/deploy/yarn/ApplicationMasterArguments.scala | 0 .../src/main}/scala/spark/deploy/yarn/Client.scala | 0 .../src/main}/scala/spark/deploy/yarn/ClientArguments.scala | 0 .../src/main}/scala/spark/deploy/yarn/WorkerRunnable.scala | 0 .../src/main}/scala/spark/deploy/yarn/YarnAllocationHandler.scala | 0 .../scala/spark/scheduler/cluster/YarnClusterScheduler.scala | 0 10 files changed, 0 insertions(+), 0 deletions(-) rename {core/src/hadoop2-yarn => yarn/src/main}/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala (100%) rename {core/src/hadoop2-yarn => yarn/src/main}/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala (100%) rename {core/src/hadoop2-yarn => yarn/src/main}/scala/spark/deploy/SparkHadoopUtil.scala (100%) rename {core/src/hadoop2-yarn => yarn/src/main}/scala/spark/deploy/yarn/ApplicationMaster.scala (100%) rename {core/src/hadoop2-yarn => yarn/src/main}/scala/spark/deploy/yarn/ApplicationMasterArguments.scala (100%) rename {core/src/hadoop2-yarn => yarn/src/main}/scala/spark/deploy/yarn/Client.scala (100%) rename {core/src/hadoop2-yarn => yarn/src/main}/scala/spark/deploy/yarn/ClientArguments.scala (100%) rename {core/src/hadoop2-yarn => yarn/src/main}/scala/spark/deploy/yarn/WorkerRunnable.scala (100%) rename {core/src/hadoop2-yarn => yarn/src/main}/scala/spark/deploy/yarn/YarnAllocationHandler.scala (100%) rename {core/src/hadoop2-yarn => yarn/src/main}/scala/spark/scheduler/cluster/YarnClusterScheduler.scala (100%) diff --git a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/yarn/src/main/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala similarity index 100% rename from core/src/hadoop2-yarn/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala rename to yarn/src/main/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala diff --git a/core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/yarn/src/main/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala similarity index 100% rename from core/src/hadoop2-yarn/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala rename to yarn/src/main/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala b/yarn/src/main/scala/spark/deploy/SparkHadoopUtil.scala similarity index 100% rename from core/src/hadoop2-yarn/scala/spark/deploy/SparkHadoopUtil.scala rename to yarn/src/main/scala/spark/deploy/SparkHadoopUtil.scala diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala similarity index 100% rename from core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMaster.scala rename to yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/src/main/scala/spark/deploy/yarn/ApplicationMasterArguments.scala similarity index 100% rename from core/src/hadoop2-yarn/scala/spark/deploy/yarn/ApplicationMasterArguments.scala rename to yarn/src/main/scala/spark/deploy/yarn/ApplicationMasterArguments.scala diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/spark/deploy/yarn/Client.scala similarity index 100% rename from core/src/hadoop2-yarn/scala/spark/deploy/yarn/Client.scala rename to yarn/src/main/scala/spark/deploy/yarn/Client.scala diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/spark/deploy/yarn/ClientArguments.scala similarity index 100% rename from core/src/hadoop2-yarn/scala/spark/deploy/yarn/ClientArguments.scala rename to yarn/src/main/scala/spark/deploy/yarn/ClientArguments.scala diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala b/yarn/src/main/scala/spark/deploy/yarn/WorkerRunnable.scala similarity index 100% rename from core/src/hadoop2-yarn/scala/spark/deploy/yarn/WorkerRunnable.scala rename to yarn/src/main/scala/spark/deploy/yarn/WorkerRunnable.scala diff --git a/core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/spark/deploy/yarn/YarnAllocationHandler.scala similarity index 100% rename from core/src/hadoop2-yarn/scala/spark/deploy/yarn/YarnAllocationHandler.scala rename to yarn/src/main/scala/spark/deploy/yarn/YarnAllocationHandler.scala diff --git a/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/src/main/scala/spark/scheduler/cluster/YarnClusterScheduler.scala similarity index 100% rename from core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala rename to yarn/src/main/scala/spark/scheduler/cluster/YarnClusterScheduler.scala From f67b94ad4fc8c9e7a71dd7f65d617743947ae91c Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 17 Jul 2013 17:31:26 -0700 Subject: [PATCH 0729/2521] remove core/src/hadoop{1,2} dirs --- .../hadoop/mapred/HadoopMapRedUtil.scala | 27 ----------- .../mapreduce/HadoopMapReduceUtil.scala | 30 ------------ .../scala/spark/deploy/SparkHadoopUtil.scala | 47 ------------------- .../hadoop/mapred/HadoopMapRedUtil.scala | 0 .../mapreduce/HadoopMapReduceUtil.scala | 0 .../scala/spark/deploy/SparkHadoopUtil.scala | 0 6 files changed, 104 deletions(-) delete mode 100644 core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala delete mode 100644 core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala delete mode 100644 core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala rename core/src/{hadoop1 => main}/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala (100%) rename core/src/{hadoop1 => main}/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala (100%) rename core/src/{hadoop1 => main}/scala/spark/deploy/SparkHadoopUtil.scala (100%) diff --git a/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala deleted file mode 100644 index 4b3d84670c2..00000000000 --- a/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala +++ /dev/null @@ -1,27 +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 org.apache.hadoop.mapred - -trait HadoopMapRedUtil { - def newJobContext(conf: JobConf, jobId: JobID): JobContext = new JobContextImpl(conf, jobId) - - def newTaskAttemptContext(conf: JobConf, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContextImpl(conf, attemptId) - - def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = new TaskAttemptID(jtIdentifier, - jobId, isMap, taskId, attemptId) -} diff --git a/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala deleted file mode 100644 index aa3b1ed3a58..00000000000 --- a/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala +++ /dev/null @@ -1,30 +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 org.apache.hadoop.mapreduce - -import org.apache.hadoop.conf.Configuration -import task.{TaskAttemptContextImpl, JobContextImpl} - -trait HadoopMapReduceUtil { - def newJobContext(conf: Configuration, jobId: JobID): JobContext = new JobContextImpl(conf, jobId) - - def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContextImpl(conf, attemptId) - - def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = new TaskAttemptID(jtIdentifier, - jobId, isMap, taskId, attemptId) -} diff --git a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala deleted file mode 100644 index 617954cb987..00000000000 --- a/core/src/hadoop2/scala/spark/deploy/SparkHadoopUtil.scala +++ /dev/null @@ -1,47 +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.deploy -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.mapred.JobConf - - -/** - * Contains util methods to interact with Hadoop from spark. - */ -object SparkHadoopUtil { - - def getUserNameFromEnvironment(): String = { - // defaulting to -D ... - System.getProperty("user.name") - } - - def runAsUser(func: (Product) => Unit, args: Product) { - - // Add support, if exists - for now, simply run func ! - func(args) - } - - // 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) {} - - def isYarnMode(): Boolean = { false } - -} diff --git a/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/main/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala similarity index 100% rename from core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala rename to core/src/main/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala diff --git a/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/main/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala similarity index 100% rename from core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala rename to core/src/main/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala diff --git a/core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/spark/deploy/SparkHadoopUtil.scala similarity index 100% rename from core/src/hadoop1/scala/spark/deploy/SparkHadoopUtil.scala rename to core/src/main/scala/spark/deploy/SparkHadoopUtil.scala From 69c3bbf688cdd21171413d415cfc6d6cb8e77bd5 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 17 Jul 2013 17:33:38 -0700 Subject: [PATCH 0730/2521] dynamically detect hadoop version --- .../hadoop/mapred/HadoopMapRedUtil.scala | 26 +++++++++++--- .../mapreduce/HadoopMapReduceUtil.scala | 30 +++++++++++++--- project/SparkBuild.scala | 35 ++----------------- 3 files changed, 51 insertions(+), 40 deletions(-) diff --git a/core/src/main/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/main/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala index 25386b2796e..6cfafd3760f 100644 --- a/core/src/main/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala +++ b/core/src/main/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala @@ -18,10 +18,28 @@ package org.apache.hadoop.mapred trait HadoopMapRedUtil { - def newJobContext(conf: JobConf, jobId: JobID): JobContext = new JobContext(conf, jobId) + def newJobContext(conf: JobConf, jobId: JobID): JobContext = { + val klass = firstAvailableClass("org.apache.hadoop.mapred.JobContextImpl", "org.apache.hadoop.mapred.JobContext"); + val ctor = klass.getDeclaredConstructor(classOf[JobConf], classOf[org.apache.hadoop.mapreduce.JobID]) + ctor.newInstance(conf, jobId).asInstanceOf[JobContext] + } - def newTaskAttemptContext(conf: JobConf, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContext(conf, attemptId) + def newTaskAttemptContext(conf: JobConf, attemptId: TaskAttemptID): TaskAttemptContext = { + val klass = firstAvailableClass("org.apache.hadoop.mapred.TaskAttemptContextImpl", "org.apache.hadoop.mapred.TaskAttemptContext") + val ctor = klass.getDeclaredConstructor(classOf[JobConf], classOf[TaskAttemptID]) + ctor.newInstance(conf, attemptId).asInstanceOf[TaskAttemptContext] + } - def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = new TaskAttemptID(jtIdentifier, - jobId, isMap, taskId, attemptId) + def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = { + new TaskAttemptID(jtIdentifier, jobId, isMap, taskId, attemptId) + } + + private def firstAvailableClass(first: String, second: String): Class[_] = { + try { + Class.forName(first) + } catch { + case e: ClassNotFoundException => + Class.forName(second) + } + } } diff --git a/core/src/main/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/main/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala index b1002e0cace..0f77828dc89 100644 --- a/core/src/main/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala +++ b/core/src/main/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala @@ -20,10 +20,32 @@ package org.apache.hadoop.mapreduce import org.apache.hadoop.conf.Configuration trait HadoopMapReduceUtil { - def newJobContext(conf: Configuration, jobId: JobID): JobContext = new JobContext(conf, jobId) + def newJobContext(conf: Configuration, jobId: JobID): JobContext = { + val klass = firstAvailableClass( + "org.apache.hadoop.mapreduce.task.JobContextImpl", + "org.apache.hadoop.mapreduce.JobContext") + val ctor = klass.getDeclaredConstructor(classOf[Configuration], classOf[JobID]) + ctor.newInstance(conf, jobId).asInstanceOf[JobContext] + } - def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContext(conf, attemptId) + def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext = { + val klass = firstAvailableClass( + "org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl", + "org.apache.hadoop.mapreduce.TaskAttemptContext") + val ctor = klass.getDeclaredConstructor(classOf[Configuration], classOf[TaskAttemptID]) + ctor.newInstance(conf, attemptId).asInstanceOf[TaskAttemptContext] + } - def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = new TaskAttemptID(jtIdentifier, - jobId, isMap, taskId, attemptId) + def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = { + new TaskAttemptID(jtIdentifier, jobId, isMap, taskId, attemptId) + } + + private def firstAvailableClass(first: String, second: String): Class[_] = { + try { + Class.forName(first) + } catch { + case e: ClassNotFoundException => + Class.forName(second) + } + } } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f6519c82874..a06550bb971 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -27,13 +27,8 @@ object SparkBuild extends Build { // Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or // "1.0.4" for Apache releases, or "0.20.2-cdh3u5" for Cloudera Hadoop. val HADOOP_VERSION = "1.0.4" - val HADOOP_MAJOR_VERSION = "1" - val HADOOP_YARN = false - - // For Hadoop 2 versions such as "2.0.0-mr1-cdh4.1.1", set the HADOOP_MAJOR_VERSION to "2" //val HADOOP_VERSION = "2.0.0-mr1-cdh4.1.1" - //val HADOOP_MAJOR_VERSION = "2" - //val HADOOP_YARN = false + val HADOOP_YARN = false // For Hadoop 2 YARN support //val HADOOP_VERSION = "2.0.2-alpha" @@ -184,37 +179,13 @@ object SparkBuild extends Build { "org.apache.mesos" % "mesos" % "0.12.1", "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test", + "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION, "com.codahale.metrics" % "metrics-core" % "3.0.0", "com.codahale.metrics" % "metrics-jvm" % "3.0.0", "com.codahale.metrics" % "metrics-json" % "3.0.0", "com.twitter" % "chill_2.9.3" % "0.3.1", "com.twitter" % "chill-java" % "0.3.1" - ) ++ ( - if (HADOOP_MAJOR_VERSION == "2") { - if (HADOOP_YARN) { - Seq( - // Exclude rule required for all ? - "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( - "org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm), - "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty, excludeAsm) - ) - } - } else { - Seq("org.apache.hadoop" % "hadoop-core" % HADOOP_VERSION excludeAll(excludeJackson, excludeNetty) ) - }), - unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / - ( if (HADOOP_YARN && HADOOP_MAJOR_VERSION == "2") { - "src/hadoop2-yarn/scala" - } else { - "src/hadoop" + HADOOP_MAJOR_VERSION + "/scala" - } ) - } + ) ) ++ assemblySettings ++ extraAssemblySettings def rootSettings = sharedSettings ++ Seq( From 273b499b9ac8373f0f92ebf8e4141fe51cec4a33 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Thu, 18 Jul 2013 13:36:34 -0700 Subject: [PATCH 0731/2521] yarn sbt --- project/SparkBuild.scala | 28 +++++++++++++++------------- 1 file changed, 15 insertions(+), 13 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a06550bb971..99351ca9356 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -24,21 +24,10 @@ import AssemblyKeys._ //import com.jsuereth.pgp.sbtplugin.PgpKeys._ object SparkBuild extends Build { - // Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or - // "1.0.4" for Apache releases, or "0.20.2-cdh3u5" for Cloudera Hadoop. - val HADOOP_VERSION = "1.0.4" - //val HADOOP_VERSION = "2.0.0-mr1-cdh4.1.1" - val HADOOP_YARN = false - - // For Hadoop 2 YARN support - //val HADOOP_VERSION = "2.0.2-alpha" - //val HADOOP_MAJOR_VERSION = "2" - //val HADOOP_YARN = true - // HBase version; set as appropriate. val HBASE_VERSION = "0.94.6" - lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming, mllib, tools) + lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming, mllib, tools, yarn) lazy val core = Project("core", file("core"), settings = coreSettings) @@ -54,6 +43,8 @@ object SparkBuild extends Build { lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn (core) + lazy val yarn = Project("yarn", file("yarn"), settings = yarnSettings) 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") @@ -179,7 +170,7 @@ object SparkBuild extends Build { "org.apache.mesos" % "mesos" % "0.12.1", "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test", - "org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION, + "org.apache.hadoop" % "hadoop-client" % "1.0.4", "com.codahale.metrics" % "metrics-core" % "3.0.0", "com.codahale.metrics" % "metrics-jvm" % "3.0.0", "com.codahale.metrics" % "metrics-json" % "3.0.0", @@ -244,6 +235,17 @@ object SparkBuild extends Build { ) ) ++ assemblySettings ++ extraAssemblySettings + def yarnSettings = sharedSettings ++ Seq( + name := "spark-yarn", + libraryDependencies ++= Seq( + // Exclude rule required for all ? + "org.apache.hadoop" % "hadoop-client" % "2.0.2-alpha" excludeAll(excludeJackson, excludeNetty), + "org.apache.hadoop" % "hadoop-yarn-api" % "2.0.2-alpha" excludeAll(excludeJackson, excludeNetty), + "org.apache.hadoop" % "hadoop-yarn-common" % "2.0.2-alpha" excludeAll(excludeJackson, excludeNetty), + "org.apache.hadoop" % "hadoop-yarn-client" % "2.0.2-alpha" excludeAll(excludeJackson, excludeNetty) + ) + ) + def extraAssemblySettings() = Seq(test in assembly := {}) ++ Seq( mergeStrategy in assembly := { case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard From 5d0785b4e5bb681770675a1729f1742a8cd3b491 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Sat, 20 Jul 2013 11:34:13 -0700 Subject: [PATCH 0732/2521] remove hadoop-yarn's org/apache/... --- .../hadoop/mapred/HadoopMapRedUtil.scala | 30 ------------------- .../mapreduce/HadoopMapReduceUtil.scala | 30 ------------------- 2 files changed, 60 deletions(-) delete mode 100644 yarn/src/main/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala delete mode 100644 yarn/src/main/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala diff --git a/yarn/src/main/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/yarn/src/main/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala deleted file mode 100644 index 0f972b7a0b7..00000000000 --- a/yarn/src/main/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala +++ /dev/null @@ -1,30 +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 org.apache.hadoop.mapred - -import org.apache.hadoop.mapreduce.TaskType - -trait HadoopMapRedUtil { - def newJobContext(conf: JobConf, jobId: JobID): JobContext = new JobContextImpl(conf, jobId) - - def newTaskAttemptContext(conf: JobConf, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContextImpl(conf, attemptId) - - def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = - new TaskAttemptID(jtIdentifier, jobId, if (isMap) TaskType.MAP else TaskType.REDUCE, taskId, attemptId) -} diff --git a/yarn/src/main/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/yarn/src/main/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala deleted file mode 100644 index 1a7cdf47881..00000000000 --- a/yarn/src/main/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala +++ /dev/null @@ -1,30 +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 org.apache.hadoop.mapreduce - -import org.apache.hadoop.conf.Configuration -import task.{TaskAttemptContextImpl, JobContextImpl} - -trait HadoopMapReduceUtil { - def newJobContext(conf: Configuration, jobId: JobID): JobContext = new JobContextImpl(conf, jobId) - - def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContextImpl(conf, attemptId) - - def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = - new TaskAttemptID(jtIdentifier, jobId, if (isMap) TaskType.MAP else TaskType.REDUCE, taskId, attemptId) -} From 8b1c1520fc26132a21062ebb063dea25e9b36b8b Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Sat, 20 Jul 2013 11:34:18 -0700 Subject: [PATCH 0733/2521] add comment --- .../org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/main/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala index 0f77828dc89..dd624d88908 100644 --- a/core/src/main/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala +++ b/core/src/main/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala @@ -22,16 +22,16 @@ import org.apache.hadoop.conf.Configuration trait HadoopMapReduceUtil { def newJobContext(conf: Configuration, jobId: JobID): JobContext = { val klass = firstAvailableClass( - "org.apache.hadoop.mapreduce.task.JobContextImpl", - "org.apache.hadoop.mapreduce.JobContext") + "org.apache.hadoop.mapreduce.task.JobContextImpl", // hadoop2, hadoop2-yarn + "org.apache.hadoop.mapreduce.JobContext") // hadoop1 val ctor = klass.getDeclaredConstructor(classOf[Configuration], classOf[JobID]) ctor.newInstance(conf, jobId).asInstanceOf[JobContext] } def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext = { val klass = firstAvailableClass( - "org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl", - "org.apache.hadoop.mapreduce.TaskAttemptContext") + "org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl", // hadoop2, hadoop2-yarn + "org.apache.hadoop.mapreduce.TaskAttemptContext") // hadoop1 val ctor = klass.getDeclaredConstructor(classOf[Configuration], classOf[TaskAttemptID]) ctor.newInstance(conf, attemptId).asInstanceOf[TaskAttemptContext] } From cb4ef19214332b5e9c2e0d0bfa0a72262122d04e Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Mon, 22 Jul 2013 14:03:31 -0700 Subject: [PATCH 0734/2521] yarn support --- bin/compute-classpath.cmd | 2 ++ bin/compute-classpath.sh | 2 ++ project/SparkBuild.scala | 12 ++++++------ 3 files changed, 10 insertions(+), 6 deletions(-) diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index eb836b0ffd1..9178b852e6b 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -34,6 +34,7 @@ set EXAMPLES_DIR=%FWDIR%examples set BAGEL_DIR=%FWDIR%bagel set MLLIB_DIR=%FWDIR%mllib set TOOLS_DIR=%FWDIR%tools +set YARN_DIR=%FWDIR%yarn set STREAMING_DIR=%FWDIR%streaming set PYSPARK_DIR=%FWDIR%python @@ -50,6 +51,7 @@ 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 +set CLASSPATH=%CLASSPATH%;%YARN_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 e4ce1ca8481..db6fc866abf 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -37,6 +37,7 @@ EXAMPLES_DIR="$FWDIR/examples" BAGEL_DIR="$FWDIR/bagel" MLLIB_DIR="$FWDIR/mllib" TOOLS_DIR="$FWDIR/tools" +YARN_DIR="$FWDIR/yarn" STREAMING_DIR="$FWDIR/streaming" PYSPARK_DIR="$FWDIR/python" @@ -72,6 +73,7 @@ function dev_classpath { 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" + CLASSPATH="$CLASSPATH:$YARN_DIR/target/scala-$SCALA_VERSION/classes" for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do CLASSPATH="$CLASSPATH:$jar" done diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 99351ca9356..ede49ea38b4 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -170,7 +170,7 @@ object SparkBuild extends Build { "org.apache.mesos" % "mesos" % "0.12.1", "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test", - "org.apache.hadoop" % "hadoop-client" % "1.0.4", + "org.apache.hadoop" % "hadoop-client" % "1.0.4" excludeAll(excludeJackson, excludeNetty, excludeAsm), "com.codahale.metrics" % "metrics-core" % "3.0.0", "com.codahale.metrics" % "metrics-jvm" % "3.0.0", "com.codahale.metrics" % "metrics-json" % "3.0.0", @@ -239,12 +239,12 @@ object SparkBuild extends Build { name := "spark-yarn", libraryDependencies ++= Seq( // Exclude rule required for all ? - "org.apache.hadoop" % "hadoop-client" % "2.0.2-alpha" excludeAll(excludeJackson, excludeNetty), - "org.apache.hadoop" % "hadoop-yarn-api" % "2.0.2-alpha" excludeAll(excludeJackson, excludeNetty), - "org.apache.hadoop" % "hadoop-yarn-common" % "2.0.2-alpha" excludeAll(excludeJackson, excludeNetty), - "org.apache.hadoop" % "hadoop-yarn-client" % "2.0.2-alpha" excludeAll(excludeJackson, excludeNetty) + "org.apache.hadoop" % "hadoop-client" % "2.0.2-alpha" excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-api" % "2.0.2-alpha" excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-common" % "2.0.2-alpha" excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-client" % "2.0.2-alpha" excludeAll(excludeJackson, excludeNetty, excludeAsm) ) - ) + ) ++ assemblySettings ++ extraAssemblySettings def extraAssemblySettings() = Seq(test in assembly := {}) ++ Seq( mergeStrategy in assembly := { From 43ebcb84840dc9db61e5912d9a37707c065edc5a Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Tue, 23 Jul 2013 17:37:10 -0700 Subject: [PATCH 0735/2521] rename HadoopMapRedUtil => SparkHadoopMapRedUtil, HadoopMapReduceUtil => SparkHadoopMapReduceUtil --- .../{HadoopMapRedUtil.scala => SparkHadoopMapRedUtil.scala} | 2 +- ...oopMapReduceUtil.scala => SparkHadoopMapReduceUtil.scala} | 2 +- core/src/main/scala/spark/HadoopWriter.scala | 2 +- core/src/main/scala/spark/PairRDDFunctions.scala | 5 +++-- core/src/main/scala/spark/rdd/NewHadoopRDD.scala | 2 +- 5 files changed, 7 insertions(+), 6 deletions(-) rename core/src/main/scala/org/apache/hadoop/mapred/{HadoopMapRedUtil.scala => SparkHadoopMapRedUtil.scala} (98%) rename core/src/main/scala/org/apache/hadoop/mapreduce/{HadoopMapReduceUtil.scala => SparkHadoopMapReduceUtil.scala} (98%) diff --git a/core/src/main/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala similarity index 98% rename from core/src/main/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala rename to core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala index 6cfafd3760f..f87460039b0 100644 --- a/core/src/main/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala +++ b/core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala @@ -17,7 +17,7 @@ package org.apache.hadoop.mapred -trait HadoopMapRedUtil { +trait SparkHadoopMapRedUtil { def newJobContext(conf: JobConf, jobId: JobID): JobContext = { val klass = firstAvailableClass("org.apache.hadoop.mapred.JobContextImpl", "org.apache.hadoop.mapred.JobContext"); val ctor = klass.getDeclaredConstructor(classOf[JobConf], classOf[org.apache.hadoop.mapreduce.JobID]) diff --git a/core/src/main/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala similarity index 98% rename from core/src/main/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala rename to core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala index dd624d88908..bea62536771 100644 --- a/core/src/main/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala +++ b/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala @@ -19,7 +19,7 @@ package org.apache.hadoop.mapreduce import org.apache.hadoop.conf.Configuration -trait HadoopMapReduceUtil { +trait SparkHadoopMapReduceUtil { def newJobContext(conf: Configuration, jobId: JobID): JobContext = { val klass = firstAvailableClass( "org.apache.hadoop.mapreduce.task.JobContextImpl", // hadoop2, hadoop2-yarn diff --git a/core/src/main/scala/spark/HadoopWriter.scala b/core/src/main/scala/spark/HadoopWriter.scala index b1fe0075a35..60840ce77ec 100644 --- a/core/src/main/scala/spark/HadoopWriter.scala +++ b/core/src/main/scala/spark/HadoopWriter.scala @@ -36,7 +36,7 @@ import spark.SerializableWritable * Saves the RDD using a JobConf, which should contain an output key class, an output value class, * a filename to write to, etc, exactly like in a Hadoop MapReduce job. */ -class HadoopWriter(@transient jobConf: JobConf) extends Logging with HadoopMapRedUtil with Serializable { +class HadoopWriter(@transient jobConf: JobConf) extends Logging with SparkHadoopMapRedUtil with Serializable { private val now = new Date() private val conf = new SerializableWritable(jobConf) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 6b0cc2fbf15..aeeac65ccaa 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -37,7 +37,8 @@ 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} +import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, + RecordWriter => NewRecordWriter, Job => NewAPIHadoopJob, SparkHadoopMapReduceUtil} import org.apache.hadoop.security.UserGroupInformation import spark.partial.BoundedDouble @@ -53,7 +54,7 @@ import spark.Partitioner._ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( self: RDD[(K, V)]) extends Logging - with HadoopMapReduceUtil + with SparkHadoopMapReduceUtil with Serializable { /** diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index 0b716081695..184685528e3 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -43,7 +43,7 @@ class NewHadoopRDD[K, V]( valueClass: Class[V], @transient conf: Configuration) extends RDD[(K, V)](sc, Nil) - with HadoopMapReduceUtil + with SparkHadoopMapReduceUtil with Logging { // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it From 4f43fd791ab0e84693e2337358c6b880a1593e54 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 24 Jul 2013 12:41:40 -0700 Subject: [PATCH 0736/2521] make SparkHadoopUtil a member of SparkEnv --- core/src/main/scala/spark/SparkContext.scala | 11 +++++++---- core/src/main/scala/spark/SparkEnv.scala | 2 ++ core/src/main/scala/spark/Utils.scala | 7 ++----- .../main/scala/spark/deploy/SparkHadoopUtil.scala | 2 +- .../spark/executor/StandaloneExecutorBackend.scala | 6 +++--- core/src/main/scala/spark/rdd/CheckpointRDD.scala | 14 ++++++++------ core/src/main/scala/spark/rdd/HadoopRDD.scala | 6 +++--- .../scala/spark/scheduler/InputFormatInfo.scala | 9 +++++---- .../main/scala/spark/examples/SparkHdfsLR.scala | 3 +-- 9 files changed, 32 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 80c65dfebd2..f020b2554be 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -58,7 +58,7 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.mesos.MesosNativeLibrary -import spark.deploy.{LocalSparkCluster, SparkHadoopUtil} +import spark.deploy.LocalSparkCluster import spark.partial.{ApproximateEvaluator, PartialResult} import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD} import spark.scheduler.{DAGScheduler, DAGSchedulerSource, ResultTask, ShuffleMapTask, SparkListener, @@ -241,7 +241,8 @@ class SparkContext( /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = { - val conf = SparkHadoopUtil.newConfiguration() + val env = SparkEnv.get + val conf = env.hadoop.newConfiguration() // Explicitly check for S3 environment variables if (System.getenv("AWS_ACCESS_KEY_ID") != null && System.getenv("AWS_SECRET_ACCESS_KEY") != null) { conf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) @@ -629,10 +630,11 @@ class SparkContext( logWarning("null specified as parameter to addJar", new SparkException("null specified as parameter to addJar")) } else { + val env = SparkEnv.get val uri = new URI(path) val key = uri.getScheme match { case null | "file" => - if (SparkHadoopUtil.isYarnMode()) { + if (env.hadoop.isYarnMode()) { logWarning("local jar specified as parameter to addJar under Yarn mode") return } @@ -815,8 +817,9 @@ class SparkContext( * prevent accidental overriding of checkpoint files in the existing directory. */ def setCheckpointDir(dir: String, useExisting: Boolean = false) { + val env = SparkEnv.get val path = new Path(dir) - val fs = path.getFileSystem(SparkHadoopUtil.newConfiguration()) + val fs = path.getFileSystem(env.hadoop.newConfiguration()) if (!useExisting) { if (fs.exists(path)) { throw new Exception("Checkpoint directory '" + path + "' already exists.") diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 0adbf1d96e9..73990f04237 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -25,6 +25,7 @@ import akka.remote.RemoteActorRefProvider import spark.broadcast.BroadcastManager import spark.metrics.MetricsSystem +import spark.deploy.SparkHadoopUtil import spark.storage.BlockManager import spark.storage.BlockManagerMaster import spark.network.ConnectionManager @@ -60,6 +61,7 @@ class SparkEnv ( // If executorId is NOT found, return defaultHostPort var executorIdToHostPort: Option[(String, String) => String]) { + val hadoop = new SparkHadoopUtil private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() def stop() { diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 673f9a810dd..7ea9b0c28a8 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -266,8 +266,9 @@ private object Utils extends Logging { } case _ => // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others + val env = SparkEnv.get val uri = new URI(url) - val conf = SparkHadoopUtil.newConfiguration() + val conf = env.hadoop.newConfiguration() val fs = FileSystem.get(uri, conf) val in = fs.open(new Path(uri)) val out = new FileOutputStream(tempFile) @@ -433,10 +434,6 @@ private object Utils extends Logging { try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } } } - def getUserNameFromEnvironment(): String = { - SparkHadoopUtil.getUserNameFromEnvironment - } - // 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)]() diff --git a/core/src/main/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/spark/deploy/SparkHadoopUtil.scala index 617954cb987..c4ed0bb17ea 100644 --- a/core/src/main/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/spark/deploy/SparkHadoopUtil.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.mapred.JobConf /** * Contains util methods to interact with Hadoop from spark. */ -object SparkHadoopUtil { +class SparkHadoopUtil { def getUserNameFromEnvironment(): String = { // defaulting to -D ... diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index e47fe500216..a9e06f8d545 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -22,9 +22,8 @@ import java.nio.ByteBuffer import akka.actor.{ActorRef, Actor, Props, Terminated} import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} -import spark.{Logging, Utils} +import spark.{Logging, Utils, SparkEnv} import spark.TaskState.TaskState -import spark.deploy.SparkHadoopUtil import spark.scheduler.cluster.StandaloneClusterMessages._ import spark.util.AkkaUtils @@ -82,7 +81,8 @@ private[spark] class StandaloneExecutorBackend( private[spark] object StandaloneExecutorBackend { def run(driverUrl: String, executorId: String, hostname: String, cores: Int) { - SparkHadoopUtil.runAsUser(run0, Tuple4[Any, Any, Any, Any] (driverUrl, executorId, hostname, cores)) + val env = SparkEnv.get + env.hadoop.runAsUser(run0, Tuple4[Any, Any, Any, Any] (driverUrl, executorId, hostname, cores)) } // This will be run 'as' the user diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala index 6794e0e2018..1ad5fe65390 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala @@ -25,7 +25,6 @@ import org.apache.hadoop.util.ReflectionUtils import org.apache.hadoop.fs.Path import java.io.{File, IOException, EOFException} import java.text.NumberFormat -import spark.deploy.SparkHadoopUtil private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {} @@ -82,8 +81,9 @@ private[spark] object CheckpointRDD extends Logging { } def writeToFile[T](path: String, blockSize: Int = -1)(ctx: TaskContext, iterator: Iterator[T]) { + val env = SparkEnv.get val outputDir = new Path(path) - val fs = outputDir.getFileSystem(SparkHadoopUtil.newConfiguration()) + val fs = outputDir.getFileSystem(env.hadoop.newConfiguration()) val finalOutputName = splitIdToFile(ctx.splitId) val finalOutputPath = new Path(outputDir, finalOutputName) @@ -101,7 +101,7 @@ private[spark] object CheckpointRDD extends Logging { // This is mainly for testing purpose fs.create(tempOutputPath, false, bufferSize, fs.getDefaultReplication, blockSize) } - val serializer = SparkEnv.get.serializer.newInstance() + val serializer = env.serializer.newInstance() val serializeStream = serializer.serializeStream(fileOutputStream) serializeStream.writeAll(iterator) serializeStream.close() @@ -121,10 +121,11 @@ private[spark] object CheckpointRDD extends Logging { } def readFromFile[T](path: Path, context: TaskContext): Iterator[T] = { - val fs = path.getFileSystem(SparkHadoopUtil.newConfiguration()) + val env = SparkEnv.get + val fs = path.getFileSystem(env.hadoop.newConfiguration()) val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt val fileInputStream = fs.open(path, bufferSize) - val serializer = SparkEnv.get.serializer.newInstance() + val serializer = env.serializer.newInstance() val deserializeStream = serializer.deserializeStream(fileInputStream) // Register an on-task-completion callback to close the input stream. @@ -140,10 +141,11 @@ private[spark] object CheckpointRDD extends Logging { import spark._ val Array(cluster, hdfsPath) = args + val env = SparkEnv.get val sc = new SparkContext(cluster, "CheckpointRDD Test") val rdd = sc.makeRDD(1 to 10, 10).flatMap(x => 1 to 10000) val path = new Path(hdfsPath, "temp") - val fs = path.getFileSystem(SparkHadoopUtil.newConfiguration()) + val fs = path.getFileSystem(env.hadoop.newConfiguration()) sc.runJob(rdd, CheckpointRDD.writeToFile(path.toString, 1024) _) val cpRDD = new CheckpointRDD[Int](sc, path.toString) assert(cpRDD.partitions.length == rdd.partitions.length, "Number of partitions is not the same") diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index fd00d59c771..6c41b977809 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -32,8 +32,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.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, SparkEnv, TaskContext} import spark.util.NextIterator import org.apache.hadoop.conf.Configurable @@ -68,7 +67,8 @@ class HadoopRDD[K, V]( private val confBroadcast = sc.broadcast(new SerializableWritable(conf)) override def getPartitions: Array[Partition] = { - SparkHadoopUtil.addCredentials(conf); + val env = SparkEnv.get + env.hadoop.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 65f8c3200e4..8f1b9b29b54 100644 --- a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/spark/scheduler/InputFormatInfo.scala @@ -17,7 +17,7 @@ package spark.scheduler -import spark.Logging +import spark.{Logging, SparkEnv} import scala.collection.immutable.Set import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.apache.hadoop.security.UserGroupInformation @@ -26,7 +26,6 @@ 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 /** @@ -88,8 +87,9 @@ 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 env = SparkEnv.get val conf = new JobConf(configuration) - SparkHadoopUtil.addCredentials(conf); + env.hadoop.addCredentials(conf) FileInputFormat.setInputPaths(conf, path) val instance: org.apache.hadoop.mapreduce.InputFormat[_, _] = @@ -108,8 +108,9 @@ 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 env = SparkEnv.get val jobConf = new JobConf(configuration) - SparkHadoopUtil.addCredentials(jobConf); + env.hadoop.addCredentials(jobConf) FileInputFormat.setInputPaths(jobConf, path) val instance: org.apache.hadoop.mapred.InputFormat[_, _] = diff --git a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala index ef6e09a8e84..43c91156640 100644 --- a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala @@ -21,7 +21,6 @@ import java.util.Random import scala.math.exp import spark.util.Vector import spark._ -import spark.deploy.SparkHadoopUtil import spark.scheduler.InputFormatInfo /** @@ -52,7 +51,7 @@ object SparkHdfsLR { System.exit(1) } val inputPath = args(1) - val conf = SparkHadoopUtil.newConfiguration() + val conf = SparkEnv.get.hadoop.newConfiguration() val sc = new SparkContext(args(0), "SparkHdfsLR", System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")), Map(), InputFormatInfo.computePreferredLocations( From bd0bab47c9602462628b1d3c90d5eb5d889f4596 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 24 Jul 2013 13:07:27 -0700 Subject: [PATCH 0737/2521] SparkEnv isn't available this early, and not needed anyway --- .../scala/spark/deploy/SparkHadoopUtil.scala | 11 ----------- .../executor/StandaloneExecutorBackend.scala | 14 -------------- .../scala/spark/deploy/SparkHadoopUtil.scala | 16 ---------------- 3 files changed, 41 deletions(-) diff --git a/core/src/main/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/spark/deploy/SparkHadoopUtil.scala index c4ed0bb17ea..882161e6699 100644 --- a/core/src/main/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/spark/deploy/SparkHadoopUtil.scala @@ -25,17 +25,6 @@ import org.apache.hadoop.mapred.JobConf */ class SparkHadoopUtil { - def getUserNameFromEnvironment(): String = { - // defaulting to -D ... - System.getProperty("user.name") - } - - def runAsUser(func: (Product) => Unit, args: Product) { - - // Add support, if exists - for now, simply run func ! - func(args) - } - // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems def newConfiguration(): Configuration = new Configuration() diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala index a9e06f8d545..b5fb6dbe29e 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala @@ -81,20 +81,6 @@ private[spark] class StandaloneExecutorBackend( private[spark] object StandaloneExecutorBackend { def run(driverUrl: String, executorId: String, hostname: String, cores: Int) { - val env = SparkEnv.get - env.hadoop.runAsUser(run0, Tuple4[Any, Any, Any, Any] (driverUrl, executorId, hostname, cores)) - } - - // This will be run 'as' the user - def run0(args: Product) { - assert(4 == args.productArity) - runImpl(args.productElement(0).asInstanceOf[String], - args.productElement(1).asInstanceOf[String], - args.productElement(2).asInstanceOf[String], - args.productElement(3).asInstanceOf[Int]) - } - - private def runImpl(driverUrl: String, executorId: String, hostname: String, cores: Int) { // Debug code Utils.checkHost(hostname) diff --git a/yarn/src/main/scala/spark/deploy/SparkHadoopUtil.scala b/yarn/src/main/scala/spark/deploy/SparkHadoopUtil.scala index 6122fdced0d..a812bcf8672 100644 --- a/yarn/src/main/scala/spark/deploy/SparkHadoopUtil.scala +++ b/yarn/src/main/scala/spark/deploy/SparkHadoopUtil.scala @@ -32,22 +32,6 @@ object SparkHadoopUtil { val yarnConf = newConfiguration() - def getUserNameFromEnvironment(): String = { - // defaulting to env if -D is not present ... - val retval = System.getProperty(Environment.USER.name, System.getenv(Environment.USER.name)) - - // If nothing found, default to user we are running as - if (retval == null) System.getProperty("user.name") else retval - } - - def runAsUser(func: (Product) => Unit, args: Product) { - runAsUser(func, args, getUserNameFromEnvironment()) - } - - def runAsUser(func: (Product) => Unit, args: Product, user: String) { - 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. def isYarnMode(): Boolean = { val yarnMode = System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")) From e2d7656ca3d561f0a6fc8dd81ef46e4aa6ba608e Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 24 Jul 2013 14:01:48 -0700 Subject: [PATCH 0738/2521] re-enable YARN support --- core/src/main/scala/spark/SparkEnv.scala | 14 +++++++++- .../spark/deploy/yarn/ApplicationMaster.scala | 4 +-- .../main/scala/spark/deploy/yarn/Client.scala | 7 +++-- .../YarnSparkHadoopUtil.scala} | 26 +++++-------------- 4 files changed, 26 insertions(+), 25 deletions(-) rename yarn/src/main/scala/spark/deploy/{SparkHadoopUtil.scala => yarn/YarnSparkHadoopUtil.scala} (71%) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 73990f04237..5f71df33b68 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -61,9 +61,21 @@ class SparkEnv ( // If executorId is NOT found, return defaultHostPort var executorIdToHostPort: Option[(String, String) => String]) { - val hadoop = new SparkHadoopUtil private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() + val hadoop = { + val yarnMode = java.lang.Boolean.valueOf(System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))) + if(yarnMode) { + try { + Class.forName("spark.deploy.yarn.YarnSparkHadoopUtil").newInstance.asInstanceOf[SparkHadoopUtil] + } catch { + case th: Throwable => throw new SparkException("Unable to load YARN support", th) + } + } else { + new SparkHadoopUtil + } + } + def stop() { pythonWorkers.foreach { case(key, worker) => worker.stop() } httpFileServer.stop() diff --git a/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala index 1b06169739c..d69a969d42f 100644 --- a/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -130,11 +130,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e try { val socket = new Socket(driverHost, driverPort.toInt) socket.close() - logInfo("Master now available: " + driverHost + ":" + driverPort) + logInfo("Driver now available: " + driverHost + ":" + driverPort) driverUp = true } catch { case e: Exception => - logError("Failed to connect to driver at " + driverHost + ":" + driverPort) + logWarning("Failed to connect to driver at " + driverHost + ":" + driverPort + ", retrying") Thread.sleep(100) } } diff --git a/yarn/src/main/scala/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/spark/deploy/yarn/Client.scala index 8bcbfc27351..9d3860b863b 100644 --- a/yarn/src/main/scala/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/spark/deploy/yarn/Client.scala @@ -165,7 +165,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./*") Apps.addToEnvironment(env, Environment.CLASSPATH.name, "$CLASSPATH") Client.populateHadoopClasspath(yarnConf, env) - SparkHadoopUtil.setYarnMode(env) + env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_JAR_PATH") = localResources("spark.jar").getResource().getScheme.toString() + "://" + localResources("spark.jar").getResource().getFile().toString() @@ -313,8 +313,11 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl object Client { def main(argStrings: Array[String]) { + // Set an env variable indicating we are running in YARN mode. + // Note that anything with SPARK prefix gets propagated to all (remote) processes + System.setProperty("SPARK_YARN_MODE", "true") + val args = new ClientArguments(argStrings) - SparkHadoopUtil.setYarnMode() new Client(args).run } diff --git a/yarn/src/main/scala/spark/deploy/SparkHadoopUtil.scala b/yarn/src/main/scala/spark/deploy/yarn/YarnSparkHadoopUtil.scala similarity index 71% rename from yarn/src/main/scala/spark/deploy/SparkHadoopUtil.scala rename to yarn/src/main/scala/spark/deploy/yarn/YarnSparkHadoopUtil.scala index a812bcf8672..77c4ee7f3f6 100644 --- a/yarn/src/main/scala/spark/deploy/SparkHadoopUtil.scala +++ b/yarn/src/main/scala/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -15,8 +15,9 @@ * limitations under the License. */ -package spark.deploy +package spark.deploy.yarn +import spark.deploy.SparkHadoopUtil import collection.mutable.HashMap import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.UserGroupInformation @@ -28,32 +29,17 @@ import java.security.PrivilegedExceptionAction /** * Contains util methods to interact with Hadoop from spark. */ -object SparkHadoopUtil { - - val yarnConf = newConfiguration() +class YarnSparkHadoopUtil extends SparkHadoopUtil { // 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. - def isYarnMode(): Boolean = { - val yarnMode = System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")) - java.lang.Boolean.valueOf(yarnMode) - } - - // Set an env variable indicating we are running in YARN mode. - // Note that anything with SPARK prefix gets propagated to all (remote) processes - def setYarnMode() { - System.setProperty("SPARK_YARN_MODE", "true") - } - - def setYarnMode(env: HashMap[String, String]) { - env("SPARK_YARN_MODE") = "true" - } + override def isYarnMode(): Boolean = { true } // 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()) + override 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) { + override def addCredentials(conf: JobConf) { val jobCreds = conf.getCredentials(); jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) } From 8bb0bd11cea8a16d21c56c1b80b31a1c5605a414 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 24 Jul 2013 14:16:50 -0700 Subject: [PATCH 0739/2521] YARN ApplicationMaster shouldn't wait forever --- .../main/scala/spark/deploy/yarn/ApplicationMaster.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala index d69a969d42f..15dbd1c0fb2 100644 --- a/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -124,7 +124,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private def waitForSparkMaster() { logInfo("Waiting for spark driver to be reachable.") var driverUp = false - while(!driverUp) { + var tries = 0 + while(!driverUp && tries < 10) { val driverHost = System.getProperty("spark.driver.host") val driverPort = System.getProperty("spark.driver.port") try { @@ -136,6 +137,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e case e: Exception => logWarning("Failed to connect to driver at " + driverHost + ":" + driverPort + ", retrying") Thread.sleep(100) + tries = tries + 1 } } } @@ -176,7 +178,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e var sparkContext: SparkContext = null ApplicationMaster.sparkContextRef.synchronized { var count = 0 - while (ApplicationMaster.sparkContextRef.get() == null) { + while (ApplicationMaster.sparkContextRef.get() == null && count < 10) { logInfo("Waiting for spark context initialization ... " + count) count = count + 1 ApplicationMaster.sparkContextRef.wait(10000L) From 14b6bcdf93642624c42fa04aeaff9fff97f6e07f Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Mon, 29 Jul 2013 16:04:48 -0700 Subject: [PATCH 0740/2521] update YARN docs --- 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 66fb8d73e80..9c2cedfd886 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -55,7 +55,7 @@ This would be used to connect to the cluster, write to the dfs and submit jobs t The command to launch the YARN Client is as follows: - SPARK_JAR= ./run spark.deploy.yarn.Client \ + SPARK_JAR= ./run spark.deploy.yarn.Client \ --jar \ --class \ --args \ @@ -68,7 +68,7 @@ The command to launch the YARN Client is as follows: For example: - SPARK_JAR=./core/target/spark-core-assembly-{{site.SPARK_VERSION}}.jar ./run spark.deploy.yarn.Client \ + SPARK_JAR=./yarn/target/spark-yarn-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 yarn-standalone \ From 8f979edef5b80967b81323e13dcafd5aac92feb1 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Tue, 6 Aug 2013 15:47:49 -0700 Subject: [PATCH 0741/2521] Fix newTaskAttemptID to work under YARN --- .../mapreduce/SparkHadoopMapReduceUtil.scala | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala b/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala index bea62536771..93180307fa3 100644 --- a/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala +++ b/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala @@ -18,6 +18,7 @@ package org.apache.hadoop.mapreduce import org.apache.hadoop.conf.Configuration +import java.lang.{Integer => JInteger, Boolean => JBoolean} trait SparkHadoopMapReduceUtil { def newJobContext(conf: Configuration, jobId: JobID): JobContext = { @@ -37,7 +38,24 @@ trait SparkHadoopMapReduceUtil { } def newTaskAttemptID(jtIdentifier: String, jobId: Int, isMap: Boolean, taskId: Int, attemptId: Int) = { - new TaskAttemptID(jtIdentifier, jobId, isMap, taskId, attemptId) + val klass = Class.forName("org.apache.hadoop.mapreduce.TaskAttemptID"); + try { + // first, attempt to use the old-style constructor that takes a boolean isMap (not available in YARN) + val ctor = klass.getDeclaredConstructor(classOf[String], classOf[Int], classOf[Boolean], + classOf[Int], classOf[Int]) + ctor.newInstance(jtIdentifier, new JInteger(jobId), new JBoolean(isMap), new JInteger(taskId), new + JInteger(attemptId)).asInstanceOf[TaskAttemptID] + } catch { + case exc: NoSuchMethodException => { + // failed, look for the new ctor that takes a TaskType (not available in 1.x) + val taskTypeClass = Class.forName("org.apache.hadoop.mapreduce.TaskType").asInstanceOf[Class[Enum[_]]] + val taskType = taskTypeClass.getMethod("valueOf", classOf[String]).invoke(taskTypeClass, if(isMap) "MAP" else "REDUCE") + val ctor = klass.getDeclaredConstructor(classOf[String], classOf[Int], taskTypeClass, + classOf[Int], classOf[Int]) + ctor.newInstance(jtIdentifier, new JInteger(jobId), taskType, new JInteger(taskId), new + JInteger(attemptId)).asInstanceOf[TaskAttemptID] + } + } } private def firstAvailableClass(first: String, second: String): Class[_] = { From a06a9d5c5fd6584a57292a0115253e0a8a45d490 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Tue, 6 Aug 2013 16:16:07 -0700 Subject: [PATCH 0742/2521] Rename HadoopWriter to SparkHadoopWriter since it's outside of our package --- core/src/main/scala/spark/PairRDDFunctions.scala | 6 +++--- .../spark/{HadoopWriter.scala => SparkHadoopWriter.scala} | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) rename core/src/main/scala/spark/{HadoopWriter.scala => SparkHadoopWriter.scala} (96%) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index aeeac65ccaa..6701f24ff96 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -32,7 +32,7 @@ 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 +import org.apache.hadoop.mapred.SparkHadoopWriter import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.OutputFormat @@ -653,7 +653,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( conf.set("mapred.output.compression.type", CompressionType.BLOCK.toString) } conf.setOutputCommitter(classOf[FileOutputCommitter]) - FileOutputFormat.setOutputPath(conf, HadoopWriter.createPathFromString(path, conf)) + FileOutputFormat.setOutputPath(conf, SparkHadoopWriter.createPathFromString(path, conf)) saveAsHadoopDataset(conf) } @@ -679,7 +679,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( logInfo("Saving as hadoop file of type (" + keyClass.getSimpleName+ ", " + valueClass.getSimpleName+ ")") - val writer = new HadoopWriter(conf) + val writer = new SparkHadoopWriter(conf) writer.preSetup() def writeToFile(context: TaskContext, iter: Iterator[(K,V)]) { diff --git a/core/src/main/scala/spark/HadoopWriter.scala b/core/src/main/scala/spark/SparkHadoopWriter.scala similarity index 96% rename from core/src/main/scala/spark/HadoopWriter.scala rename to core/src/main/scala/spark/SparkHadoopWriter.scala index 60840ce77ec..6b330ef5727 100644 --- a/core/src/main/scala/spark/HadoopWriter.scala +++ b/core/src/main/scala/spark/SparkHadoopWriter.scala @@ -36,7 +36,7 @@ import spark.SerializableWritable * Saves the RDD using a JobConf, which should contain an output key class, an output value class, * a filename to write to, etc, exactly like in a Hadoop MapReduce job. */ -class HadoopWriter(@transient jobConf: JobConf) extends Logging with SparkHadoopMapRedUtil with Serializable { +class SparkHadoopWriter(@transient jobConf: JobConf) extends Logging with SparkHadoopMapRedUtil with Serializable { private val now = new Date() private val conf = new SerializableWritable(jobConf) @@ -165,7 +165,7 @@ class HadoopWriter(@transient jobConf: JobConf) extends Logging with SparkHadoop splitID = splitid attemptID = attemptid - jID = new SerializableWritable[JobID](HadoopWriter.createJobID(now, jobid)) + jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobid)) taID = new SerializableWritable[TaskAttemptID]( new TaskAttemptID(new TaskID(jID.value, true, splitID), attemptID)) } @@ -179,7 +179,7 @@ class HadoopWriter(@transient jobConf: JobConf) extends Logging with SparkHadoop } } -object HadoopWriter { +object SparkHadoopWriter { def createJobID(time: Date, id: Int): JobID = { val formatter = new SimpleDateFormat("yyyyMMddHHmm") val jobtrackerID = formatter.format(new Date()) From a0f08484636395394ac2e0d22ca0ca3e1606664c Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 14 Aug 2013 15:36:12 -0700 Subject: [PATCH 0743/2521] Update default version of Hadoop to 1.2.1 --- pom.xml | 2 +- project/SparkBuild.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 7e6d38df9f9..135b18bd267 100644 --- a/pom.xml +++ b/pom.xml @@ -540,7 +540,7 @@ org.apache.hadoop hadoop-core - 1.0.4 + 1.2.1 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ede49ea38b4..350a36a9643 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -170,7 +170,7 @@ object SparkBuild extends Build { "org.apache.mesos" % "mesos" % "0.12.1", "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test", - "org.apache.hadoop" % "hadoop-client" % "1.0.4" excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-client" % "1.2.1" excludeAll(excludeJackson, excludeNetty, excludeAsm), "com.codahale.metrics" % "metrics-core" % "3.0.0", "com.codahale.metrics" % "metrics-jvm" % "3.0.0", "com.codahale.metrics" % "metrics-json" % "3.0.0", From eddbf43b548897dc9602635f5f02145ad288e6ec Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 15 Aug 2013 17:49:37 -0700 Subject: [PATCH 0744/2521] Revert "Merge pull request #834 from Daemoen/master" This reverts commit 230ab2722ebd399afcf64c1a131f4929f602177d, reversing changes made to 659553b21ddd7504889ce113a816c1db4a73f167. --- core/src/main/scala/spark/deploy/JsonProtocol.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index 9901bb8a8cf..bd1db7c2940 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -33,8 +33,7 @@ private[spark] object JsonProtocol { ("cores" -> obj.cores) ~ ("coresused" -> obj.coresUsed) ~ ("memory" -> obj.memory) ~ - ("memoryused" -> obj.memoryUsed) ~ - ("state" -> obj.getState) + ("memoryused" -> obj.memoryUsed) } def writeApplicationInfo(obj: ApplicationInfo) = { From c961c19b7b4077a6308d4564e4d182933bc38b6d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 15 Aug 2013 18:23:01 -0700 Subject: [PATCH 0745/2521] Use the JSON formatter from Scala library and removed dependency on lift-json. It made the JSON creation slightly more complicated, but reduces one external dependency. The scala library also properly escape "/" (which lift-json doesn't). --- core/pom.xml | 4 - .../scala/spark/deploy/JsonProtocol.scala | 108 +++++++++--------- .../deploy/master/ui/ApplicationPage.scala | 5 +- .../spark/deploy/master/ui/IndexPage.scala | 5 +- .../spark/deploy/worker/ui/IndexPage.scala | 5 +- core/src/main/scala/spark/ui/JettyUtils.scala | 7 +- pom.xml | 5 - project/SparkBuild.scala | 1 - 8 files changed, 64 insertions(+), 76 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index dfadd22d421..73426a9ec58 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -90,10 +90,6 @@ org.scala-lang scala-library - - net.liftweb - lift-json_2.9.2 - it.unimi.dsi fastutil diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index bd1db7c2940..6b71b953dd7 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -17,7 +17,7 @@ package spark.deploy -import net.liftweb.json.JsonDSL._ +import scala.util.parsing.json.{JSONArray, JSONObject, JSONType} import spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} import spark.deploy.master.{ApplicationInfo, WorkerInfo} @@ -25,61 +25,63 @@ import spark.deploy.worker.ExecutorRunner private[spark] 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) - } - 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) - } + def writeWorkerInfo(obj: WorkerInfo): JSONType = JSONObject(Map( + "id" -> obj.id, + "host" -> obj.host, + "port" -> obj.port, + "webuiaddress" -> obj.webUiAddress, + "cores" -> obj.cores, + "coresused" -> obj.coresUsed, + "memory" -> obj.memory, + "memoryused" -> obj.memoryUsed, + "state" -> obj.state.toString + )) - def writeApplicationDescription(obj: ApplicationDescription) = { - ("name" -> obj.name) ~ - ("cores" -> obj.maxCores) ~ - ("memoryperslave" -> obj.memoryPerSlave) ~ - ("user" -> obj.user) - } + def writeApplicationInfo(obj: ApplicationInfo): JSONType = JSONObject(Map( + "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 + )) - def writeExecutorRunner(obj: ExecutorRunner) = { - ("id" -> obj.execId) ~ - ("memory" -> obj.memory) ~ - ("appid" -> obj.appId) ~ - ("appdesc" -> writeApplicationDescription(obj.appDesc)) - } + def writeApplicationDescription(obj: ApplicationDescription): JSONType = JSONObject(Map( + "name" -> obj.name, + "cores" -> obj.maxCores, + "memoryperslave" -> obj.memoryPerSlave, + "user" -> obj.user + )) - def writeMasterState(obj: MasterStateResponse) = { - ("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)) - } + def writeExecutorRunner(obj: ExecutorRunner): JSONType = JSONObject(Map( + "id" -> obj.execId, + "memory" -> obj.memory, + "appid" -> obj.appId, + "appdesc" -> writeApplicationDescription(obj.appDesc) + )) - def writeWorkerState(obj: WorkerStateResponse) = { - ("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)) - } + def writeMasterState(obj: MasterStateResponse): JSONType = JSONObject(Map( + "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" -> JSONArray(obj.activeApps.toList.map(writeApplicationInfo)), + "completedapps" -> JSONArray(obj.completedApps.toList.map(writeApplicationInfo)) + )) + + def writeWorkerState(obj: WorkerStateResponse): JSONType = JSONObject(Map( + "id" -> obj.workerId, + "masterurl" -> obj.masterUrl, + "masterwebuiurl" -> obj.masterWebUiUrl, + "cores" -> obj.cores, + "coresused" -> obj.coresUsed, + "memory" -> obj.memory, + "memoryused" -> obj.memoryUsed, + "executors" -> JSONArray(obj.executors.toList.map(writeExecutorRunner)), + "finishedexecutors" -> JSONArray(obj.finishedExecutors.toList.map(writeExecutorRunner)) + )) } 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 405a1ec3a60..494a9b914d7 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -17,6 +17,7 @@ package spark.deploy.master.ui +import scala.util.parsing.json.JSONType import scala.xml.Node import akka.dispatch.Await @@ -25,8 +26,6 @@ import akka.util.duration._ import javax.servlet.http.HttpServletRequest -import net.liftweb.json.JsonAST.JValue - import spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import spark.deploy.JsonProtocol import spark.deploy.master.ExecutorInfo @@ -37,7 +36,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { implicit val timeout = parent.timeout /** Executor details for a particular application */ - def renderJson(request: HttpServletRequest): JValue = { + def renderJson(request: HttpServletRequest): JSONType = { val appId = request.getParameter("appId") val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, 30 seconds) 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 2000211b983..28e421e3bc1 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -19,14 +19,13 @@ package spark.deploy.master.ui import javax.servlet.http.HttpServletRequest +import scala.util.parsing.json.JSONType import scala.xml.Node import akka.dispatch.Await import akka.pattern.ask import akka.util.duration._ -import net.liftweb.json.JsonAST.JValue - import spark.Utils import spark.deploy.DeployWebUI import spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} @@ -38,7 +37,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { val master = parent.masterActorRef implicit val timeout = parent.timeout - def renderJson(request: HttpServletRequest): JValue = { + def renderJson(request: HttpServletRequest): JSONType = { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, 30 seconds) JsonProtocol.writeMasterState(state) 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 b67059068bf..02993d58a0a 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -19,14 +19,13 @@ package spark.deploy.worker.ui import javax.servlet.http.HttpServletRequest +import scala.util.parsing.json.JSONType import scala.xml.Node import akka.dispatch.Await import akka.pattern.ask import akka.util.duration._ -import net.liftweb.json.JsonAST.JValue - import spark.Utils import spark.deploy.JsonProtocol import spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} @@ -39,7 +38,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { val worker = parent.worker val timeout = parent.timeout - def renderJson(request: HttpServletRequest): JValue = { + def renderJson(request: HttpServletRequest): JSONType = { val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] val workerState = Await.result(stateFuture, 30 seconds) JsonProtocol.writeWorkerState(workerState) diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index f66fe399052..ba58f357299 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -21,10 +21,9 @@ import javax.servlet.http.{HttpServletResponse, HttpServletRequest} import scala.annotation.tailrec import scala.util.{Try, Success, Failure} +import scala.util.parsing.json.JSONType import scala.xml.Node -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 @@ -39,8 +38,8 @@ private[spark] object JettyUtils extends Logging { 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 jsonResponderToHandler(responder: Responder[JSONType]): Handler = + createHandler(responder, "text/json", (in: JSONType) => in.toString) implicit def htmlResponderToHandler(responder: Responder[Seq[Node]]): Handler = createHandler(responder, "text/html", (in: Seq[Node]) => "" + in.toString) diff --git a/pom.xml b/pom.xml index 7e6d38df9f9..1811c62b55a 100644 --- a/pom.xml +++ b/pom.xml @@ -254,11 +254,6 @@ 10.4.2.0 test - - net.liftweb - lift-json_2.9.2 - 2.5 - com.codahale.metrics metrics-core diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f6519c82874..852f40d3fd6 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -180,7 +180,6 @@ object SparkBuild extends Build { "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", "org.apache.mesos" % "mesos" % "0.12.1", "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test", From a5193a8fac78f8860ae1dc3969e9a8528a1d292c Mon Sep 17 00:00:00 2001 From: seanm Date: Thu, 15 Aug 2013 20:52:58 -0600 Subject: [PATCH 0746/2521] fixing typo --- conf/slaves | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/conf/slaves b/conf/slaves index 6e315a85402..da0a01343d2 100644 --- a/conf/slaves +++ b/conf/slaves @@ -1,2 +1,2 @@ -# A Spark Worker will be started on each of the machines listes below. +# A Spark Worker will be started on each of the machines listed below. localhost \ No newline at end of file From 8fc40818d714651c0fb360a26b64a3ab12559961 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 15 Aug 2013 23:08:48 -0700 Subject: [PATCH 0747/2521] Fix --- mllib/src/main/scala/spark/mllib/recommendation/ALS.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index aeacbca5108..9097f46db90 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -133,9 +133,9 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l var products = productOutLinks.mapPartitions {itr => val rand = new Random() itr.map({case (x, y) => - (x, y.elementIds.map(u => randomFactor(rank, rand))) + (x, y.elementIds.map(u => randomFactor(rank, rand))) }) - }) + } for (iter <- 0 until iterations) { // perform ALS update From c7e348faec45ad1d996d16639015c4bc4fc3bc92 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Thu, 15 Aug 2013 16:01:19 -0700 Subject: [PATCH 0748/2521] Implementing SPARK-878 for PySpark: adding zip and egg files to context and passing it down to workers which add these to their sys.path --- .../main/scala/spark/api/python/PythonRDD.scala | 9 ++++++++- python/pyspark/context.py | 14 +++++++++++--- python/pyspark/rdd.py | 4 +++- python/pyspark/tests.py | 11 +++++++++++ python/pyspark/worker.py | 13 ++++++++++++- python/test_support/userlib-0.1-py2.7.egg | Bin 0 -> 1945 bytes 6 files changed, 45 insertions(+), 6 deletions(-) create mode 100644 python/test_support/userlib-0.1-py2.7.egg diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 2dd79f7100e..49671437d04 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -33,6 +33,7 @@ private[spark] class PythonRDD[T: ClassManifest]( parent: RDD[T], command: Seq[String], envVars: JMap[String, String], + pythonIncludes: JList[String], preservePartitoning: Boolean, pythonExec: String, broadcastVars: JList[Broadcast[Array[Byte]]], @@ -44,10 +45,11 @@ 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: JMap[String, String], + pythonIncludes: JList[String], preservePartitoning: Boolean, pythonExec: String, broadcastVars: JList[Broadcast[Array[Byte]]], accumulator: Accumulator[JList[Array[Byte]]]) = - this(parent, PipedRDD.tokenize(command), envVars, preservePartitoning, pythonExec, + this(parent, PipedRDD.tokenize(command), envVars, pythonIncludes, preservePartitoning, pythonExec, broadcastVars, accumulator) override def getPartitions = parent.partitions @@ -79,6 +81,11 @@ private[spark] class PythonRDD[T: ClassManifest]( dataOut.writeInt(broadcast.value.length) dataOut.write(broadcast.value) } + // Python includes (*.zip and *.egg files) + dataOut.writeInt(pythonIncludes.length) + for (f <- pythonIncludes) { + PythonRDD.writeAsPickle(f, dataOut) + } dataOut.flush() // Serialized user code for (elem <- command) { diff --git a/python/pyspark/context.py b/python/pyspark/context.py index c2b49ff37ac..2803ce90f3e 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -46,6 +46,7 @@ class SparkContext(object): _next_accum_id = 0 _active_spark_context = None _lock = Lock() + _python_includes = None # zip and egg files that need to be added to PYTHONPATH def __init__(self, master, jobName, sparkHome=None, pyFiles=None, environment=None, batchSize=1024): @@ -103,11 +104,14 @@ def __init__(self, master, jobName, sparkHome=None, pyFiles=None, # send. self._pickled_broadcast_vars = set() + SparkFiles._sc = self + root_dir = SparkFiles.getRootDirectory() + sys.path.append(root_dir) + # Deploy any code dependencies specified in the constructor + self._python_includes = list() for path in (pyFiles or []): self.addPyFile(path) - SparkFiles._sc = self - sys.path.append(SparkFiles.getRootDirectory()) # Create a temporary directory inside spark.local.dir: local_dir = self._jvm.spark.Utils.getLocalDir() @@ -257,7 +261,11 @@ def addPyFile(self, path): HTTP, HTTPS or FTP URI. """ self.addFile(path) - filename = path.split("/")[-1] + (dirname, filename) = os.path.split(path) # dirname may be directory or HDFS/S3 prefix + + if filename.endswith('.zip') or filename.endswith('.ZIP') or filename.endswith('.egg'): + self._python_includes.append(filename) + sys.path.append(os.path.join(SparkFiles.getRootDirectory(), filename)) # for tests in local mode def setCheckpointDir(self, dirName, useExisting=False): """ diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 51c2cb98067..99f5967a8ed 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -758,8 +758,10 @@ def batched_func(split, iterator): class_manifest = self._prev_jrdd.classManifest() env = MapConverter().convert(self.ctx.environment, self.ctx._gateway._gateway_client) + includes = ListConverter().convert(self.ctx._python_includes, + self.ctx._gateway._gateway_client) python_rdd = self.ctx._jvm.PythonRDD(self._prev_jrdd.rdd(), - pipe_command, env, self.preservesPartitioning, self.ctx.pythonExec, + pipe_command, env, includes, self.preservesPartitioning, self.ctx.pythonExec, broadcast_vars, self.ctx._javaAccumulator, class_manifest) self._jrdd_val = python_rdd.asJavaRDD() return self._jrdd_val diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index f75215a7813..29d6a128f6a 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -125,6 +125,17 @@ def func(): from userlibrary import UserClass self.assertEqual("Hello World!", UserClass().hello()) + def test_add_egg_file_locally(self): + # To ensure that we're actually testing addPyFile's effects, check that + # this fails due to `userlibrary` not being on the Python path: + def func(): + from userlib import UserClass + self.assertRaises(ImportError, func) + path = os.path.join(SPARK_HOME, "python/test_support/userlib-0.1-py2.7.egg") + self.sc.addPyFile(path) + from userlib import UserClass + self.assertEqual("Hello World from inside a package!", UserClass().hello()) + class TestIO(PySparkTestCase): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 75d692beeb0..695f6dfb844 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -49,15 +49,26 @@ def main(infile, outfile): split_index = read_int(infile) if split_index == -1: # for unit tests return + + # fetch name of workdir 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) + + # fetch names and values of broadcast variables num_broadcast_variables = read_int(infile) for _ in range(num_broadcast_variables): bid = read_long(infile) value = read_with_length(infile) _broadcastRegistry[bid] = Broadcast(bid, load_pickle(value)) + + # fetch names of includes (*.zip and *.egg files) and construct PYTHONPATH + sys.path.append(spark_files_dir) # *.py files that were added will be copied here + num_python_includes = read_int(infile) + for _ in range(num_python_includes): + sys.path.append(os.path.join(spark_files_dir, load_pickle(read_with_length(infile)))) + + # now load function func = load_obj(infile) bypassSerializer = load_obj(infile) if bypassSerializer: diff --git a/python/test_support/userlib-0.1-py2.7.egg b/python/test_support/userlib-0.1-py2.7.egg new file mode 100644 index 0000000000000000000000000000000000000000..1674c9cb2227e160aec55c74e9ba1ddba850f580 GIT binary patch literal 1945 zcmWIWW@Zs#U|`^2Sdq%-{OPC4zkVQZ3lQ@IacOaCQBG!(er{rBo?bzvZr?^OCPM+2 z-!(^Pyju5>Ifws9kif>ilh5?x&a}Ph`t|+UfpA{q4-(mHcXF|6ID3k0S`rmJ^Pu27 zX2pP|PDj;+_Dt5>5pw-nXZI!d?|Lntzt1q@eSGhRp0S;^U`hJyZ6+W7I>sGuc2o~I z;$-<*VantEcQTf&4@>aB-66eQMnKwrV%L&?hZb7x-!-%B+8>4hxZ7LO_?&A_9oL=$ zbn|W?2Kfl)_W1bByv&mLc%b`}lmDGLz4eV;d4LhKhymk9W&!bThE5TMq?8kh2`5rh zI+9Wnd=is5t|hQ}rTk-O;&oG4-6HJKBDP%2^0|tb`0vbyFVikRj(5)S6>%#*g z4>4Xznml`c(5%T>?3=boqzIt-qG4rZelQ~gLjn^6g8-5*pfQlVbi#eF!v-Slm#_J* zt#~NyvVK)UwfP*@J=O@3Im_WVolA58N~>g61`pSxe0_vE)< z^ZdsjUjEk5J^74tL*7Q7qYqkq3~ia@R@xk@{p5N=T`@m+{)dK?i}POB9N~!PDabP2 znxJE9d~uu8)sXNlS7bEW#Y9#FH7t}rbS~rCraZfArw&~@*7N!K)x8r0<{n>Yd*t

    waS|`|O?l3T$=A=7qdQyq0*MfDE zS-ZRgRd|K9^1{m+-Ws>d>^fX$@A>54M%x1?ijPX2D3<;<+wF7fk=$4Lh4pO9-@=?k}n_0dd;3x|M^$py6en$SghCm=SC!>>%c6)9BG#!3k-N^AQnc7 z0HkE(t$pfz=n3C5XM#_h(LJHB7099Mr(1N+Q%CO^6IygS{lCb@1vG^Nh{b@|)!kj! z)6dOcza+mPJ}0#-HAkN)2L?f8-Bp`(* z^5{v|nXIY_yAzriEjv{+f4c7E5CpoBkx7IZ6hzp|O`rrAY-t2hu#yOxhmkd7E4Uz9 zfrW769wg03=`R`G1oT1!VL}ry>7ZGUq8nR^N9bk-CO(*MB>T~=M^EGk0|J2tz!MQl zl1DcKJ*gwi=tnjKmhkau2c>%$*wT0qSv$5|fNm6eCO{bVAK56REP-wUdYVO;Fo^{z YJYXp}z?+o~q=XX)eSyWk87qhf04R`+K>z>% literal 0 HcmV?d00001 From 53b2639a1e26e94f4a4fa8856f462b281c90d8b1 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 15 Aug 2013 17:14:58 -0700 Subject: [PATCH 0749/2521] Reversing the argument order in zipPartitions to enable stronger type inference. --- core/src/main/scala/spark/RDD.scala | 21 ++++++++----------- .../scala/spark/api/java/JavaRDDLike.scala | 6 +++--- core/src/test/scala/spark/JavaAPISuite.java | 2 +- .../scala/spark/ZippedPartitionsSuite.scala | 2 +- 4 files changed, 14 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 160b3e9d837..503ea6ccbf4 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -515,22 +515,19 @@ abstract class RDD[T: ClassManifest]( * *same number of partitions*, but does *not* require them to have the same number * of elements in each partition. */ - def zipPartitions[B: ClassManifest, V: ClassManifest]( - f: (Iterator[T], Iterator[B]) => Iterator[V], - rdd2: RDD[B]): RDD[V] = + def zipPartitions[B: ClassManifest, V: ClassManifest] + (rdd2: RDD[B]) + (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2) - def zipPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest]( - f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V], - rdd2: RDD[B], - rdd3: RDD[C]): RDD[V] = + def zipPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest] + (rdd2: RDD[B], rdd3: RDD[C]) + (f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] = new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3) - def zipPartitions[B: ClassManifest, C: ClassManifest, D: ClassManifest, V: ClassManifest]( - f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], - rdd2: RDD[B], - rdd3: RDD[C], - rdd4: RDD[D]): RDD[V] = + def zipPartitions[B: ClassManifest, C: ClassManifest, D: ClassManifest, V: ClassManifest] + (rdd2: RDD[B], rdd3: RDD[C], rdd4: RDD[D]) + (f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] = new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4) diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index e0255ed23ea..2c2b138f16c 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -207,12 +207,12 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * 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] = { + other: JavaRDDLike[U, _], + f: FlatMapFunction2[java.util.Iterator[T], java.util.Iterator[U], V]): 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()) + rdd.zipPartitions(other.rdd)(fn)(other.classManifest, f.elementType()))(f.elementType()) } // Actions (launch a job to return a value to the user program) diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 4ab271de1ab..c337c49268a 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -748,7 +748,7 @@ public Iterable call(Iterator i, Iterator s) { } }; - JavaRDD sizes = rdd1.zipPartitions(sizesFn, rdd2); + JavaRDD sizes = rdd1.zipPartitions(rdd2, sizesFn); Assert.assertEquals("[3, 2, 3, 2]", sizes.collect().toString()); } diff --git a/core/src/test/scala/spark/ZippedPartitionsSuite.scala b/core/src/test/scala/spark/ZippedPartitionsSuite.scala index 5e6d7b09d8d..bb5d379273b 100644 --- a/core/src/test/scala/spark/ZippedPartitionsSuite.scala +++ b/core/src/test/scala/spark/ZippedPartitionsSuite.scala @@ -40,7 +40,7 @@ class ZippedPartitionsSuite extends FunSuite with SharedSparkContext { val data2 = sc.makeRDD(Array("1", "2", "3", "4", "5", "6"), 2) val data3 = sc.makeRDD(Array(1.0, 2.0), 2) - val zippedRDD = data1.zipPartitions(ZippedPartitionsSuite.procZippedData, data2, data3) + val zippedRDD = data1.zipPartitions(data2, data3)(ZippedPartitionsSuite.procZippedData) val obtainedSizes = zippedRDD.collect() val expectedSizes = Array(2, 3, 1, 2, 3, 1) From 3f98eff63a3df35f6dc56f0786c828cdbe4ffcf1 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 14 Aug 2013 17:34:34 -0700 Subject: [PATCH 0750/2521] Allow make-distribution.sh to specify Hadoop version used --- make-distribution.sh | 53 +++++++++++++++++++++++++++++++--------- project/SparkBuild.scala | 28 ++++++++++++++++----- 2 files changed, 64 insertions(+), 17 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index 0a8941c1f85..a101024de54 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -24,9 +24,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 +# Optional Arguments +# --tgz: Additionally creates spark-$VERSION-bin.tar.gz +# --hadoop VERSION: Builds against specified version of Hadoop. +# --with-yarn: Enables support for Hadoop YARN. # # Recommended deploy/testing procedure (standalone mode): # 1) Rsync / deploy the dist/ dir to one host @@ -44,20 +45,50 @@ DISTDIR="$FWDIR/dist" export TERM=dumb # Prevents color codes in SBT output 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" +# Initialize defaults +SPARK_HADOOP_VERSION=1.2.1 +SPARK_YARN_MODE=false +MAKE_TGZ=false + +# Parse arguments +while (( "$#" )); do + case $1 in + --hadoop) + SPARK_HADOOP_VERSION="$2" + shift + ;; + --with-yarn) + SPARK_YARN_MODE=true + ;; + --tgz) + MAKE_TGZ=true + ;; + esac + shift +done + +if [ "$MAKE_TGZ" == "true" ]; then + echo "Making spark-$VERSION-hadoop_$SPARK_HADOOP_VERSION-bin.tar.gz" else echo "Making distribution for Spark $VERSION in $DISTDIR..." fi +echo "Hadoop version set to $SPARK_HADOOP_VERSION" +if [ "$SPARK_YARN_MODE" == "true" ]; then + echo "YARN enabled" +else + echo "YARN disabled" +fi # Build fat JAR -$FWDIR/sbt/sbt "repl/assembly" +export SPARK_HADOOP_VERSION +export SPARK_YARN_MODE +"$FWDIR/sbt/sbt" "repl/assembly" # Make directories rm -rf "$DISTDIR" mkdir -p "$DISTDIR/jars" -echo "$VERSION" >$DISTDIR/RELEASE +echo "$VERSION" > "$DISTDIR/RELEASE" # Copy jars cp $FWDIR/repl/target/*.jar "$DISTDIR/jars/" @@ -69,9 +100,9 @@ cp "$FWDIR/run" "$FWDIR/spark-shell" "$DISTDIR" cp "$FWDIR/spark-executor" "$DISTDIR" -if [ "$1" == "tgz" ]; then +if [ "$MAKE_TGZ" == "true" ]; then TARDIR="$FWDIR/spark-$VERSION" - cp -r $DISTDIR $TARDIR - tar -zcf spark-$VERSION-bin.tar.gz -C $FWDIR spark-$VERSION - rm -rf $TARDIR + cp -r "$DISTDIR" "$TARDIR" + tar -zcf "spark-$VERSION-hadoop_$SPARK_HADOOP_VERSION-bin.tar.gz" -C "$FWDIR" "spark-$VERSION" + rm -rf "$TARDIR" fi diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 350a36a9643..23c71799192 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -24,10 +24,15 @@ import AssemblyKeys._ //import com.jsuereth.pgp.sbtplugin.PgpKeys._ object SparkBuild extends Build { + // Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or + // "1.0.4" for Apache releases, or "0.20.2-cdh3u5" for Cloudera Hadoop. + val HADOOP_VERSION = "1.2.1" + val HADOOP_YARN = false + // HBase version; set as appropriate. val HBASE_VERSION = "0.94.6" - lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming, mllib, tools, yarn) + lazy val root = Project("root", file("."), settings = rootSettings) aggregate(allProjects:_*) lazy val core = Project("core", file("core"), settings = coreSettings) @@ -49,6 +54,17 @@ object SparkBuild extends Build { lazy val MavenCompile = config("m2r") extend(Compile) lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") + // Allows build configuration to be set through environment variables + lazy val hadoopVersion = scala.util.Properties.envOrElse("SPARK_HADOOP_VERSION", HADOOP_VERSION) + lazy val isYarnMode = scala.util.Properties.envOrNone("SPARK_YARN_MODE") match { + case None => HADOOP_YARN + case Some(v) => v.toBoolean + } + + // Conditionally include the yarn sub-project + lazy val maybeYarn = if(isYarnMode) Seq[ProjectReference](yarn) else Seq[ProjectReference]() + lazy val allProjects = Seq[ProjectReference](core, repl, examples, bagel, streaming, mllib, tools) ++ maybeYarn + def sharedSettings = Defaults.defaultSettings ++ Seq( organization := "org.spark-project", version := "0.8.0-SNAPSHOT", @@ -170,7 +186,7 @@ object SparkBuild extends Build { "org.apache.mesos" % "mesos" % "0.12.1", "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test", - "org.apache.hadoop" % "hadoop-client" % "1.2.1" excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm), "com.codahale.metrics" % "metrics-core" % "3.0.0", "com.codahale.metrics" % "metrics-jvm" % "3.0.0", "com.codahale.metrics" % "metrics-json" % "3.0.0", @@ -239,10 +255,10 @@ object SparkBuild extends Build { name := "spark-yarn", libraryDependencies ++= Seq( // Exclude rule required for all ? - "org.apache.hadoop" % "hadoop-client" % "2.0.2-alpha" excludeAll(excludeJackson, excludeNetty, excludeAsm), - "org.apache.hadoop" % "hadoop-yarn-api" % "2.0.2-alpha" excludeAll(excludeJackson, excludeNetty, excludeAsm), - "org.apache.hadoop" % "hadoop-yarn-common" % "2.0.2-alpha" excludeAll(excludeJackson, excludeNetty, excludeAsm), - "org.apache.hadoop" % "hadoop-yarn-client" % "2.0.2-alpha" excludeAll(excludeJackson, excludeNetty, excludeAsm) + "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm) ) ) ++ assemblySettings ++ extraAssemblySettings From 8add2d7a59c59e72539da86a58b9c2980843f1e0 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 14 Aug 2013 17:49:42 -0700 Subject: [PATCH 0751/2521] Fix repl/assembly when YARN enabled --- project/SparkBuild.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 23c71799192..fa9ec7deca9 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -36,7 +36,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(bagel) dependsOn(mllib) + lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn(core) dependsOn(bagel) dependsOn(mllib) dependsOn(maybeYarn:_*) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) dependsOn (streaming) dependsOn(mllib) @@ -62,8 +62,9 @@ object SparkBuild extends Build { } // Conditionally include the yarn sub-project - lazy val maybeYarn = if(isYarnMode) Seq[ProjectReference](yarn) else Seq[ProjectReference]() - lazy val allProjects = Seq[ProjectReference](core, repl, examples, bagel, streaming, mllib, tools) ++ maybeYarn + lazy val maybeYarn = if(isYarnMode) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]() + lazy val maybeYarnRef = if(isYarnMode) Seq[ProjectReference](yarn) else Seq[ProjectReference]() + lazy val allProjects = Seq[ProjectReference](core, repl, examples, bagel, streaming, mllib, tools) ++ maybeYarnRef def sharedSettings = Defaults.defaultSettings ++ Seq( organization := "org.spark-project", From 353fab2440dbf1369df20393e0377de2b327de72 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Thu, 15 Aug 2013 12:10:31 -0700 Subject: [PATCH 0752/2521] Initial changes to make Maven build agnostic of hadoop version --- assembly/pom.xml | 8 +-- bagel/pom.xml | 43 ++++---------- core/pom.xml | 63 ++------------------ examples/pom.xml | 144 +++++++++++++++++++--------------------------- mllib/pom.xml | 43 ++++---------- pom.xml | 32 ++++------- repl-bin/pom.xml | 76 ++++++++++-------------- repl/pom.xml | 72 +++++++---------------- streaming/pom.xml | 42 ++++---------- tools/pom.xml | 53 +++++------------ 10 files changed, 175 insertions(+), 401 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index cc5a4875af8..76ac9f54780 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -38,12 +38,6 @@ - - hadoop1 - - hadoop1 - - hadoop2 @@ -89,4 +83,4 @@ ${project.version} - \ No newline at end of file + diff --git a/bagel/pom.xml b/bagel/pom.xml index 60bbc49e6c5..3c82af3b335 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -32,11 +32,20 @@ http://spark-project.org/ + + org.spark-project + spark-core + ${project.version} + + + org.apache.hadoop + hadoop-client + provided + org.eclipse.jetty jetty-server - org.scalatest scalatest_${scala.version} @@ -60,33 +69,6 @@ - - hadoop1 - - - org.spark-project - spark-core - ${project.version} - hadoop1 - - - org.apache.hadoop - hadoop-core - provided - - - - - - org.apache.maven.plugins - maven-jar-plugin - - hadoop1 - - - - - hadoop2 @@ -96,11 +78,6 @@ ${project.version} hadoop2 - - org.apache.hadoop - hadoop-core - provided - org.apache.hadoop hadoop-client diff --git a/core/pom.xml b/core/pom.xml index dfadd22d421..680ae94a115 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -32,6 +32,11 @@ http://spark-project.org/ + + org.apache.hadoop + hadoop-client + provided + org.eclipse.jetty jetty-server @@ -130,7 +135,6 @@ com.codahale.metrics metrics-json - org.apache.derby derby @@ -210,66 +214,9 @@ - - hadoop1 - - - org.apache.hadoop - hadoop-core - provided - - - - - - org.codehaus.mojo - build-helper-maven-plugin - - - add-source - generate-sources - - add-source - - - - src/main/scala - src/hadoop1/scala - - - - - add-scala-test-sources - generate-test-sources - - add-test-source - - - - src/test/scala - - - - - - - org.apache.maven.plugins - maven-jar-plugin - - hadoop1 - - - - - hadoop2 - - org.apache.hadoop - hadoop-core - provided - org.apache.hadoop hadoop-client diff --git a/examples/pom.xml b/examples/pom.xml index a051da8a778..6a9c19ed6f4 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -32,6 +32,31 @@ http://spark-project.org/ + + org.spark-project + spark-core + ${project.version} + + + org.spark-project + spark-streaming + ${project.version} + + + org.spark-project + spark-mllib + ${project.version} + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.hbase + hbase + 0.94.6 + org.scala-lang scala-library @@ -55,41 +80,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 - - - + + 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 @@ -103,50 +128,6 @@ - - hadoop1 - - - org.spark-project - spark-core - ${project.version} - hadoop1 - - - org.spark-project - spark-streaming - ${project.version} - hadoop1 - - - org.spark-project - spark-mllib - ${project.version} - hadoop1 - - - org.apache.hadoop - hadoop-core - provided - - - org.apache.hbase - hbase - 0.94.6 - - - - - - org.apache.maven.plugins - maven-jar-plugin - - hadoop1 - - - - - hadoop2 @@ -168,11 +149,6 @@ ${project.version} hadoop2 - - org.apache.hadoop - hadoop-core - provided - org.apache.hadoop hadoop-client diff --git a/mllib/pom.xml b/mllib/pom.xml index a07480fbe2b..36f410d3b02 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -32,6 +32,16 @@ http://spark-project.org/ + + org.spark-project + spark-core + ${project.version} + + + org.apache.hadoop + hadoop-client + provided + org.eclipse.jetty jetty-server @@ -41,7 +51,6 @@ jblas 1.2.3 - org.scalatest scalatest_${scala.version} @@ -70,33 +79,6 @@ - - hadoop1 - - - org.spark-project - spark-core - ${project.version} - hadoop1 - - - org.apache.hadoop - hadoop-core - provided - - - - - - org.apache.maven.plugins - maven-jar-plugin - - hadoop1 - - - - - hadoop2 @@ -106,11 +88,6 @@ ${project.version} hadoop2 - - org.apache.hadoop - hadoop-core - provided - org.apache.hadoop hadoop-client diff --git a/pom.xml b/pom.xml index 135b18bd267..8d34fff3eeb 100644 --- a/pom.xml +++ b/pom.xml @@ -325,6 +325,17 @@ 0.8 test + + org.apache.hadoop + hadoop-client + 1.2.1 + + + asm + asm + + + @@ -530,22 +541,6 @@ - - hadoop1 - - 1 - - - - - org.apache.hadoop - hadoop-core - 1.2.1 - - - - - hadoop2 @@ -553,11 +548,6 @@ - - org.apache.hadoop - hadoop-core - 2.0.0-mr1-cdh${cdh.version} - org.apache.hadoop hadoop-client diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 7c4e722cc15..81aba06e145 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -37,6 +37,37 @@ root + + + org.spark-project + spark-core + ${project.version} + + + org.spark-project + spark-bagel + ${project.version} + runtime + + + org.spark-project + spark-examples + ${project.version} + runtime + + + org.spark-project + spark-repl + ${project.version} + runtime + + + org.apache.hadoop + hadoop-client + runtime + + + @@ -85,46 +116,6 @@ - - hadoop1 - - hadoop1 - - - - org.spark-project - spark-core - ${project.version} - hadoop1 - - - org.spark-project - spark-bagel - ${project.version} - hadoop1 - runtime - - - org.spark-project - spark-examples - ${project.version} - hadoop1 - runtime - - - org.spark-project - spark-repl - ${project.version} - hadoop1 - runtime - - - org.apache.hadoop - hadoop-core - runtime - - - hadoop2 @@ -158,11 +149,6 @@ hadoop2 runtime - - org.apache.hadoop - hadoop-core - runtime - org.apache.hadoop hadoop-client diff --git a/repl/pom.xml b/repl/pom.xml index 862595b9f94..81cebe178ab 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -37,6 +37,28 @@ + + org.spark-project + spark-core + ${project.version} + + + org.spark-project + spark-bagel + ${project.version} + runtime + + + org.spark-project + spark-examples + ${project.version} + runtime + + + org.apache.hadoop + hadoop-client + provided + org.eclipse.jetty jetty-server @@ -57,7 +79,6 @@ org.slf4j slf4j-log4j12 - org.scalatest scalatest_${scala.version} @@ -117,50 +138,6 @@ - - hadoop1 - - hadoop1 - - - - org.spark-project - spark-core - ${project.version} - hadoop1 - - - org.spark-project - spark-bagel - ${project.version} - hadoop1 - runtime - - - org.spark-project - spark-examples - ${project.version} - hadoop1 - runtime - - - org.apache.hadoop - hadoop-core - provided - - - - - - org.apache.maven.plugins - maven-jar-plugin - - hadoop1 - - - - - hadoop2 @@ -187,11 +164,6 @@ hadoop2 runtime - - org.apache.hadoop - hadoop-core - provided - org.apache.hadoop hadoop-client diff --git a/streaming/pom.xml b/streaming/pom.xml index 7e6b06d772f..9b478f7a055 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -40,6 +40,16 @@ + + org.spark-project + spark-core + ${project.version} + + + org.apache.hadoop + hadoop-client + provided + org.eclipse.jetty jetty-server @@ -117,33 +127,6 @@ - - hadoop1 - - - org.spark-project - spark-core - ${project.version} - hadoop1 - - - org.apache.hadoop - hadoop-core - provided - - - - - - org.apache.maven.plugins - maven-jar-plugin - - hadoop1 - - - - - hadoop2 @@ -153,11 +136,6 @@ ${project.version} hadoop2 - - org.apache.hadoop - hadoop-core - provided - org.apache.hadoop hadoop-client diff --git a/tools/pom.xml b/tools/pom.xml index 878eb82f182..c123c2ab237 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -31,6 +31,21 @@ http://spark-project.org/ + + org.spark-project + spark-core + ${project.version} + + + org.spark-project + spark-streaming + ${project.version} + + + org.apache.hadoop + hadoop-client + provided + org.scalatest scalatest_${scala.version} @@ -58,39 +73,6 @@ - - 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 @@ -106,11 +88,6 @@ ${project.version} hadoop2 - - org.apache.hadoop - hadoop-core - provided - org.apache.hadoop hadoop-client From 11b42a84db255eb659412e9d0bf4622cb2e8b20a Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Thu, 15 Aug 2013 15:31:31 -0700 Subject: [PATCH 0753/2521] Maven build now works with CDH hadoop-2.0.0-mr1 --- assembly/pom.xml | 14 ----------- bagel/pom.xml | 27 -------------------- core/pom.xml | 52 -------------------------------------- examples/pom.xml | 44 -------------------------------- mllib/pom.xml | 27 -------------------- pom.xml | 55 +++++++++++++++------------------------- repl-bin/pom.xml | 40 ----------------------------- repl/pom.xml | 64 ++++++++--------------------------------------- streaming/pom.xml | 27 -------------------- tools/pom.xml | 33 ------------------------ 10 files changed, 30 insertions(+), 353 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 76ac9f54780..3d645e0379c 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -37,20 +37,6 @@ - - - hadoop2 - - hadoop2 - - - - hadoop2-yarn - - hadoop2-yarn - - - org.spark-project diff --git a/bagel/pom.xml b/bagel/pom.xml index 3c82af3b335..1b555bead76 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -69,33 +69,6 @@ - - hadoop2 - - - org.spark-project - spark-core - ${project.version} - hadoop2 - - - org.apache.hadoop - hadoop-client - provided - - - - - - org.apache.maven.plugins - maven-jar-plugin - - hadoop2 - - - - - hadoop2-yarn diff --git a/core/pom.xml b/core/pom.xml index 680ae94a115..9310d000fd0 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -214,58 +214,6 @@ - - hadoop2 - - - org.apache.hadoop - hadoop-client - provided - - - - - - org.codehaus.mojo - build-helper-maven-plugin - - - add-source - generate-sources - - add-source - - - - src/main/scala - src/hadoop2/scala - - - - - add-scala-test-sources - generate-test-sources - - add-test-source - - - - src/test/scala - - - - - - - org.apache.maven.plugins - maven-jar-plugin - - hadoop2 - - - - - hadoop2-yarn diff --git a/examples/pom.xml b/examples/pom.xml index 6a9c19ed6f4..6e54e94cf54 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -128,50 +128,6 @@ - - hadoop2 - - - org.spark-project - spark-core - ${project.version} - hadoop2 - - - org.spark-project - spark-streaming - ${project.version} - hadoop2 - - - org.spark-project - spark-mllib - ${project.version} - hadoop2 - - - org.apache.hadoop - hadoop-client - provided - - - org.apache.hbase - hbase - 0.94.6 - - - - - - org.apache.maven.plugins - maven-jar-plugin - - hadoop2 - - - - - hadoop2-yarn diff --git a/mllib/pom.xml b/mllib/pom.xml index 36f410d3b02..863aef93925 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -79,33 +79,6 @@ - - hadoop2 - - - org.spark-project - spark-core - ${project.version} - hadoop2 - - - org.apache.hadoop - hadoop-client - provided - - - - - - org.apache.maven.plugins - maven-jar-plugin - - hadoop2 - - - - - hadoop2-yarn diff --git a/pom.xml b/pom.xml index 8d34fff3eeb..054f5d170ca 100644 --- a/pom.xml +++ b/pom.xml @@ -73,8 +73,9 @@ 0.12.1 2.0.3 1.7.2 - 4.1.2 1.2.17 + 1.2.1 + 64m 512m @@ -328,7 +329,7 @@ org.apache.hadoop hadoop-client - 1.2.1 + ${hadoop.version} asm @@ -336,6 +337,23 @@ + + + org.apache.avro + avro + 1.7.4 + + + org.apache.avro + avro-ipc + 1.7.4 + + + org.jboss.netty + netty + + + @@ -541,39 +559,6 @@ - - hadoop2 - - 2 - - - - - org.apache.hadoop - hadoop-client - 2.0.0-mr1-cdh${cdh.version} - - - - org.apache.avro - avro - 1.7.4 - - - org.apache.avro - avro-ipc - 1.7.4 - - - org.jboss.netty - netty - - - - - - - hadoop2-yarn diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 81aba06e145..eaee8ea016f 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -116,46 +116,6 @@ - - hadoop2 - - hadoop2 - - - - org.spark-project - spark-core - ${project.version} - hadoop2 - - - org.spark-project - spark-bagel - ${project.version} - hadoop2 - runtime - - - org.spark-project - spark-examples - ${project.version} - hadoop2 - runtime - - - org.spark-project - spark-repl - ${project.version} - hadoop2 - runtime - - - org.apache.hadoop - hadoop-client - runtime - - - hadoop2-yarn diff --git a/repl/pom.xml b/repl/pom.xml index 81cebe178ab..032c20e1185 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -59,6 +59,16 @@ hadoop-client provided + + org.apache.avro + avro + provided + + + org.apache.avro + avro-ipc + provided + org.eclipse.jetty jetty-server @@ -138,60 +148,6 @@ - - hadoop2 - - hadoop2 - - - - org.spark-project - spark-core - ${project.version} - hadoop2 - - - org.spark-project - spark-bagel - ${project.version} - hadoop2 - runtime - - - org.spark-project - spark-examples - ${project.version} - hadoop2 - runtime - - - org.apache.hadoop - hadoop-client - provided - - - org.apache.avro - avro - provided - - - org.apache.avro - avro-ipc - provided - - - - - - org.apache.maven.plugins - maven-jar-plugin - - hadoop2 - - - - - hadoop2-yarn diff --git a/streaming/pom.xml b/streaming/pom.xml index 9b478f7a055..612ff0a024b 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -127,33 +127,6 @@ - - hadoop2 - - - org.spark-project - spark-core - ${project.version} - hadoop2 - - - org.apache.hadoop - hadoop-client - provided - - - - - - org.apache.maven.plugins - maven-jar-plugin - - hadoop2 - - - - - hadoop2-yarn diff --git a/tools/pom.xml b/tools/pom.xml index c123c2ab237..5864c9f2178 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -73,39 +73,6 @@ - - hadoop2 - - - org.spark-project - spark-core - ${project.version} - hadoop2 - - - org.spark-project - spark-streaming - ${project.version} - hadoop2 - - - org.apache.hadoop - hadoop-client - provided - - - - - - org.apache.maven.plugins - maven-jar-plugin - - hadoop2 - - - - - hadoop2-yarn From 9dd15fe700ad8f52739cce58cbdf198fab8fd5d8 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Thu, 15 Aug 2013 15:40:37 -0700 Subject: [PATCH 0754/2521] Don't mark hadoop-client as 'provided' --- bagel/pom.xml | 1 - core/pom.xml | 1 - examples/pom.xml | 1 - mllib/pom.xml | 1 - 4 files changed, 4 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 1b555bead76..fc5dce7ffd8 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -40,7 +40,6 @@ org.apache.hadoop hadoop-client - provided org.eclipse.jetty diff --git a/core/pom.xml b/core/pom.xml index 9310d000fd0..c5baecfaad5 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -35,7 +35,6 @@ org.apache.hadoop hadoop-client - provided org.eclipse.jetty diff --git a/examples/pom.xml b/examples/pom.xml index 6e54e94cf54..4ccc6aa1983 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -50,7 +50,6 @@ org.apache.hadoop hadoop-client - provided org.apache.hbase diff --git a/mllib/pom.xml b/mllib/pom.xml index 863aef93925..801aa6e7191 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -40,7 +40,6 @@ org.apache.hadoop hadoop-client - provided org.eclipse.jetty From 741ecd56fe714fe42c22518aefcfa48fa3a448c7 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Thu, 15 Aug 2013 16:32:15 -0700 Subject: [PATCH 0755/2521] Forgot to remove a few references to ${classifier} --- assembly/pom.xml | 5 ----- repl-bin/pom.xml | 8 ++++---- 2 files changed, 4 insertions(+), 9 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 3d645e0379c..ca20ccadba8 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -41,31 +41,26 @@ org.spark-project spark-core - ${classifier.name} ${project.version} org.spark-project spark-bagel - ${classifier.name} ${project.version} org.spark-project spark-mllib - ${classifier.name} ${project.version} org.spark-project spark-repl - ${classifier.name} ${project.version} org.spark-project spark-streaming - ${classifier.name} ${project.version} diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index eaee8ea016f..270a1601204 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -32,8 +32,8 @@ http://spark-project.org/ - spark-${classifier} - /usr/share/spark-${classifier} + spark + /usr/share/spark root @@ -75,7 +75,7 @@ maven-shade-plugin false - ${project.build.directory}/${project.artifactId}-${project.version}-shaded-${classifier}.jar + ${project.build.directory}/${project.artifactId}-${project.version}-shaded.jar *:* @@ -207,7 +207,7 @@ gzip - ${project.build.directory}/${project.artifactId}-${project.version}-shaded-${classifier}.jar + ${project.build.directory}/${project.artifactId}-${project.version}-shaded.jar file perm From ad580b94d506b3dbb8b4206326e4df2e1104e3b3 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Thu, 15 Aug 2013 16:49:24 -0700 Subject: [PATCH 0756/2521] Maven build now also works with YARN --- bagel/pom.xml | 40 ------------ bin/compute-classpath.sh | 2 +- core/pom.xml | 70 --------------------- examples/pom.xml | 57 ----------------- mllib/pom.xml | 40 ------------ pom.xml | 128 +++++++++++++++++++++++++++++++++++++++ repl-bin/pom.xml | 47 +------------- repl/pom.xml | 64 +------------------- streaming/pom.xml | 40 ------------ tools/pom.xml | 46 -------------- yarn/pom.xml | 106 ++++++++++++++++++++++++++++++++ 11 files changed, 237 insertions(+), 403 deletions(-) create mode 100644 yarn/pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index fc5dce7ffd8..ae40f38e431 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -66,44 +66,4 @@ - - - - hadoop2-yarn - - - org.spark-project - spark-core - ${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 - - - - - - diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index db6fc866abf..f975d3bfb9c 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -63,7 +63,7 @@ function dev_classpath { 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 + for jar in `find "$REPL_BIN_DIR/target" -name 'spark-repl-*-shaded.jar'`; do CLASSPATH="$CLASSPATH:$jar" done # The shaded JAR doesn't contain examples, so include those separately diff --git a/core/pom.xml b/core/pom.xml index c5baecfaad5..90d279c6350 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -211,74 +211,4 @@ - - - - hadoop2-yarn - - - org.apache.hadoop - hadoop-client - provided - - - org.apache.hadoop - hadoop-yarn-api - provided - - - org.apache.hadoop - hadoop-yarn-common - provided - - - org.apache.hadoop - hadoop-yarn-client - provided - - - - - - org.codehaus.mojo - build-helper-maven-plugin - - - add-source - generate-sources - - add-source - - - - src/main/scala - src/hadoop2-yarn/scala - - - - - add-scala-test-sources - generate-test-sources - - add-test-source - - - - src/test/scala - - - - - - - org.apache.maven.plugins - maven-jar-plugin - - hadoop2-yarn - - - - - - diff --git a/examples/pom.xml b/examples/pom.xml index 4ccc6aa1983..4eb32935f49 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -125,61 +125,4 @@ - - - - hadoop2-yarn - - - org.spark-project - spark-core - ${project.version} - hadoop2-yarn - - - org.spark-project - spark-streaming - ${project.version} - hadoop2-yarn - - - org.spark-project - spark-mllib - ${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.hbase - hbase - 0.94.6 - - - - - - org.apache.maven.plugins - maven-jar-plugin - - hadoop2-yarn - - - - - - diff --git a/mllib/pom.xml b/mllib/pom.xml index 801aa6e7191..3292f6dad03 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -76,44 +76,4 @@ - - - - hadoop2-yarn - - - org.spark-project - spark-core - ${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 - - - - - - diff --git a/pom.xml b/pom.xml index 054f5d170ca..4714576f3be 100644 --- a/pom.xml +++ b/pom.xml @@ -335,6 +335,26 @@ asm asm + + org.jboss.netty + netty + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-jaxrs + + + org.codehaus.jackson + jackson-xc + @@ -568,6 +588,10 @@ 2.0.5-alpha + + yarn + + maven-root @@ -589,21 +613,125 @@ org.apache.hadoop hadoop-client ${yarn.version} + + + asm + asm + + + org.jboss.netty + netty + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-jaxrs + + + org.codehaus.jackson + jackson-xc + + org.apache.hadoop hadoop-yarn-api ${yarn.version} + + + asm + asm + + + org.jboss.netty + netty + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-jaxrs + + + org.codehaus.jackson + jackson-xc + + org.apache.hadoop hadoop-yarn-common ${yarn.version} + + + asm + asm + + + org.jboss.netty + netty + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-jaxrs + + + org.codehaus.jackson + jackson-xc + + org.apache.hadoop hadoop-yarn-client ${yarn.version} + + + asm + asm + + + org.jboss.netty + netty + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + org.codehaus.jackson + jackson-jaxrs + + + org.codehaus.jackson + jackson-xc + + diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 270a1601204..f3bde607445 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -118,56 +118,11 @@ hadoop2-yarn - - hadoop2-yarn - org.spark-project - spark-core + spark-yarn ${project.version} - hadoop2-yarn - - - org.spark-project - spark-bagel - ${project.version} - hadoop2-yarn - runtime - - - org.spark-project - spark-examples - ${project.version} - hadoop2-yarn - runtime - - - org.spark-project - spark-repl - ${project.version} - hadoop2-yarn - runtime - - - org.apache.hadoop - hadoop-client - runtime - - - org.apache.hadoop - hadoop-yarn-api - runtime - - - org.apache.hadoop - hadoop-yarn-common - runtime - - - org.apache.hadoop - hadoop-yarn-client - runtime diff --git a/repl/pom.xml b/repl/pom.xml index 032c20e1185..429de7861f8 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -146,78 +146,16 @@ - hadoop2-yarn - - hadoop2-yarn - org.spark-project - spark-core - ${project.version} - hadoop2-yarn - - - org.spark-project - spark-bagel - ${project.version} - hadoop2-yarn - runtime - - - org.spark-project - spark-examples - ${project.version} - hadoop2-yarn - runtime - - - org.spark-project - spark-streaming + spark-yarn ${project.version} - hadoop2-yarn - runtime - - - org.apache.hadoop - hadoop-client - provided - - - org.apache.hadoop - hadoop-yarn-api - provided - - - org.apache.hadoop - hadoop-yarn-common - provided - - - org.apache.avro - avro - provided - - - org.apache.avro - avro-ipc - provided - - - - org.apache.maven.plugins - maven-jar-plugin - - hadoop2-yarn - - - - diff --git a/streaming/pom.xml b/streaming/pom.xml index 612ff0a024b..18609901223 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -125,44 +125,4 @@ - - - - hadoop2-yarn - - - org.spark-project - spark-core - ${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 - - - - - - diff --git a/tools/pom.xml b/tools/pom.xml index 5864c9f2178..9177d85b2f4 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -71,50 +71,4 @@ - - - - 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 - - - - - - diff --git a/yarn/pom.xml b/yarn/pom.xml new file mode 100644 index 00000000000..6acde8e98ca --- /dev/null +++ b/yarn/pom.xml @@ -0,0 +1,106 @@ + + + + + 4.0.0 + + org.spark-project + spark-parent + 0.8.0-SNAPSHOT + ../pom.xml + + + org.spark-project + spark-yarn + jar + Spark Project YARN Support + http://spark-project.org/ + + + + org.spark-project + spark-core + ${project.version} + + + org.apache.hadoop + hadoop-yarn-api + + + org.apache.hadoop + hadoop-yarn-common + + + org.apache.hadoop + hadoop-yarn-client + + + org.apache.avro + avro + + + org.apache.avro + avro-ipc + + + + + target/scala-${scala.version}/classes + target/scala-${scala.version}/test-classes + + + org.apache.maven.plugins + maven-shade-plugin + + false + ${project.build.directory}/${project.artifactId}-${project.version}-shaded.jar + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + package + + shade + + + + + + reference.conf + + + + + + + + + From c1e547bb7f21ebed198bea0aed0a122eb0d70835 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Fri, 16 Aug 2013 12:26:45 -0700 Subject: [PATCH 0757/2521] Updates to repl and example POMs to match SBT build --- examples/pom.xml | 10 ++++++++++ repl-bin/pom.xml | 6 ------ repl/pom.xml | 6 ------ 3 files changed, 10 insertions(+), 12 deletions(-) diff --git a/examples/pom.xml b/examples/pom.xml index 4eb32935f49..023ad8cb458 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -55,6 +55,16 @@ org.apache.hbase hbase 0.94.6 + + + asm + asm + + + org.jboss.netty + netty + + org.scala-lang diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index f3bde607445..f132c44fb9e 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -49,12 +49,6 @@ ${project.version} runtime - - org.spark-project - spark-examples - ${project.version} - runtime - org.spark-project spark-repl diff --git a/repl/pom.xml b/repl/pom.xml index 429de7861f8..82e26defbc7 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -48,12 +48,6 @@ ${project.version} runtime - - org.spark-project - spark-examples - ${project.version} - runtime - org.apache.hadoop hadoop-client From b1d99744a813a72301260612943f46853794f00f Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Fri, 16 Aug 2013 13:49:26 -0700 Subject: [PATCH 0758/2521] Fix SBT build under Hadoop 0.23.x --- project/SparkBuild.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index fa9ec7deca9..4023626c16a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -193,6 +193,17 @@ object SparkBuild extends Build { "com.codahale.metrics" % "metrics-json" % "3.0.0", "com.twitter" % "chill_2.9.3" % "0.3.1", "com.twitter" % "chill-java" % "0.3.1" + ) ++ ( + if (isYarnMode) { + // This kludge is needed for 0.23.x + Seq( + "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm) + ) + } else { + Seq() + } ) ) ++ assemblySettings ++ extraAssemblySettings From 67b593607c7df934d5a73012fe9cce220b25f321 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Fri, 16 Aug 2013 13:53:16 -0700 Subject: [PATCH 0759/2521] Rename YARN build flag to SPARK_WITH_YARN --- make-distribution.sh | 8 ++++---- project/SparkBuild.scala | 12 +++++++----- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index a101024de54..55dc22b9927 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -47,7 +47,7 @@ VERSION=$($FWDIR/sbt/sbt "show version" | tail -1 | cut -f 2 | sed 's/^\([a-zA-Z # Initialize defaults SPARK_HADOOP_VERSION=1.2.1 -SPARK_YARN_MODE=false +SPARK_WITH_YARN=false MAKE_TGZ=false # Parse arguments @@ -58,7 +58,7 @@ while (( "$#" )); do shift ;; --with-yarn) - SPARK_YARN_MODE=true + SPARK_WITH_YARN=true ;; --tgz) MAKE_TGZ=true @@ -74,7 +74,7 @@ else fi echo "Hadoop version set to $SPARK_HADOOP_VERSION" -if [ "$SPARK_YARN_MODE" == "true" ]; then +if [ "$SPARK_WITH_YARN" == "true" ]; then echo "YARN enabled" else echo "YARN disabled" @@ -82,7 +82,7 @@ fi # Build fat JAR export SPARK_HADOOP_VERSION -export SPARK_YARN_MODE +export SPARK_WITH_YARN "$FWDIR/sbt/sbt" "repl/assembly" # Make directories diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 4023626c16a..cea982b8860 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -26,8 +26,10 @@ import AssemblyKeys._ object SparkBuild extends Build { // Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or // "1.0.4" for Apache releases, or "0.20.2-cdh3u5" for Cloudera Hadoop. - val HADOOP_VERSION = "1.2.1" - val HADOOP_YARN = false + // Note that these variables can be set through the environment variables + // SPARK_HADOOP_VERSION and SPARK_WITH_YARN. + val DEFAULT_HADOOP_VERSION = "1.2.1" + val DEFAULT_WITH_YARN = false // HBase version; set as appropriate. val HBASE_VERSION = "0.94.6" @@ -55,9 +57,9 @@ object SparkBuild extends Build { lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") // Allows build configuration to be set through environment variables - lazy val hadoopVersion = scala.util.Properties.envOrElse("SPARK_HADOOP_VERSION", HADOOP_VERSION) - lazy val isYarnMode = scala.util.Properties.envOrNone("SPARK_YARN_MODE") match { - case None => HADOOP_YARN + lazy val hadoopVersion = scala.util.Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION) + lazy val isYarnMode = scala.util.Properties.envOrNone("SPARK_WITH_YARN") match { + case None => DEFAULT_WITH_YARN case Some(v) => v.toBoolean } From 5d050a3e1f6cb79d1f10a4d2d4207b04107b2078 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 16 Aug 2013 23:23:16 -0700 Subject: [PATCH 0760/2521] Removed the unused shuffleId in ShuffleDependency's constructor. --- core/src/main/scala/spark/Dependency.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/spark/Dependency.scala b/core/src/main/scala/spark/Dependency.scala index d17e70a4faf..b1edaa06f80 100644 --- a/core/src/main/scala/spark/Dependency.scala +++ b/core/src/main/scala/spark/Dependency.scala @@ -39,7 +39,6 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** * Represents a dependency on the output of a shuffle stage. - * @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 From 10af952a3d81c8d7a1178c61cbcd1b5269dc1fe8 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 17 Aug 2013 21:07:34 -0700 Subject: [PATCH 0761/2521] Removed the mapSideCombine option in CoGroupedRDD. --- .../main/scala/spark/rdd/CoGroupedRDD.scala | 38 +++---------------- 1 file changed, 5 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index c540cd36eb9..019b12d2d5e 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -23,7 +23,7 @@ import java.util.{HashMap => JHashMap} import scala.collection.JavaConversions import scala.collection.mutable.ArrayBuffer -import spark.{Aggregator, Partition, Partitioner, RDD, SparkEnv, TaskContext} +import spark.{Partition, Partitioner, RDD, SparkEnv, TaskContext} import spark.{Dependency, OneToOneDependency, ShuffleDependency} @@ -52,13 +52,6 @@ class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep]) override def hashCode(): Int = idx } -private[spark] class CoGroupAggregator - extends Aggregator[Any, Any, ArrayBuffer[Any]]( - { x => ArrayBuffer(x) }, - { (b, x) => b += x }, - { (b1, b2) => b1 ++ b2 }) - with Serializable - /** * A RDD that cogroups its parents. For each key k in parent RDDs, the resulting RDD contains a @@ -66,34 +59,21 @@ 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. 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 = false, val serializerClass: String = null) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { - private val aggr = new CoGroupAggregator - override def getDependencies: Seq[Dependency[_]] = { - rdds.map { rdd => + rdds.map { rdd: RDD[(K, _)] => if (rdd.partitioner == Some(part)) { logInfo("Adding one-to-one dependency with " + rdd) new OneToOneDependency(rdd) } else { logInfo("Adding shuffle dependency with " + rdd) - if (mapSideCombine) { - val mapSideCombinedRDD = rdd.mapPartitions(aggr.combineValuesByKey(_), true) - new ShuffleDependency[Any, ArrayBuffer[Any]](mapSideCombinedRDD, part, serializerClass) - } else { - new ShuffleDependency[Any, Any](rdd.asInstanceOf[RDD[(Any, Any)]], part, serializerClass) - } + new ShuffleDependency[Any, Any](rdd.asInstanceOf[RDD[(Any, Any)]], part, serializerClass) } } } @@ -145,16 +125,8 @@ class CoGroupedRDD[K]( case ShuffleCoGroupSplitDep(shuffleId) => { // Read map outputs of shuffle 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, 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, ser).foreach { - case (key, value) => getSeq(key)(depNum) += value - } + fetcher.fetch[K, Any](shuffleId, split.index, context.taskMetrics, ser).foreach { + case (key, value) => getSeq(key)(depNum) += value } } } From 0e84fee76b529089fb52f15151202e9a7b847ed5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 17 Aug 2013 21:13:41 -0700 Subject: [PATCH 0762/2521] Removed the mapSideCombine option in partitionBy. --- .../main/scala/spark/PairRDDFunctions.scala | 28 ++++--------------- .../scala/spark/api/java/JavaPairRDD.scala | 6 +--- 2 files changed, 6 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 6b0cc2fbf15..fa9df3a97e7 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -233,31 +233,13 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( } /** - * Return a copy of the RDD partitioned using the specified partitioner. If `mapSideCombine` - * is true, Spark will group values of the same key together on the map side before the - * repartitioning, to only send each key over the network once. If a large number of - * duplicated keys are expected, and the size of the keys are large, `mapSideCombine` should - * be set to true. + * Return a copy of the RDD partitioned using the specified partitioner. */ - def partitionBy(partitioner: Partitioner, mapSideCombine: Boolean = false): RDD[(K, V)] = { - if (getKeyClass().isArray) { - if (mapSideCombine) { - throw new SparkException("Cannot use map-side combining with array keys.") - } - if (partitioner.isInstanceOf[HashPartitioner]) { - throw new SparkException("Default partitioner cannot partition array keys.") - } - } - if (mapSideCombine) { - 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) - bufs.flatMapValues(buf => buf) - } else { - new ShuffledRDD[K, V](self, partitioner) + def partitionBy(partitioner: Partitioner): RDD[(K, V)] = { + if (getKeyClass().isArray && partitioner.isInstanceOf[HashPartitioner]) { + throw new SparkException("Default partitioner cannot partition array keys.") } + new ShuffledRDD[K, V](self, partitioner) } /** diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index ff12e8b76cf..c2995b836a1 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -253,11 +253,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif fromRDD(rdd.subtract(other, p)) /** - * Return a copy of the RDD partitioned using the specified partitioner. If `mapSideCombine` - * is true, Spark will group values of the same key together on the map side before the - * repartitioning, to only send each key over the network once. If a large number of - * duplicated keys are expected, and the size of the keys are large, `mapSideCombine` should - * be set to true. + * Return a copy of the RDD partitioned using the specified partitioner. */ def partitionBy(partitioner: Partitioner): JavaPairRDD[K, V] = fromRDD(rdd.partitionBy(partitioner)) From 2c00ea3efc7d9a23af8ba11352460294e1865942 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 17 Aug 2013 21:43:29 -0700 Subject: [PATCH 0763/2521] Moved shuffle serializer setting from a constructor parameter to a setSerializer method in various RDDs that involve shuffle operations. --- .../main/scala/spark/PairRDDFunctions.scala | 13 +++++---- .../main/scala/spark/rdd/CoGroupedRDD.scala | 12 +++++--- .../main/scala/spark/rdd/ShuffledRDD.scala | 29 ++++++++++++++----- .../main/scala/spark/rdd/SubtractedRDD.scala | 10 +++++-- core/src/test/scala/spark/ShuffleSuite.scala | 19 ++++-------- 5 files changed, 51 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index fa9df3a97e7..0be4b4feb8c 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -85,17 +85,18 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( } val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners) if (self.partitioner == Some(partitioner)) { - self.mapPartitions(aggregator.combineValuesByKey(_), true) + self.mapPartitions(aggregator.combineValuesByKey, true) } else if (mapSideCombine) { - val mapSideCombined = self.mapPartitions(aggregator.combineValuesByKey(_), true) - val partitioned = new ShuffledRDD[K, C](mapSideCombined, partitioner, serializerClass) - partitioned.mapPartitions(aggregator.combineCombinersByKey(_), true) + val mapSideCombined = self.mapPartitions(aggregator.combineValuesByKey, true) + val partitioned = new ShuffledRDD[K, C](mapSideCombined, partitioner) + .setSerializer(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, serializerClass) - values.mapPartitions(aggregator.combineValuesByKey(_), true) + val values = new ShuffledRDD[K, V](self, partitioner).setSerializer(serializerClass) + values.mapPartitions(aggregator.combineValuesByKey, true) } } diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 019b12d2d5e..c2d95dc0604 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -60,12 +60,16 @@ class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep]) * @param rdds parent RDDs. * @param part partitioner used to partition the shuffle output. */ -class CoGroupedRDD[K]( - @transient var rdds: Seq[RDD[(K, _)]], - part: Partitioner, - val serializerClass: String = null) +class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { + private var serializerClass: String = null + + def setSerializer(cls: String): CoGroupedRDD[K] = { + serializerClass = cls + this + } + override def getDependencies: Seq[Dependency[_]] = { rdds.map { rdd: RDD[(K, _)] => if (rdd.partitioner == Some(part)) { diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala index 0137f80953e..bcf7d0d89c4 100644 --- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala @@ -17,8 +17,9 @@ package spark.rdd -import spark.{Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext} -import spark.SparkContext._ +import spark._ +import scala.Some +import scala.Some private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @@ -30,15 +31,24 @@ 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, - serializerClass: String = null) - extends RDD[(K, V)](prev.context, List(new ShuffleDependency(prev, part, serializerClass))) { + @transient var prev: RDD[(K, V)], + part: Partitioner) + extends RDD[(K, V)](prev.context, Nil) { + + private var serializerClass: String = null + + def setSerializer(cls: String): ShuffledRDD[K, V] = { + serializerClass = cls + this + } + + override def getDependencies: Seq[Dependency[_]] = { + List(new ShuffleDependency(prev, part, serializerClass)) + } override val partitioner = Some(part) @@ -51,4 +61,9 @@ class ShuffledRDD[K, V]( SparkEnv.get.shuffleFetcher.fetch[K, V](shuffledId, split.index, context.taskMetrics, SparkEnv.get.serializerManager.get(serializerClass)) } + + override def clearDependencies() { + super.clearDependencies() + prev = null + } } diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala index 0402b9f2501..46b8cafaac3 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -49,10 +49,16 @@ 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, - val serializerClass: String = null) + part: Partitioner) extends RDD[(K, V)](rdd1.context, Nil) { + private var serializerClass: String = null + + def setSerializer(cls: String): SubtractedRDD[K, V, W] = { + serializerClass = cls + this + } + override def getDependencies: Seq[Dependency[_]] = { Seq(rdd1, rdd2).map { rdd => if (rdd.partitioner == Some(part)) { diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 752e4b85e67..c686b8cc5af 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -17,17 +17,8 @@ package spark -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashSet - import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers -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._ @@ -59,8 +50,8 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { } // 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 c = new ShuffledRDD(b, new HashPartitioner(NUM_BLOCKS)) + .setSerializer(classOf[spark.KryoSerializer].getName) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId assert(c.count === 10) @@ -81,7 +72,8 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { } // 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) + val c = new ShuffledRDD(b, new HashPartitioner(3)) + .setSerializer(classOf[spark.KryoSerializer].getName) assert(c.count === 10) } @@ -96,7 +88,8 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { // 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 c = new ShuffledRDD(b, new HashPartitioner(10)) + .setSerializer(classOf[spark.KryoSerializer].getName) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId assert(c.count === 4) From 8fd37adf832a7cd234dca64ea7679668b89fafa8 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sun, 18 Aug 2013 10:57:35 -0700 Subject: [PATCH 0764/2521] Merged with changes to zipPartitions --- .../src/main/scala/spark/rdd/IndexedRDD.scala | 25 +++++++++---------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index 6b1cb7608bb..37ff11e6396 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -262,7 +262,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // However it is possible that both RDDs are missing a value for a given key in // which case the returned RDD should have a null value val newValues = - valuesRDD.zipPartitions[ Seq[Seq[W]], Seq[Seq[(Seq[V], Seq[W])]] ]( + valuesRDD.zipPartitions(other.valuesRDD)( (thisIter, otherIter) => { val thisValues: Seq[Seq[V]] = thisIter.next() assert(!thisIter.hasNext()) @@ -277,7 +277,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( case (a, b) => Seq((a,b)) }.toSeq List(tmp).iterator - }, other.valuesRDD) + }) new IndexedRDD[K, (Seq[V], Seq[W])](index, newValues) } case other: IndexedRDD[_, _] if other.index.partitioner == index.partitioner => { @@ -285,7 +285,8 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // then we we need to first merge the indicies and then use the merged index to // merge the values. val newIndex = - index.zipPartitions[JHashMap[K,Int], JHashMap[K,Int]]( (thisIter, otherIter) => { + index.zipPartitions(other.index)( + (thisIter, otherIter) => { val thisIndex = thisIter.next() assert(!thisIter.hasNext()) val otherIndex = otherIter.next() @@ -302,13 +303,10 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( ctr += 1 } List(newIndex).iterator - }, other.index).cache() + }).cache() // Use the new index along with the this and the other indices to merge the values val newValues = - newIndex.zipPartitions[ - (JHashMap[K, Int], Seq[Seq[V]]), - (JHashMap[K, Int], Seq[Seq[W]]), - Seq[Seq[(Seq[V],Seq[W])]] ]( + newIndex.zipPartitions(tuples, other.tuples)( (newIndexIter, thisTuplesIter, otherTuplesIter) => { // Get the new index for this partition val newIndex = newIndexIter.next() @@ -334,7 +332,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } } List(newValues.toSeq).iterator - }, tuples, other.tuples) + }) new IndexedRDD(newIndex, newValues) } case _ => { @@ -351,7 +349,8 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( new ShuffledRDD[K,W](other, partitioner) } // Join the other RDD with this RDD building a new valueset and new index on the fly - val groups = tuples.zipPartitions[(K, W), (JHashMap[K, Int], Seq[Seq[(Seq[V],Seq[W])]]) ]( + val groups = + tuples.zipPartitions(otherShuffled)( (thisTuplesIter, otherTuplesIter) => { // Get the corresponding indicies and values for this IndexedRDD val (thisIndex, thisValues) = thisTuplesIter.next() @@ -383,7 +382,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = newValues.view.map{ case (s, ab) => Seq((s, ab.toSeq)) }.toSeq List( (newIndex, newValuesArray) ).iterator - }, otherShuffled).cache() + }).cache() // Extract the index and values from the above RDD val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) @@ -615,7 +614,7 @@ object IndexedRDD { } // Use the index to build the new values table - val values = index.zipPartitions[ (K, V), Seq[Seq[V]] ]( + val values = index.zipPartitions(shuffledTbl)( (indexIter, tblIter) => { // There is only one map val index: JHashMap[K,Int] = indexIter.next() @@ -633,7 +632,7 @@ object IndexedRDD { values(ind).asInstanceOf[ArrayBuffer[V]].append(v) } List(values.toSeq).iterator - }, shuffledTbl) + }) new IndexedRDD[K,V](index, values) } From b659af83d3f91f0f339d874b2742ddca20a9f610 Mon Sep 17 00:00:00 2001 From: Evan Sparks Date: Wed, 14 Aug 2013 16:24:23 -0700 Subject: [PATCH 0765/2521] Adding Linear Regression, and refactoring Ridge Regression. --- .../mllib/regression/LinearRegression.scala | 168 +++++++++++ .../mllib/regression/RidgeRegression.scala | 271 ++++++++---------- .../util/LinearRegressionDataGenerator.scala | 98 +++++++ .../util/RidgeRegressionDataGenerator.scala | 5 +- .../regression/JavaLinearRegressionSuite.java | 96 +++++++ .../regression/JavaRidgeRegressionSuite.java | 96 +++++++ .../regression/LinearRegressionSuite.scala | 87 ++++++ .../regression/RidgeRegressionSuite.scala | 68 +++-- 8 files changed, 713 insertions(+), 176 deletions(-) create mode 100644 mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala create mode 100644 mllib/src/main/scala/spark/mllib/util/LinearRegressionDataGenerator.scala create mode 100644 mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java create mode 100644 mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java create mode 100644 mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala diff --git a/mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala new file mode 100644 index 00000000000..0ea5348a1fb --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala @@ -0,0 +1,168 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.mllib.regression + +import spark.{Logging, RDD, SparkContext} +import spark.mllib.optimization._ +import spark.mllib.util.MLUtils + +import org.jblas.DoubleMatrix + +/** + * Regression model trained using LinearRegression. + * + * @param weights Weights computed for every feature. + * @param intercept Intercept computed for this model. + */ +class LinearRegressionModel( + override val weights: Array[Double], + override val intercept: Double) + extends GeneralizedLinearModel(weights, intercept) + with RegressionModel with Serializable { + + override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, + intercept: Double) = { + dataMatrix.dot(weightMatrix) + intercept + } +} + +/** + * Train a regression model with no regularization using Stochastic Gradient Descent. + */ +class LinearRegressionWithSGD private ( + var stepSize: Double, + var numIterations: Int, + var miniBatchFraction: Double, + var addIntercept: Boolean) + extends GeneralizedLinearAlgorithm[LinearRegressionModel] + with Serializable { + + val gradient = new SquaredGradient() + val updater = new SimpleUpdater() + val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) + .setNumIterations(numIterations) + .setMiniBatchFraction(miniBatchFraction) + + /** + * Construct a LinearRegression object with default parameters + */ + def this() = this(1.0, 100, 1.0, true) + + def createModel(weights: Array[Double], intercept: Double) = { + new LinearRegressionModel(weights, intercept) + } +} + +/** + * Top-level methods for calling LinearRegression. + */ +object LinearRegressionWithSGD { + + /** + * Train a Linear Regression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in + * gradient descent are initialized using the initial weights provided. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param miniBatchFraction Fraction of data to be used per iteration. + * @param initialWeights Initial set of weights to be used. Array should be equal in size to + * the number of features in the data. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double, + initialWeights: Array[Double]) + : LinearRegressionModel = + { + new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction, true).run(input, + initialWeights) + } + + /** + * Train a LinearRegression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param miniBatchFraction Fraction of data to be used per iteration. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double) + : LinearRegressionModel = + { + new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction, true).run(input) + } + + /** + * Train a LinearRegression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. We use the entire data set to + * update the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param stepSize Step size to be used for each iteration of Gradient Descent. + * @param numIterations Number of iterations of gradient descent to run. + * @return a LinearRegressionModel which has the weights and offset from training. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double) + : LinearRegressionModel = + { + train(input, numIterations, stepSize, 1.0) + } + + /** + * Train a LinearRegression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using a step size of 1.0. We use the entire data set to + * update the gradient in each iteration. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @return a LinearRegressionModel which has the weights and offset from training. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int) + : LinearRegressionModel = + { + train(input, numIterations, 1.0, 1.0) + } + + def main(args: Array[String]) { + if (args.length != 5) { + println("Usage: LinearRegression ") + System.exit(1) + } + val sc = new SparkContext(args(0), "LinearRegression") + val data = MLUtils.loadLabeledData(sc, args(1)) + val model = LinearRegressionWithSGD.train(data, args(3).toInt, args(2).toDouble) + + sc.stop() + } +} diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index b42d94af416..addf8cd59ec 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -18,200 +18,159 @@ package spark.mllib.regression import spark.{Logging, RDD, SparkContext} +import spark.mllib.optimization._ import spark.mllib.util.MLUtils 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 + * Regression model trained using RidgeRegression. + * + * @param weights Weights computed for every feature. + * @param intercept Intercept computed for this model. */ class RidgeRegressionModel( - val weights: DoubleMatrix, - val intercept: Double, - val lambdaOpt: Double, - val lambdas: Seq[(Double, Double, DoubleMatrix)]) - 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(localWeights)).get(0) + localIntercept - } - } - - override def predict(testData: Array[Double]): Double = { - (new DoubleMatrix(1, testData.length, testData:_*).mmul(this.weights)).get(0) + this.intercept + override val weights: Array[Double], + override val intercept: Double) + extends GeneralizedLinearModel(weights, intercept) + with RegressionModel with Serializable { + + override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, + intercept: Double) = { + dataMatrix.dot(weightMatrix) + intercept } } -class RidgeRegression private (var lambdaLow: Double, var lambdaHigh: Double) - extends Logging { - - def this() = this(0.0, 100.0) +/** + * Train a regression model with L2-regularization using Stochastic Gradient Descent. + */ +class RidgeRegressionWithSGD private ( + var stepSize: Double, + var numIterations: Int, + var regParam: Double, + var miniBatchFraction: Double, + var addIntercept: Boolean) + extends GeneralizedLinearAlgorithm[RidgeRegressionModel] + with Serializable { + + val gradient = new SquaredGradient() + val updater = new SquaredL2Updater() + val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) + .setNumIterations(numIterations) + .setRegParam(regParam) + .setMiniBatchFraction(miniBatchFraction) /** - * Set the lower bound on binary search for lambda's. Default is 0. + * Construct a RidgeRegression object with default parameters */ - def setLowLambda(low: Double) = { - this.lambdaLow = low - this + def this() = this(1.0, 100, 1.0, 1.0, true) + + def createModel(weights: Array[Double], intercept: Double) = { + new RidgeRegressionModel(weights, intercept) } +} + +/** + * Top-level methods for calling RidgeRegression. + */ +object RidgeRegressionWithSGD { /** - * Set the upper bound on binary search for lambda's. Default is 100.0. + * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in + * gradient descent are initialized using the initial weights provided. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + * @param initialWeights Initial set of weights to be used. Array should be equal in size to + * the number of features in the data. */ - def setHighLambda(hi: Double) = { - this.lambdaHigh = hi - this + def train( + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, + initialWeights: Array[Double]) + : RidgeRegressionModel = + { + new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction, true).run(input, + initialWeights) } - def train(inputLabeled: RDD[LabeledPoint]): RidgeRegressionModel = { - val input = inputLabeled.map(labeledPoint => (labeledPoint.label, labeledPoint.features)) - 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): Seq[(Double, Double, DoubleMatrix)] = { - val buffer = mutable.ListBuffer.empty[(Double, Double, DoubleMatrix)] - - @tailrec - 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) - 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) { - buffer += lowValue += highValue - loop(newLow, newHigh) - } else { - buffer += lowValue += highValue - buffer.result() - } - } - - loop(low, high) - } - - // 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 + /** + * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. Each iteration uses + * `miniBatchFraction` fraction of the data to calculate the gradient. + * + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @param stepSize Step size to be used for each iteration of gradient descent. + * @param regParam Regularization parameter. + * @param miniBatchFraction Fraction of data to be used per iteration. + */ + def train( + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double) + : RidgeRegressionModel = + { + new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction, true).run(input) } -} - -/** - * Top-level methods for calling Ridge Regression. - */ -object RidgeRegression { - // NOTE(shivaram): We use multiple train methods instead of default arguments to support - // Java programs. /** - * 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. + * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using the specified step size. We use the entire data set to + * update the gradient in each iteration. * - * @param input RDD of (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 + * @param input RDD of (label, array of features) pairs. + * @param stepSize Step size to be used for each iteration of Gradient Descent. + * @param regParam Regularization parameter. + * @param numIterations Number of iterations of gradient descent to run. + * @return a RidgeRegressionModel which has the weights and offset from training. */ def train( - input: RDD[LabeledPoint], - lambdaLow: Double, - lambdaHigh: Double) - : RidgeRegressionModel = + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + regParam: Double) + : RidgeRegressionModel = { - new RidgeRegression(lambdaLow, lambdaHigh).train(input) + train(input, numIterations, stepSize, regParam, 1.0) } /** - * 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. + * Train a RidgeRegression model given an RDD of (label, features) pairs. We run a fixed number + * of iterations of gradient descent using a step size of 1.0. We use the entire data set to + * update the gradient in each iteration. * - * @param input RDD of (response, array of features) pairs. + * @param input RDD of (label, array of features) pairs. + * @param numIterations Number of iterations of gradient descent to run. + * @return a RidgeRegressionModel which has the weights and offset from training. */ - def train(input: RDD[LabeledPoint]) : RidgeRegressionModel = { - train(input, 0.0, 100.0) + def train( + input: RDD[LabeledPoint], + numIterations: Int) + : RidgeRegressionModel = + { + train(input, numIterations, 1.0, 1.0, 1.0) } def main(args: Array[String]) { - if (args.length != 2) { - println("Usage: RidgeRegression ") + if (args.length != 5) { + println("Usage: RidgeRegression ") System.exit(1) } val sc = new SparkContext(args(0), "RidgeRegression") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = RidgeRegression.train(data, 0, 1000) + val model = RidgeRegressionWithSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + sc.stop() } } diff --git a/mllib/src/main/scala/spark/mllib/util/LinearRegressionDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LinearRegressionDataGenerator.scala new file mode 100644 index 00000000000..39e2a30b55d --- /dev/null +++ b/mllib/src/main/scala/spark/mllib/util/LinearRegressionDataGenerator.scala @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.mllib.util + +import scala.util.Random + +import org.jblas.DoubleMatrix + +import spark.{RDD, SparkContext} +import spark.mllib.regression.LabeledPoint + +/** + * Generate sample data used for LinearRegression. This class generates + * uniformly random values for every feature and adds Gaussian noise with mean `eps` to the + * response variable `Y`. + * + */ +object LinearRegressionDataGenerator { + + /** + * Generate an RDD containing sample data for LinearRegression. + * + * @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. + * + * @return RDD of LabeledPoint containing sample data. + */ + def generateLinearRDD( + sc: SparkContext, + nexamples: Int, + nfeatures: Int, + eps: Double, + nparts: Int = 2, + intercept: Double = 0.0) : RDD[LabeledPoint] = { + org.jblas.util.Random.seed(42) + // Random values distributed uniformly in [-0.5, 0.5] + val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) + w.put(0, 0, 10) + w.put(1, 0, 10) + + val data: RDD[LabeledPoint] = 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).add(intercept) + + 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 => + LabeledPoint(yObs.get(i, 0), X.getRow(i).toArray) + } + } + data + } + + def main(args: Array[String]) { + if (args.length < 2) { + println("Usage: LinearRegressionGenerator " + + " [num_examples] [num_features] [num_partitions]") + System.exit(1) + } + + val sparkMaster: String = args(0) + val outputPath: String = args(1) + val nexamples: Int = if (args.length > 2) args(2).toInt else 1000 + val nfeatures: Int = if (args.length > 3) args(3).toInt else 100 + val parts: Int = if (args.length > 4) args(4).toInt else 2 + val eps = 10 + + val sc = new SparkContext(sparkMaster, "LinearRegressionDataGenerator") + val data = generateLinearRDD(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 index 4d329168bed..08dce723b82 100644 --- a/mllib/src/main/scala/spark/mllib/util/RidgeRegressionDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/RidgeRegressionDataGenerator.scala @@ -48,7 +48,8 @@ object RidgeRegressionDataGenerator { nexamples: Int, nfeatures: Int, eps: Double, - nparts: Int = 2) : RDD[LabeledPoint] = { + nparts: Int = 2, + intercept: Double = 0.0) : RDD[LabeledPoint] = { org.jblas.util.Random.seed(42) // Random values distributed uniformly in [-0.5, 0.5] val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) @@ -60,7 +61,7 @@ object RidgeRegressionDataGenerator { val examplesInPartition = nexamples / nparts val X = DoubleMatrix.rand(examplesInPartition, nfeatures) - val y = X.mmul(w) + val y = X.mmul(w).add(intercept) val rnd = new Random(42 + p) diff --git a/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java new file mode 100644 index 00000000000..14d3d4ef398 --- /dev/null +++ b/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.mllib.regression; + +import java.io.Serializable; +import java.util.List; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import spark.api.java.JavaRDD; +import spark.api.java.JavaSparkContext; + +public class JavaLinearRegressionSuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaLinearRegressionSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + System.clearProperty("spark.driver.port"); + } + + int validatePrediction(List validationData, LinearRegressionModel model) { + int numAccurate = 0; + for (LabeledPoint point: validationData) { + Double prediction = model.predict(point.features()); + // A prediction is off if the prediction is more than 0.5 away from expected value. + if (Math.abs(prediction - point.label()) <= 0.5) { + numAccurate++; + } + } + return numAccurate; + } + + @Test + public void runLinearRegressionUsingConstructor() { + int nPoints = 10000; + double A = 2.0; + double[] weights = {-1.5, 1.0e-2}; + + JavaRDD testRDD = sc.parallelize(LinearRegressionSuite.generateLinearRegressionInputAsList(A, + weights, nPoints, 42), 2).cache(); + List validationData = + LinearRegressionSuite.generateLinearRegressionInputAsList(A, weights, nPoints, 17); + + LinearRegressionWithSGD svmSGDImpl = new LinearRegressionWithSGD(); + svmSGDImpl.optimizer().setStepSize(1.0) + .setRegParam(0.01) + .setNumIterations(20); + LinearRegressionModel model = svmSGDImpl.run(testRDD.rdd()); + + int numAccurate = validatePrediction(validationData, model); + Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); + } + + @Test + public void runLinearRegressionUsingStaticMethods() { + int nPoints = 10000; + double A = 2.0; + double[] weights = {-1.5, 1.0e-2}; + + JavaRDD testRDD = sc.parallelize(LinearRegressionSuite.generateLinearRegressionInputAsList(A, + weights, nPoints, 42), 2).cache(); + List validationData = + LinearRegressionSuite.generateLinearRegressionInputAsList(A, weights, nPoints, 17); + + LinearRegressionModel model = LinearRegressionWithSGD.train(testRDD.rdd(), 100, 1.0, 1.0); + + int numAccurate = validatePrediction(validationData, model); + Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); + } + +} diff --git a/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java b/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java new file mode 100644 index 00000000000..4f379b51d5b --- /dev/null +++ b/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.mllib.regression; + +import java.io.Serializable; +import java.util.List; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import spark.api.java.JavaRDD; +import spark.api.java.JavaSparkContext; + +public class JavaRidgeRegressionSuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaRidgeRegressionSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + System.clearProperty("spark.driver.port"); + } + + int validatePrediction(List validationData, RidgeRegressionModel model) { + int numAccurate = 0; + for (LabeledPoint point: validationData) { + Double prediction = model.predict(point.features()); + // A prediction is off if the prediction is more than 0.5 away from expected value. + if (Math.abs(prediction - point.label()) <= 0.5) { + numAccurate++; + } + } + return numAccurate; + } + + @Test + public void runRidgeRegressionUsingConstructor() { + int nPoints = 10000; + double A = 2.0; + double[] weights = {-1.5, 1.0e-2}; + + JavaRDD testRDD = sc.parallelize(RidgeRegressionSuite.generateRidgeRegressionInputAsList(A, + weights, nPoints, 42), 2).cache(); + List validationData = + RidgeRegressionSuite.generateRidgeRegressionInputAsList(A, weights, nPoints, 17); + + RidgeRegressionWithSGD svmSGDImpl = new RidgeRegressionWithSGD(); + svmSGDImpl.optimizer().setStepSize(1.0) + .setRegParam(0.01) + .setNumIterations(20); + RidgeRegressionModel model = svmSGDImpl.run(testRDD.rdd()); + + int numAccurate = validatePrediction(validationData, model); + Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); + } + + @Test + public void runRidgeRegressionUsingStaticMethods() { + int nPoints = 10000; + double A = 2.0; + double[] weights = {-1.5, 1.0e-2}; + + JavaRDD testRDD = sc.parallelize(RidgeRegressionSuite.generateRidgeRegressionInputAsList(A, + weights, nPoints, 42), 2).cache(); + List validationData = + RidgeRegressionSuite.generateRidgeRegressionInputAsList(A, weights, nPoints, 17); + + RidgeRegressionModel model = RidgeRegressionWithSGD.train(testRDD.rdd(), 100, 1.0, 0.01, 1.0); + + int numAccurate = validatePrediction(validationData, model); + Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); + } + +} diff --git a/mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala new file mode 100644 index 00000000000..c794c1cac59 --- /dev/null +++ b/mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.mllib.regression + +import scala.collection.JavaConversions._ +import scala.util.Random + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import spark.SparkContext +import spark.SparkContext._ +import spark.mllib.util.LinearRegressionDataGenerator +import spark.mllib.regression.LabeledPoint +import org.jblas.DoubleMatrix + +object LinearRegressionSuite { + + def generateLinearRegressionInputAsList( + intercept: Double, + weights: Array[Double], + nPoints: Int, + seed: Int): java.util.List[LabeledPoint] = { + seqAsJavaList(generateLinearRegressionInput(intercept, weights, nPoints, seed)) + } + + + // Generate noisy input of the form Y = x.dot(weights) + intercept + noise + def generateLinearRegressionInput( + intercept: Double, + weights: Array[Double], + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val rnd = new Random(seed) + val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) + val x = Array.fill[Array[Double]](nPoints)( + Array.fill[Double](weights.length)(rnd.nextGaussian())) + val y = x.map(xi => + (new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) + intercept + 0.1 * rnd.nextGaussian() + ) + y.zip(x).map(p => LabeledPoint(p._1, p._2)) + } + +} + +class LinearRegressionSuite extends FunSuite with BeforeAndAfterAll { + @transient private var sc: SparkContext = _ + + override def beforeAll() { + sc = new SparkContext("local", "test") + } + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + // Test if we can correctly learn Y = 3 + 10*X1 + 10*X2 when + // X1 and X2 are collinear. + test("multi-collinear variables") { + val testRDD = LinearRegressionDataGenerator.generateLinearRDD(sc, 100, 2, 0.0, intercept=3.0).cache() + val linReg = new LinearRegressionWithSGD() + linReg.optimizer.setNumIterations(1000).setStepSize(1.0) + + val model = linReg.run(testRDD) + + assert(model.intercept >= 2.5 && model.intercept <= 3.5) + assert(model.weights.length === 2) + assert(model.weights(0) >= 9.0 && model.weights(0) <= 11.0) + assert(model.weights(1) >= 9.0 && model.weights(1) <= 11.0) + } +} diff --git a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala index e2b244894d4..aaac083ad9e 100644 --- a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala @@ -17,6 +17,7 @@ package spark.mllib.regression +import scala.collection.JavaConversions._ import scala.util.Random import org.scalatest.BeforeAndAfterAll @@ -24,6 +25,37 @@ import org.scalatest.FunSuite import spark.SparkContext import spark.SparkContext._ +import spark.mllib.util.RidgeRegressionDataGenerator +import org.jblas.DoubleMatrix + +object RidgeRegressionSuite { + + def generateRidgeRegressionInputAsList( + intercept: Double, + weights: Array[Double], + nPoints: Int, + seed: Int): java.util.List[LabeledPoint] = { + seqAsJavaList(generateRidgeRegressionInput(intercept, weights, nPoints, seed)) + } + + + // Generate noisy input of the form Y = x.dot(weights) + intercept + noise + def generateRidgeRegressionInput( + intercept: Double, + weights: Array[Double], + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val rnd = new Random(seed) + val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) + val x = Array.fill[Array[Double]](nPoints)( + Array.fill[Double](weights.length)(rnd.nextGaussian())) + val y = x.map(xi => + (new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) + intercept + 0.1 * rnd.nextGaussian() + ) + y.zip(x).map(p => LabeledPoint(p._1, p._2)) + } + +} class RidgeRegressionSuite extends FunSuite with BeforeAndAfterAll { @@ -38,31 +70,31 @@ class RidgeRegressionSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } - // Test if we can correctly learn Y = 3 + X1 + X2 when + // Test if we can correctly learn Y = 3 + 10*X1 + 10*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 testRDD = RidgeRegressionDataGenerator.generateRidgeRDD(sc, 100, 2, 0.0, intercept=3.0).cache() + val ridgeReg = new RidgeRegressionWithSGD() + ridgeReg.optimizer.setNumIterations(1000).setRegParam(0.0).setStepSize(1.0) - val xMat = (0 until 20).map(i => Array(x1(i), x2(i))).toArray + val model = ridgeReg.run(testRDD) - val y = xMat.map(i => 3 + i(0) + i(1)) - val testData = (0 until 20).map(i => LabeledPoint(y(i), xMat(i))).toArray + assert(model.intercept >= 2.5 && model.intercept <= 3.5) + assert(model.weights.length === 2) + assert(model.weights(0) >= 9.0 && model.weights(0) <= 11.0) + assert(model.weights(1) >= 9.0 && model.weights(1) <= 11.0) + } - val testRDD = sc.parallelize(testData, 2) - testRDD.cache() - val ridgeReg = new RidgeRegression().setLowLambda(0) - .setHighLambda(10) + test("multi-collinear variables with regularization") { + val testRDD = RidgeRegressionDataGenerator.generateRidgeRDD(sc, 100, 2, 0.0, intercept=3.0).cache() + val ridgeReg = new RidgeRegressionWithSGD() + ridgeReg.optimizer.setNumIterations(1000).setRegParam(1.0).setStepSize(1.0) - val model = ridgeReg.train(testRDD) + val model = ridgeReg.run(testRDD) - assert(model.intercept >= 2.9 && model.intercept <= 3.1) + assert(model.intercept <= 5.0) 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) + assert(model.weights(0) <= 3.0) + assert(model.weights(1) <= 3.0) } } From b291db712e73fdff0c02946bac96e330b089409d Mon Sep 17 00:00:00 2001 From: Evan Sparks Date: Fri, 16 Aug 2013 17:48:26 -0700 Subject: [PATCH 0766/2521] Centralizing linear data generator and mllib regression tests to use it. --- .../spark/mllib/util/LassoDataGenerator.scala | 48 --------- ...erator.scala => LinearDataGenerator.scala} | 76 ++++++++++---- .../util/LinearRegressionDataGenerator.scala | 98 ------------------- .../mllib/regression/JavaLassoSuite.java | 11 ++- .../regression/JavaLinearRegressionSuite.java | 9 +- .../regression/JavaRidgeRegressionSuite.java | 9 +- .../spark/mllib/regression/LassoSuite.scala | 39 +------- .../regression/LinearRegressionSuite.scala | 38 +------ .../regression/RidgeRegressionSuite.scala | 38 +------ 9 files changed, 84 insertions(+), 282 deletions(-) delete mode 100644 mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala rename mllib/src/main/scala/spark/mllib/util/{RidgeRegressionDataGenerator.scala => LinearDataGenerator.scala} (56%) delete mode 100644 mllib/src/main/scala/spark/mllib/util/LinearRegressionDataGenerator.scala diff --git a/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala deleted file mode 100644 index eeb14fc4e32..00000000000 --- a/mllib/src/main/scala/spark/mllib/util/LassoDataGenerator.scala +++ /dev/null @@ -1,48 +0,0 @@ -package spark.mllib.util - -import scala.util.Random - -import org.jblas.DoubleMatrix - -import spark.{RDD, SparkContext} -import spark.mllib.regression.LabeledPoint - -/** - * Generate sample data used for Lasso Regression. This class generates uniform random values - * for the features and adds Gaussian noise with weight 0.1 to generate response variables. - */ -object LassoDataGenerator { - - def main(args: Array[String]) { - if (args.length < 2) { - println("Usage: LassoGenerator " + - " [num_examples] [num_features] [num_partitions]") - System.exit(1) - } - - val sparkMaster: String = args(0) - val outputPath: String = args(1) - val nexamples: Int = if (args.length > 2) args(2).toInt else 1000 - val nfeatures: Int = if (args.length > 3) args(3).toInt else 2 - val parts: Int = if (args.length > 4) args(4).toInt else 2 - - val sc = new SparkContext(sparkMaster, "LassoGenerator") - - val globalRnd = new Random(94720) - val trueWeights = new DoubleMatrix(1, nfeatures+1, - Array.fill[Double](nfeatures + 1) { globalRnd.nextGaussian() }:_*) - - val data: RDD[LabeledPoint] = sc.parallelize(0 until nexamples, parts).map { idx => - val rnd = new Random(42 + idx) - - val x = Array.fill[Double](nfeatures) { - rnd.nextDouble() * 2.0 - 1.0 - } - val y = (new DoubleMatrix(1, x.length, x:_*)).dot(trueWeights) + rnd.nextGaussian() * 0.1 - LabeledPoint(y, x) - } - - 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/LinearDataGenerator.scala similarity index 56% rename from mllib/src/main/scala/spark/mllib/util/RidgeRegressionDataGenerator.scala rename to mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala index 08dce723b82..8fe3ab47549 100644 --- a/mllib/src/main/scala/spark/mllib/util/RidgeRegressionDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala @@ -23,54 +23,92 @@ import org.jblas.DoubleMatrix import spark.{RDD, SparkContext} import spark.mllib.regression.LabeledPoint +import scala.collection.JavaConversions._ +import spark.mllib.regression.LabeledPoint /** - * Generate sample data used for RidgeRegression. This class generates + * Generate sample data used for Linear Data. This class generates * uniformly random values for every feature and adds Gaussian noise with mean `eps` to the * response variable `Y`. * */ -object RidgeRegressionDataGenerator { +object LinearDataGenerator { + + /** + * Return a Java List of synthetic data randomly generated according to a multi + * collinear model. + * @param intercept Data intercept + * @param weights Weights to be applied. + * @param nPoints Number of points in sample. + * @param seed Random seed + * @return Java List of input. + */ + def generateLinearInputAsList( + intercept: Double, + weights: Array[Double], + nPoints: Int, + seed: Int): java.util.List[LabeledPoint] = { + seqAsJavaList(generateLinearInput(intercept, weights, nPoints, seed)) + } /** - * Generate an RDD containing sample data for RidgeRegression. + * + * @param intercept Data intercept + * @param weights Weights to be applied. + * @param nPoints Number of points in sample. + * @param seed Random seed + * @param eps Epsilon scaling factor. + * @return + */ + def generateLinearInput( + intercept: Double, + weights: Array[Double], + nPoints: Int, + seed: Int, + eps: Double = 0.1): Seq[LabeledPoint] = { + + val rnd = new Random(seed) + val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) + val x = Array.fill[Array[Double]](nPoints)( + Array.fill[Double](weights.length)(rnd.nextGaussian())) + val y = x.map(xi => + (new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) + intercept + eps * rnd.nextGaussian() + ) + y.zip(x).map(p => LabeledPoint(p._1, p._2)) + } + + /** + * Generate an RDD containing sample data for Linear Regression models - including Ridge, Lasso, + * and uregularized variants. * * @param sc SparkContext to be used for generating the RDD. * @param nexamples Number of examples that will be contained in the RDD. * @param nfeatures Number of features to generate for each example. * @param eps Epsilon factor by which examples are scaled. + * @param weights Weights associated with the first weights.length features. * @param nparts Number of partitions in the RDD. Default value is 2. * * @return RDD of LabeledPoint containing sample data. */ - def generateRidgeRDD( + def generateLinearRDD( sc: SparkContext, nexamples: Int, nfeatures: Int, eps: Double, + weights: Array[Double] = Array[Double](), nparts: Int = 2, intercept: Double = 0.0) : RDD[LabeledPoint] = { org.jblas.util.Random.seed(42) // Random values distributed uniformly in [-0.5, 0.5] val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) - w.put(0, 0, 10) - w.put(1, 0, 10) + + (0 until weights.length.max(nfeatures)).map(i => w.put(i, 0, weights(i))) val data: RDD[LabeledPoint] = sc.parallelize(0 until nparts, nparts).flatMap { p => - org.jblas.util.Random.seed(42 + p) + val seed = 42+p val examplesInPartition = nexamples / nparts - val X = DoubleMatrix.rand(examplesInPartition, nfeatures) - val y = X.mmul(w).add(intercept) - - 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 => - LabeledPoint(yObs.get(i, 0), X.getRow(i).toArray) - } + generateLinearInput(intercept, w.toArray, examplesInPartition, seed, eps) } data } @@ -90,7 +128,7 @@ object RidgeRegressionDataGenerator { val eps = 10 val sc = new SparkContext(sparkMaster, "RidgeRegressionDataGenerator") - val data = generateRidgeRDD(sc, nexamples, nfeatures, eps, parts) + val data = generateLinearRDD(sc, nexamples, nfeatures, eps, nparts = parts) MLUtils.saveLabeledData(data, outputPath) sc.stop() diff --git a/mllib/src/main/scala/spark/mllib/util/LinearRegressionDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LinearRegressionDataGenerator.scala deleted file mode 100644 index 39e2a30b55d..00000000000 --- a/mllib/src/main/scala/spark/mllib/util/LinearRegressionDataGenerator.scala +++ /dev/null @@ -1,98 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package spark.mllib.util - -import scala.util.Random - -import org.jblas.DoubleMatrix - -import spark.{RDD, SparkContext} -import spark.mllib.regression.LabeledPoint - -/** - * Generate sample data used for LinearRegression. This class generates - * uniformly random values for every feature and adds Gaussian noise with mean `eps` to the - * response variable `Y`. - * - */ -object LinearRegressionDataGenerator { - - /** - * Generate an RDD containing sample data for LinearRegression. - * - * @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. - * - * @return RDD of LabeledPoint containing sample data. - */ - def generateLinearRDD( - sc: SparkContext, - nexamples: Int, - nfeatures: Int, - eps: Double, - nparts: Int = 2, - intercept: Double = 0.0) : RDD[LabeledPoint] = { - org.jblas.util.Random.seed(42) - // Random values distributed uniformly in [-0.5, 0.5] - val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) - w.put(0, 0, 10) - w.put(1, 0, 10) - - val data: RDD[LabeledPoint] = 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).add(intercept) - - 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 => - LabeledPoint(yObs.get(i, 0), X.getRow(i).toArray) - } - } - data - } - - def main(args: Array[String]) { - if (args.length < 2) { - println("Usage: LinearRegressionGenerator " + - " [num_examples] [num_features] [num_partitions]") - System.exit(1) - } - - val sparkMaster: String = args(0) - val outputPath: String = args(1) - val nexamples: Int = if (args.length > 2) args(2).toInt else 1000 - val nfeatures: Int = if (args.length > 3) args(3).toInt else 100 - val parts: Int = if (args.length > 4) args(4).toInt else 2 - val eps = 10 - - val sc = new SparkContext(sparkMaster, "LinearRegressionDataGenerator") - val data = generateLinearRDD(sc, nexamples, nfeatures, eps, parts) - - MLUtils.saveLabeledData(data, outputPath) - sc.stop() - } -} diff --git a/mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java b/mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java index e26d7b385c7..8d692c2d0d5 100644 --- a/mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java +++ b/mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java @@ -27,6 +27,7 @@ import spark.api.java.JavaRDD; import spark.api.java.JavaSparkContext; +import spark.mllib.util.LinearDataGenerator; public class JavaLassoSuite implements Serializable { private transient JavaSparkContext sc; @@ -61,10 +62,10 @@ public void runLassoUsingConstructor() { double A = 2.0; double[] weights = {-1.5, 1.0e-2}; - JavaRDD testRDD = sc.parallelize(LassoSuite.generateLassoInputAsList(A, - weights, nPoints, 42), 2).cache(); + JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, + weights, nPoints, 42), 2).cache(); List validationData = - LassoSuite.generateLassoInputAsList(A, weights, nPoints, 17); + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17); LassoWithSGD svmSGDImpl = new LassoWithSGD(); svmSGDImpl.optimizer().setStepSize(1.0) @@ -82,10 +83,10 @@ public void runLassoUsingStaticMethods() { double A = 2.0; double[] weights = {-1.5, 1.0e-2}; - JavaRDD testRDD = sc.parallelize(LassoSuite.generateLassoInputAsList(A, + JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42), 2).cache(); List validationData = - LassoSuite.generateLassoInputAsList(A, weights, nPoints, 17); + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17); LassoModel model = LassoWithSGD.train(testRDD.rdd(), 100, 1.0, 0.01, 1.0); diff --git a/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java index 14d3d4ef398..d2d8a62980c 100644 --- a/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java @@ -27,6 +27,7 @@ import spark.api.java.JavaRDD; import spark.api.java.JavaSparkContext; +import spark.mllib.util.LinearDataGenerator; public class JavaLinearRegressionSuite implements Serializable { private transient JavaSparkContext sc; @@ -61,10 +62,10 @@ public void runLinearRegressionUsingConstructor() { double A = 2.0; double[] weights = {-1.5, 1.0e-2}; - JavaRDD testRDD = sc.parallelize(LinearRegressionSuite.generateLinearRegressionInputAsList(A, + JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42), 2).cache(); List validationData = - LinearRegressionSuite.generateLinearRegressionInputAsList(A, weights, nPoints, 17); + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17); LinearRegressionWithSGD svmSGDImpl = new LinearRegressionWithSGD(); svmSGDImpl.optimizer().setStepSize(1.0) @@ -82,10 +83,10 @@ public void runLinearRegressionUsingStaticMethods() { double A = 2.0; double[] weights = {-1.5, 1.0e-2}; - JavaRDD testRDD = sc.parallelize(LinearRegressionSuite.generateLinearRegressionInputAsList(A, + JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42), 2).cache(); List validationData = - LinearRegressionSuite.generateLinearRegressionInputAsList(A, weights, nPoints, 17); + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17); LinearRegressionModel model = LinearRegressionWithSGD.train(testRDD.rdd(), 100, 1.0, 1.0); diff --git a/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java b/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java index 4f379b51d5b..72ab875985b 100644 --- a/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java +++ b/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java @@ -27,6 +27,7 @@ import spark.api.java.JavaRDD; import spark.api.java.JavaSparkContext; +import spark.mllib.util.LinearDataGenerator; public class JavaRidgeRegressionSuite implements Serializable { private transient JavaSparkContext sc; @@ -61,10 +62,10 @@ public void runRidgeRegressionUsingConstructor() { double A = 2.0; double[] weights = {-1.5, 1.0e-2}; - JavaRDD testRDD = sc.parallelize(RidgeRegressionSuite.generateRidgeRegressionInputAsList(A, + JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42), 2).cache(); List validationData = - RidgeRegressionSuite.generateRidgeRegressionInputAsList(A, weights, nPoints, 17); + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17); RidgeRegressionWithSGD svmSGDImpl = new RidgeRegressionWithSGD(); svmSGDImpl.optimizer().setStepSize(1.0) @@ -82,10 +83,10 @@ public void runRidgeRegressionUsingStaticMethods() { double A = 2.0; double[] weights = {-1.5, 1.0e-2}; - JavaRDD testRDD = sc.parallelize(RidgeRegressionSuite.generateRidgeRegressionInputAsList(A, + JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42), 2).cache(); List validationData = - RidgeRegressionSuite.generateRidgeRegressionInputAsList(A, weights, nPoints, 17); + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17); RidgeRegressionModel model = RidgeRegressionWithSGD.train(testRDD.rdd(), 100, 1.0, 0.01, 1.0); diff --git a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala index 55a738f1e49..622dbbab7fe 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala @@ -24,37 +24,8 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext +import spark.mllib.util.LinearDataGenerator -import org.jblas.DoubleMatrix - -object LassoSuite { - - def generateLassoInputAsList( - intercept: Double, - weights: Array[Double], - nPoints: Int, - seed: Int): java.util.List[LabeledPoint] = { - seqAsJavaList(generateLassoInput(intercept, weights, nPoints, seed)) - } - - - // Generate noisy input of the form Y = x.dot(weights) + intercept + noise - def generateLassoInput( - intercept: Double, - weights: Array[Double], - nPoints: Int, - seed: Int): Seq[LabeledPoint] = { - val rnd = new Random(seed) - val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) - val x = Array.fill[Array[Double]](nPoints)( - Array.fill[Double](weights.length)(rnd.nextGaussian())) - val y = x.map(xi => - (new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) + intercept + 0.1 * rnd.nextGaussian() - ) - y.zip(x).map(p => LabeledPoint(p._1, p._2)) - } - -} class LassoSuite extends FunSuite with BeforeAndAfterAll { @transient private var sc: SparkContext = _ @@ -85,7 +56,7 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { val B = -1.5 val C = 1.0e-2 - val testData = LassoSuite.generateLassoInput(A, Array[Double](B,C), nPoints, 42) + val testData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 42) val testRDD = sc.parallelize(testData, 2) testRDD.cache() @@ -101,7 +72,7 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]") - val validationData = LassoSuite.generateLassoInput(A, Array[Double](B,C), nPoints, 17) + val validationData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 17) val validationRDD = sc.parallelize(validationData, 2) // Test prediction on RDD. @@ -118,7 +89,7 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { val B = -1.5 val C = 1.0e-2 - val testData = LassoSuite.generateLassoInput(A, Array[Double](B,C), nPoints, 42) + val testData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 42) val initialB = -1.0 val initialC = -1.0 @@ -138,7 +109,7 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll { assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]") - val validationData = LassoSuite.generateLassoInput(A, Array[Double](B,C), nPoints, 17) + val validationData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 17) val validationRDD = sc.parallelize(validationData,2) // Test prediction on RDD. diff --git a/mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala index c794c1cac59..3d22b7d3856 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala @@ -17,46 +17,12 @@ package spark.mllib.regression -import scala.collection.JavaConversions._ -import scala.util.Random - import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import spark.SparkContext import spark.SparkContext._ -import spark.mllib.util.LinearRegressionDataGenerator -import spark.mllib.regression.LabeledPoint -import org.jblas.DoubleMatrix - -object LinearRegressionSuite { - - def generateLinearRegressionInputAsList( - intercept: Double, - weights: Array[Double], - nPoints: Int, - seed: Int): java.util.List[LabeledPoint] = { - seqAsJavaList(generateLinearRegressionInput(intercept, weights, nPoints, seed)) - } - - - // Generate noisy input of the form Y = x.dot(weights) + intercept + noise - def generateLinearRegressionInput( - intercept: Double, - weights: Array[Double], - nPoints: Int, - seed: Int): Seq[LabeledPoint] = { - val rnd = new Random(seed) - val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) - val x = Array.fill[Array[Double]](nPoints)( - Array.fill[Double](weights.length)(rnd.nextGaussian())) - val y = x.map(xi => - (new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) + intercept + 0.1 * rnd.nextGaussian() - ) - y.zip(x).map(p => LabeledPoint(p._1, p._2)) - } - -} +import spark.mllib.util.LinearDataGenerator class LinearRegressionSuite extends FunSuite with BeforeAndAfterAll { @transient private var sc: SparkContext = _ @@ -73,7 +39,7 @@ class LinearRegressionSuite extends FunSuite with BeforeAndAfterAll { // Test if we can correctly learn Y = 3 + 10*X1 + 10*X2 when // X1 and X2 are collinear. test("multi-collinear variables") { - val testRDD = LinearRegressionDataGenerator.generateLinearRDD(sc, 100, 2, 0.0, intercept=3.0).cache() + val testRDD = LinearDataGenerator.generateLinearRDD(sc, 100, 2, 0.0, Array(10.0, 10.0), intercept=3.0).cache() val linReg = new LinearRegressionWithSGD() linReg.optimizer.setNumIterations(1000).setStepSize(1.0) diff --git a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala index aaac083ad9e..0237ccdf87e 100644 --- a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala @@ -25,37 +25,7 @@ import org.scalatest.FunSuite import spark.SparkContext import spark.SparkContext._ -import spark.mllib.util.RidgeRegressionDataGenerator -import org.jblas.DoubleMatrix - -object RidgeRegressionSuite { - - def generateRidgeRegressionInputAsList( - intercept: Double, - weights: Array[Double], - nPoints: Int, - seed: Int): java.util.List[LabeledPoint] = { - seqAsJavaList(generateRidgeRegressionInput(intercept, weights, nPoints, seed)) - } - - - // Generate noisy input of the form Y = x.dot(weights) + intercept + noise - def generateRidgeRegressionInput( - intercept: Double, - weights: Array[Double], - nPoints: Int, - seed: Int): Seq[LabeledPoint] = { - val rnd = new Random(seed) - val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) - val x = Array.fill[Array[Double]](nPoints)( - Array.fill[Double](weights.length)(rnd.nextGaussian())) - val y = x.map(xi => - (new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) + intercept + 0.1 * rnd.nextGaussian() - ) - y.zip(x).map(p => LabeledPoint(p._1, p._2)) - } - -} +import spark.mllib.util.LinearDataGenerator class RidgeRegressionSuite extends FunSuite with BeforeAndAfterAll { @@ -73,7 +43,7 @@ class RidgeRegressionSuite extends FunSuite with BeforeAndAfterAll { // Test if we can correctly learn Y = 3 + 10*X1 + 10*X2 when // X1 and X2 are collinear. test("multi-collinear variables") { - val testRDD = RidgeRegressionDataGenerator.generateRidgeRDD(sc, 100, 2, 0.0, intercept=3.0).cache() + val testRDD = LinearDataGenerator.generateLinearRDD(sc, 100, 2, 0.0, Array(10.0, 10.0), intercept=3.0).cache() val ridgeReg = new RidgeRegressionWithSGD() ridgeReg.optimizer.setNumIterations(1000).setRegParam(0.0).setStepSize(1.0) @@ -86,7 +56,7 @@ class RidgeRegressionSuite extends FunSuite with BeforeAndAfterAll { } test("multi-collinear variables with regularization") { - val testRDD = RidgeRegressionDataGenerator.generateRidgeRDD(sc, 100, 2, 0.0, intercept=3.0).cache() + val testRDD = LinearDataGenerator.generateLinearRDD(sc, 100, 2, 0.0, Array(10.0, 10.0), intercept=3.0).cache() val ridgeReg = new RidgeRegressionWithSGD() ridgeReg.optimizer.setNumIterations(1000).setRegParam(1.0).setStepSize(1.0) @@ -94,7 +64,7 @@ class RidgeRegressionSuite extends FunSuite with BeforeAndAfterAll { assert(model.intercept <= 5.0) assert(model.weights.length === 2) - assert(model.weights(0) <= 3.0) + assert(model.weights(0) <= 4.0) assert(model.weights(1) <= 3.0) } } From 07fe910669b2ec15b6b5c1e5186df5036d05b9b1 Mon Sep 17 00:00:00 2001 From: Evan Sparks Date: Fri, 16 Aug 2013 18:00:20 -0700 Subject: [PATCH 0767/2521] Fixing typos in Java tests, and addressing alignment issues. --- .../spark/mllib/util/LinearDataGenerator.scala | 14 +++++++------- .../spark/mllib/regression/JavaLassoSuite.java | 6 +++--- .../regression/JavaLinearRegressionSuite.java | 6 +++--- .../mllib/regression/JavaRidgeRegressionSuite.java | 6 +++--- 4 files changed, 16 insertions(+), 16 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala index 8fe3ab47549..20e1656beb0 100644 --- a/mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala @@ -91,13 +91,13 @@ object LinearDataGenerator { * @return RDD of LabeledPoint containing sample data. */ def generateLinearRDD( - sc: SparkContext, - nexamples: Int, - nfeatures: Int, - eps: Double, - weights: Array[Double] = Array[Double](), - nparts: Int = 2, - intercept: Double = 0.0) : RDD[LabeledPoint] = { + sc: SparkContext, + nexamples: Int, + nfeatures: Int, + eps: Double, + weights: Array[Double] = Array[Double](), + nparts: Int = 2, + intercept: Double = 0.0) : RDD[LabeledPoint] = { org.jblas.util.Random.seed(42) // Random values distributed uniformly in [-0.5, 0.5] val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) diff --git a/mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java b/mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java index 8d692c2d0d5..428902e85c9 100644 --- a/mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java +++ b/mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java @@ -67,11 +67,11 @@ public void runLassoUsingConstructor() { List validationData = LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17); - LassoWithSGD svmSGDImpl = new LassoWithSGD(); - svmSGDImpl.optimizer().setStepSize(1.0) + LassoWithSGD lassoSGDImpl = new LassoWithSGD(); + lassoSGDImpl.optimizer().setStepSize(1.0) .setRegParam(0.01) .setNumIterations(20); - LassoModel model = svmSGDImpl.run(testRDD.rdd()); + LassoModel model = lassoSGDImpl.run(testRDD.rdd()); int numAccurate = validatePrediction(validationData, model); Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); diff --git a/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java index d2d8a62980c..9642e898449 100644 --- a/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java @@ -67,11 +67,11 @@ public void runLinearRegressionUsingConstructor() { List validationData = LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17); - LinearRegressionWithSGD svmSGDImpl = new LinearRegressionWithSGD(); - svmSGDImpl.optimizer().setStepSize(1.0) + LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(); + linSGDImpl.optimizer().setStepSize(1.0) .setRegParam(0.01) .setNumIterations(20); - LinearRegressionModel model = svmSGDImpl.run(testRDD.rdd()); + LinearRegressionModel model = linSGDImpl.run(testRDD.rdd()); int numAccurate = validatePrediction(validationData, model); Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); diff --git a/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java b/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java index 72ab875985b..5df6d8076d4 100644 --- a/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java +++ b/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java @@ -67,11 +67,11 @@ public void runRidgeRegressionUsingConstructor() { List validationData = LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17); - RidgeRegressionWithSGD svmSGDImpl = new RidgeRegressionWithSGD(); - svmSGDImpl.optimizer().setStepSize(1.0) + RidgeRegressionWithSGD ridgeSGDImpl = new RidgeRegressionWithSGD(); + ridgeSGDImpl.optimizer().setStepSize(1.0) .setRegParam(0.01) .setNumIterations(20); - RidgeRegressionModel model = svmSGDImpl.run(testRDD.rdd()); + RidgeRegressionModel model = ridgeSGDImpl.run(testRDD.rdd()); int numAccurate = validatePrediction(validationData, model); Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); From 44000b10ffed83ae605521701d104878f491f31c Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Sun, 18 Aug 2013 16:23:22 -0700 Subject: [PATCH 0768/2521] Make YARN POM file valid --- yarn/pom.xml | 61 ++++++++++++++++++++++++++++------------------------ 1 file changed, 33 insertions(+), 28 deletions(-) diff --git a/yarn/pom.xml b/yarn/pom.xml index 6acde8e98ca..07dd170eae5 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -30,34 +30,6 @@ Spark Project YARN Support http://spark-project.org/ - - - org.spark-project - spark-core - ${project.version} - - - org.apache.hadoop - hadoop-yarn-api - - - org.apache.hadoop - hadoop-yarn-common - - - org.apache.hadoop - hadoop-yarn-client - - - org.apache.avro - avro - - - org.apache.avro - avro-ipc - - - target/scala-${scala.version}/classes target/scala-${scala.version}/test-classes @@ -103,4 +75,37 @@ + + + + hadoop2-yarn + + + org.spark-project + spark-core + ${project.version} + + + org.apache.hadoop + hadoop-yarn-api + + + org.apache.hadoop + hadoop-yarn-common + + + org.apache.hadoop + hadoop-yarn-client + + + org.apache.avro + avro + + + org.apache.avro + avro-ipc + + + + From 47a7c4338a7e912d8677704204e98df15679322b Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Sun, 18 Aug 2013 16:58:08 -0700 Subject: [PATCH 0769/2521] Don't assume spark-examples JAR always exists --- bin/compute-classpath.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index f975d3bfb9c..7a21b3c4a14 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -67,8 +67,9 @@ function dev_classpath { 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" + for jar in `find "$EXAMPLES_DIR/target" -name 'spark-examples*[0-9T].jar'`; do + CLASSPATH="$CLASSPATH:$jar" + done fi CLASSPATH="$CLASSPATH:$BAGEL_DIR/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$MLLIB_DIR/target/scala-$SCALA_VERSION/classes" From 630281bf7671ffec65a4672361d3d9570cfe7a39 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sun, 18 Aug 2013 17:16:45 -0700 Subject: [PATCH 0770/2521] Corrected all indexed RDD tests. There appears to be an issue with subtract by key tests that needs to be investigated further. --- .../main/scala/spark/PairRDDFunctions.scala | 9 +- core/src/main/scala/spark/RDD.scala | 9 + .../src/main/scala/spark/rdd/IndexedRDD.scala | 302 ++++++------------ .../test/scala/spark/IndexedRDDSuite.scala | 206 ++++++++---- 4 files changed, 259 insertions(+), 267 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 469e8704097..6751da72af7 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -699,7 +699,14 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( def values: RDD[V] = self.map(_._2) - def index(existingIndex: RDD[JHashMap[K,Int]] = null): IndexedRDD[K,V] = + def indexed(numPartitions: Int): IndexedRDD[K,V] = + IndexedRDD(self.partitionBy(new HashPartitioner(numPartitions))) + + def indexed(partitioner: Partitioner): IndexedRDD[K,V] = + IndexedRDD(self.partitionBy(partitioner)) + + + def indexed(existingIndex: RDD[BlockIndex[K]] = null): IndexedRDD[K,V] = IndexedRDD(self, existingIndex) private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 503ea6ccbf4..1c5095fa865 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -54,6 +54,9 @@ import spark.rdd.ZippedRDD import spark.rdd.ZippedPartitionsRDD2 import spark.rdd.ZippedPartitionsRDD3 import spark.rdd.ZippedPartitionsRDD4 + +import spark.rdd.{IndexedRDD, BlockIndex} + import spark.storage.StorageLevel import spark.util.BoundedPriorityQueue @@ -770,6 +773,12 @@ abstract class RDD[T: ClassManifest]( return buf.toArray } + + def makeIndex(partitioner: Option[Partitioner] = None): RDD[BlockIndex[T]] = + IndexedRDD.makeIndex(this, partitioner) + + + /** * Return the first element in this RDD. */ diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index 37ff11e6396..b2a2180b8f1 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -34,7 +34,13 @@ import spark.Partitioner._ -// import java.io.{ObjectOutputStream, IOException} +/** + * And index RDD + */ +class BlockIndex[@specialized K: ClassManifest] extends JHashMap[K,Int] + +//type BlockIndex[@specialized K: ClassManifest] = JHashMap[K,Int] + /** * An IndexedRDD is an RDD[(K,V)] where each K is unique. @@ -43,20 +49,26 @@ import spark.Partitioner._ * be used to accelerate join and aggregation operations. */ class IndexedRDD[K: ClassManifest, V: ClassManifest]( - val index: RDD[ JHashMap[K, Int] ], + val index: RDD[ BlockIndex[K] ], val valuesRDD: RDD[ Seq[Seq[V]] ]) extends RDD[(K, V)](index.context, List(new OneToOneDependency(index), new OneToOneDependency(valuesRDD)) ) { - //with PairRDDFunctions[K,V] { - - - val tuples = new ZippedRDD[JHashMap[K, Int], Seq[Seq[V]]](index.context, index, valuesRDD) + /** + * An internal representation of the maps and block managers + */ + protected val tuples = new ZippedRDD(index.context, index, valuesRDD) override val partitioner = index.partitioner + + override def getPartitions: Array[Partition] = tuples.getPartitions - override def getPreferredLocations(s: Partition): Seq[String] = tuples.getPreferredLocations(s) + + + override def getPreferredLocations(s: Partition): Seq[String] = + tuples.getPreferredLocations(s) + override def cache: IndexedRDD[K,V] = { index.cache @@ -77,9 +89,10 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( */ def mapValues[U: ClassManifest](f: V => U): IndexedRDD[K, U] = { val cleanF = index.context.clean(f) - val newValues = valuesRDD.mapPartitions(_.map{ values => - values.map{_.map(x => f(x))} - }, true) + val newValues = valuesRDD.mapPartitions(_.map(values => values.map{ + case null => null + case row => row.map(x => f(x)) + }), true) new IndexedRDD[K,U](index, newValues) } @@ -90,9 +103,10 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( */ def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): IndexedRDD[K,U] = { val cleanF = index.context.clean(f) - val newValues = valuesRDD.mapPartitions(_.map{ values => - values.map{_.flatMap(x => f(x))} - }, true) + val newValues = valuesRDD.mapPartitions(_.map(values => values.map{ + case null => null + case row => row.flatMap(x => f(x)) + }), true) new IndexedRDD[K,U](index, newValues) } @@ -106,15 +120,10 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) * - `mergeCombiners`, to combine two C's into a single one. - * - * In addition, users can control the partitioning of the output RDD, and whether to perform - * map-side aggregation (if a mapper can produce multiple items with the same key). */ def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, - partitioner: Partitioner, - mapSideCombine: Boolean = true, serializerClass: String = null): IndexedRDD[K, C] = { val newValues = valuesRDD.mapPartitions( _.map{ groups: Seq[Seq[V]] => @@ -131,35 +140,12 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( new IndexedRDD[K,C](index, newValues) } - - /** - * Simplified version of combineByKey that hash-partitions the output RDD. - */ - def combineByKey[C: ClassManifest](createCombiner: V => C, - mergeValue: (C, V) => C, - mergeCombiners: (C, C) => C, - numPartitions: Int): IndexedRDD[K, C] = { - combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions)) - } - - - /** - * Simplified version of combineByKey that hash-partitions the resulting RDD using the - * existing partitioner/parallelism level. - */ - def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, - mergeCombiners: (C, C) => C) - : IndexedRDD[K, C] = { - combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(index)) - } - - /** * Merge the values for each key using an associative function and a neutral "zero value" which may * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for * list concatenation, 0 for addition, or 1 for multiplication.). */ - def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): IndexedRDD[K, V] = { + def foldByKey(zeroValue: V)(func: (V, V) => V): IndexedRDD[K, V] = { // 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) @@ -168,45 +154,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // 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) - } - - /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). - */ - def foldByKey(zeroValue: V, numPartitions: Int)(func: (V, V) => V): IndexedRDD[K, V] = { - foldByKey(zeroValue, new HashPartitioner(numPartitions))(func) - } - - - /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). - */ - def foldByKey(zeroValue: V)(func: (V, V) => V): IndexedRDD[K, V] = { - foldByKey(zeroValue, defaultPartitioner(index))(func) - } - - - /** - * Merge the values for each key using an associative reduce function. This will also perform - * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. - */ - def reduceByKey(partitioner: Partitioner, func: (V, V) => V): IndexedRDD[K, V] = { - combineByKey[V]((v: V) => v, func, func, partitioner) - } - - /** - * Merge the values for each key using an associative reduce function. This will also perform - * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions. - */ - def reduceByKey(func: (V, V) => V, numPartitions: Int): IndexedRDD[K, V] = { - reduceByKey(new HashPartitioner(numPartitions), func) + combineByKey[V]((v: V) => func(createZero(), v), func, func) } /** @@ -216,26 +164,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * parallelism level. */ def reduceByKey(func: (V, V) => V): IndexedRDD[K, V] = { - reduceByKey(defaultPartitioner(index), func) - } - - - /** - * Group the values for each key in the RDD into a single sequence. Allows controlling the - * partitioning of the resulting key-value pair RDD by passing a Partitioner. - */ - def groupByKey(partitioner: Partitioner): IndexedRDD[K, Seq[V]] = { - val newValues = valuesRDD.mapPartitions(_.map{ar => ar.map{s => Seq(s)} }, true) - new IndexedRDD[K, Seq[V]](index, newValues) - } - - - /** - * Group the values for each key in the RDD into a single sequence. Hash-partitions the - * resulting RDD with into `numPartitions` partitions. - */ - def groupByKey(numPartitions: Int): IndexedRDD[K, Seq[V]] = { - groupByKey(new HashPartitioner(numPartitions)) + combineByKey[V]((v: V) => v, func, func) } @@ -244,7 +173,8 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * resulting RDD with the existing partitioner/parallelism level. */ def groupByKey(): IndexedRDD[K, Seq[V]] = { - groupByKey(defaultPartitioner(index)) + val newValues = valuesRDD.mapPartitions(_.map{ar => ar.map{s => Seq(s)} }, true) + new IndexedRDD[K, Seq[V]](index, newValues) } @@ -252,8 +182,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): - IndexedRDD[K, (Seq[V], Seq[W])] = { + def cogroup[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { //RDD[(K, (Seq[V], Seq[W]))] = { other match { case other: IndexedRDD[_, _] if other.index == index => { @@ -291,7 +220,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( assert(!thisIter.hasNext()) val otherIndex = otherIter.next() assert(!otherIter.hasNext()) - val newIndex = new JHashMap[K, Int]() + val newIndex = new BlockIndex[K]() // @todo Merge only the keys that correspond to non-null values // Merge the keys newIndex.putAll(thisIndex) @@ -356,9 +285,9 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val (thisIndex, thisValues) = thisTuplesIter.next() assert(!thisTuplesIter.hasNext()) // Construct a new index - val newIndex = thisIndex.clone().asInstanceOf[JHashMap[K, Int]] + val newIndex = thisIndex.clone().asInstanceOf[BlockIndex[K]] // Construct a new array Buffer to store the values - val newValues = ArrayBuffer.fill[(Seq[V], ArrayBuffer[W])](thisValues.size)(null) + val newValues = ArrayBuffer.fill[(Seq[V], Seq[W])](thisValues.size)(null) // populate the newValues with the values in this IndexedRDD for ((k,i) <- thisIndex) { if (thisValues(i) != null) { @@ -375,12 +304,21 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) } else { val ind = newIndex.get(k) - newValues(ind)._2.append(w) + if(newValues(ind) == null) { + // If the other key was in the index but not in the values + // of this indexed RDD then create a new values entry for it + newValues(ind) = (Seq.empty[V], ArrayBuffer(w)) + } else { + newValues(ind)._2.asInstanceOf[ArrayBuffer[W]].append(w) + } } } // Finalize the new values array val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = - newValues.view.map{ case (s, ab) => Seq((s, ab.toSeq)) }.toSeq + newValues.view.map{ + case null => null + case (s, ab) => Seq((s, ab.toSeq)) + }.toSeq List( (newIndex, newValuesArray) ).iterator }).cache() @@ -394,13 +332,6 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } - /** - * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the - * list of values for that key in `this` as well as `other`. - */ - def cogroup[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { - cogroup(other, defaultPartitioner(this, other)) - } // /** // * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a @@ -411,14 +342,6 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // cogroup(other1, other2, defaultPartitioner(this, other1, other2)) // } - /** - * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the - * list of values for that key in `this` as well as `other`. - */ - def cogroup[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (Seq[V], Seq[W])] = { - cogroup(other, new HashPartitioner(numPartitions)) - } - // /** // * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a // * tuple with the list of values for that key in `this`, `other1` and `other2`. @@ -430,7 +353,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( /** Alias for cogroup. */ def groupWith[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { - cogroup(other, defaultPartitioner(this, other)) + cogroup(other) } // /** Alias for cogroup. */ @@ -439,26 +362,28 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( // cogroup(other1, other2, defaultPartitioner(self, other1, other2)) // } + /** * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and - * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. + * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): IndexedRDD[K, (V, W)] = { - cogroup(other, partitioner).flatMapValues { + def join[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, W)] = { + cogroup(other).flatMapValues { case (vs, ws) => for (v <- vs.iterator; w <- ws.iterator) yield (v, w) } } + /** * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the - * pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to - * partition the output RDD. + * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output + * using the existing partitioner/parallelism level. */ - def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): IndexedRDD[K, (V, Option[W])] = { - cogroup(other, partitioner).flatMapValues { + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, Option[W])] = { + cogroup(other).flatMapValues { case (vs, ws) => if (ws.isEmpty) { vs.iterator.map(v => (v, None)) @@ -466,17 +391,18 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w)) } } + } + /** * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the - * pair (k, (None, w)) if no elements in `this` have key k. Uses the given Partitioner to - * partition the output RDD. + * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting + * RDD using the existing partitioner/parallelism level. */ - def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner) - : IndexedRDD[K, (Option[V], W)] = { - cogroup(other, partitioner).flatMapValues { + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Option[V], W)] = { + cogroup(other).flatMapValues { case (vs, ws) => if (vs.isEmpty) { ws.iterator.map(w => (None, w)) @@ -484,69 +410,10 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w) } } - } - - /** - * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each - * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and - * (k, v2) is in `other`. Performs a hash join across the cluster. - */ - def join[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, W)] = { - join(other, defaultPartitioner(this, other)) - } - - /** - * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each - * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and - * (k, v2) is in `other`. Performs a hash join across the cluster. - */ - def join[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (V, W)] = { - join(other, new HashPartitioner(numPartitions)) - } - - /** - * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the - * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the - * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output - * using the existing partitioner/parallelism level. - */ - def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, Option[W])] = { - leftOuterJoin(other, defaultPartitioner(this, other)) - } - - /** - * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the - * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the - * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output - * into `numPartitions` partitions. - */ - def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (V, Option[W])] = { - leftOuterJoin(other, new HashPartitioner(numPartitions)) - } - - /** - * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the - * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the - * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting - * RDD using the existing partitioner/parallelism level. - */ - def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Option[V], W)] = { - rightOuterJoin(other, defaultPartitioner(this, other)) - } - /** - * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the - * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the - * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting - * RDD into the given number of partitions. - */ - def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): IndexedRDD[K, (Option[V], W)] = { - rightOuterJoin(other, new HashPartitioner(numPartitions)) } - - /** * Provide the RDD[(K,V)] equivalent output. */ @@ -571,7 +438,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( object IndexedRDD { def apply[K: ClassManifest, V: ClassManifest]( tbl: RDD[(K,V)], - existingIndex: RDD[JHashMap[K,Int]] = null ): IndexedRDD[K, V] = { + existingIndex: RDD[BlockIndex[K]] = null ): IndexedRDD[K, V] = { if (existingIndex == null) { // Shuffle the table (if necessary) @@ -581,7 +448,7 @@ object IndexedRDD { } else { tbl } val groups = shuffledTbl.mapPartitions( iter => { - val indexMap = new JHashMap[K, Int]() + val indexMap = new BlockIndex[K]() val values = new ArrayBuffer[Seq[V]]() for ((k,v) <- iter){ if(!indexMap.contains(k)) { @@ -617,12 +484,12 @@ object IndexedRDD { val values = index.zipPartitions(shuffledTbl)( (indexIter, tblIter) => { // There is only one map - val index: JHashMap[K,Int] = indexIter.next() + val index = indexIter.next() assert(!indexIter.hasNext()) val values = new Array[Seq[V]](index.size) for ((k,v) <- tblIter) { if (!index.contains(k)) { - throw new SparkException("Error: Try to bind an external index " + + throw new SparkException("Error: Trying to bind an external index " + "to an RDD which contains keys that are not in the index.") } val ind = index(k) @@ -638,6 +505,41 @@ object IndexedRDD { } } + /** + * Construct and index of the unique values in a given RDD. + */ + def makeIndex[K: ClassManifest](keys: RDD[K], + partitioner: Option[Partitioner] = None): RDD[BlockIndex[K]] = { + + + // Ugly hack :-(. In order to partition the keys they must have values. + val tbl = keys.mapPartitions(_.map(k => (k, false)), true) + // Shuffle the table (if necessary) + val shuffledTbl = partitioner match { + case None => { + if (tbl.partitioner.isEmpty) { + new ShuffledRDD[K, Boolean](tbl, Partitioner.defaultPartitioner(tbl)) + } else { tbl } + } + case Some(partitioner) => + tbl.partitionBy(partitioner) +// new ShuffledRDD[K, Boolean](tbl, partitioner) + } + + + val index = shuffledTbl.mapPartitions( iter => { + val indexMap = new BlockIndex[K]() + for ( (k,_) <- iter ){ + if(!indexMap.contains(k)){ + val ind = indexMap.size + indexMap.put(k, ind) + } + } + List(indexMap).iterator + }, true).cache + index + } + } diff --git a/core/src/test/scala/spark/IndexedRDDSuite.scala b/core/src/test/scala/spark/IndexedRDDSuite.scala index f30e1f57fa1..c1433c50950 100644 --- a/core/src/test/scala/spark/IndexedRDDSuite.scala +++ b/core/src/test/scala/spark/IndexedRDDSuite.scala @@ -30,10 +30,13 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashSet import spark.rdd.ShuffledRDD +import spark.rdd.IndexedRDD + import spark.SparkContext._ import spark._ + class IndexedRDDSuite extends FunSuite with SharedSparkContext { def lineage(rdd: RDD[_]): collection.mutable.HashSet[RDD[_]] = { @@ -49,7 +52,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("groupByKey") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).index() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).indexed() val groups = pairs.groupByKey().collect() assert(groups.size === 2) val valuesFor1 = groups.find(_._1 == 1).get._2 @@ -59,7 +62,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("groupByKey with duplicates") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).index() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() val groups = pairs.groupByKey().collect() assert(groups.size === 2) val valuesFor1 = groups.find(_._1 == 1).get._2 @@ -69,7 +72,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("groupByKey with negative key hash codes") { - val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1))).index() + val pairs = sc.parallelize(Array((-1, 1), (-1, 2), (-1, 3), (2, 1))).indexed() val groups = pairs.groupByKey().collect() assert(groups.size === 2) val valuesForMinus1 = groups.find(_._1 == -1).get._2 @@ -79,8 +82,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("groupByKey with many output partitions") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).index() - val groups = pairs.groupByKey(10).collect() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1))).indexed(10) + 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)) @@ -89,13 +92,13 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("reduceByKey") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).index() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() 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))).index() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed() val sums = pairs.reduceByKey(_+_).collectAsMap() assert(sums.size === 2) assert(sums(1) === 7) @@ -103,8 +106,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("reduceByKey with many output partitons") { - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).index() - val sums = pairs.reduceByKey(_+_, 10).collect() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(10) + val sums = pairs.reduceByKey(_+_).collect() assert(sums.toSet === Set((1, 7), (2, 1))) } @@ -113,7 +116,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { 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).index() + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).indexed(p) val sums = pairs.reduceByKey(_+_) assert(sums.collect().toSet === Set((1, 4), (0, 1))) assert(sums.partitioner === Some(p)) @@ -123,22 +126,10 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { assert(deps.filter(_.isInstanceOf[ShuffledRDD[_,_]]).size === 1) // 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("joinIndexVsPair") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val joined = rdd1.join(rdd2).collect() assert(joined.size === 4) @@ -151,8 +142,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("joinIndexVsIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() val joined = rdd1.join(rdd2).collect() assert(joined.size === 4) assert(joined.toSet === Set( @@ -164,8 +155,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("joinSharedIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4,-4), (4, 4) )).index() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index(rdd1.index) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4,-4), (4, 4) )).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(rdd1.index) val joined = rdd1.join(rdd2).collect() assert(joined.size === 6) assert(joined.toSet === Set( @@ -180,8 +171,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { test("join all-to-all") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))).index() - val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))).index(rdd1.index) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (1, 3))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (1, 'y'))).indexed(rdd1.index) val joined = rdd1.join(rdd2).collect() assert(joined.size === 6) assert(joined.toSet === Set( @@ -195,7 +186,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("leftOuterJoinIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() + val index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val joined = rdd1.leftOuterJoin(rdd2).collect() assert(joined.size === 5) @@ -209,8 +201,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("leftOuterJoinIndextoIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).index() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() val joined = rdd1.leftOuterJoin(rdd2).collect() assert(joined.size === 5) assert(joined.toSet === Set( @@ -223,8 +215,8 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("leftOuterJoinIndextoSharedIndex") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4, -4))).index() - val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).index(rdd1.index) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1), (4, -4))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(rdd1.index) val joined = rdd1.leftOuterJoin(rdd2).collect() assert(joined.size === 6) assert(joined.toSet === Set( @@ -237,8 +229,25 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { )) } +test("leftOuterJoinIndextoIndexExternal") { + val index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) + 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 index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val joined = rdd1.rightOuterJoin(rdd2).collect() assert(joined.size === 5) @@ -251,17 +260,58 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { )) } - test("join with no matches") { - val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + test("rightOuterJoinIndex2Index") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed() + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed() + 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("rightOuterJoinIndex2Indexshared") { + val index = sc.parallelize( 1 to 6 ).makeIndex() + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) + 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 index") { + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) 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 no matches shared index") { + val index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))).indexed(index) + 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 rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(10) val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) - val joined = rdd1.join(rdd2, 10).collect() + val joined = rdd1.join(rdd2).collect() assert(joined.size === 4) assert(joined.toSet === Set( (1, (1, 'x')), @@ -271,9 +321,25 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { )) } + test("join with many output partitions and two indices") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(10) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(20) + 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("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 index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))).indexed(index) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))).indexed(index) val joined = rdd1.groupWith(rdd2).collect() assert(joined.size === 4) assert(joined.toSet === Set( @@ -294,7 +360,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("keys and values") { - val rdd = sc.parallelize(Array((1, "a"), (2, "b"))) + val rdd = sc.parallelize(Array((1, "a"), (2, "b"))).indexed() assert(rdd.keys.collect().toList === List(1, 2)) assert(rdd.values.collect().toList === List("a", "b")) } @@ -309,12 +375,14 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { 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("default partitioner uses largest partitioner indexed to indexed") { + // val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2).indexed() + // val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000).indexed() + // val c = a.join(b) + // assert(c.partitions.size === 2000) + // } + + test("subtract") { val a = sc.parallelize(Array(1, 2, 3), 2) @@ -331,7 +399,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { 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) + val a = sc.parallelize(Array((1, "a"), (2, "b"), (3, "c"))).indexed(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) @@ -341,36 +409,42 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { } test("subtractByKey") { - val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2) + + val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 2).indexed() 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("subtractByKey with narrow dependency") { + // // use a deterministic partitioner + // val p = new Partitioner() { + // def numPartitions = 5 + // def getPartition(key: Any) = key.asInstanceOf[Int] + // } + + // val index = sc.parallelize( 1 to 6 ).makeIndex(Some(p)) + // // partitionBy so we have a narrow dependency + // val a = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c"))).indexed(index) + // // more partitions/no partitioner so a shuffle dependency + // val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4).indexed(index) + // 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 index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(index) 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 index = IndexedRDD.makeIndex( sc.parallelize( 1 to 6 ) ) + + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))).indexed(index) val bufs = pairs.mapValues(v => ArrayBuffer(v)).cache() // Fold the values using in-place mutation val sums = bufs.foldByKey(new ArrayBuffer[Int])(_ ++= _).collect() From bdd861c6c34ca3fa158707b3a1bed91ef928c1e3 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Sun, 18 Aug 2013 18:28:10 -0700 Subject: [PATCH 0771/2521] Fix Maven build with Hadoop 0.23.9 --- core/pom.xml | 8 ++++++++ pom.xml | 11 ----------- 2 files changed, 8 insertions(+), 11 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 90d279c6350..28709060922 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -36,6 +36,14 @@ org.apache.hadoop hadoop-client + + org.apache.avro + avro + + + org.apache.avro + avro-ipc + org.eclipse.jetty jetty-server diff --git a/pom.xml b/pom.xml index 4714576f3be..e7445319ddc 100644 --- a/pom.xml +++ b/pom.xml @@ -733,17 +733,6 @@ - - - org.apache.avro - avro - 1.7.4 - - - org.apache.avro - avro-ipc - 1.7.4 - From 23f4622affc25685bb74ce09ed56ef3515551d17 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Sun, 18 Aug 2013 18:53:57 -0700 Subject: [PATCH 0772/2521] Remove redundant dependencies from POMs --- bagel/pom.xml | 4 ---- examples/pom.xml | 4 ---- mllib/pom.xml | 4 ---- repl-bin/pom.xml | 5 ----- repl/pom.xml | 15 --------------- streaming/pom.xml | 5 ----- tools/pom.xml | 5 ----- 7 files changed, 42 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index ae40f38e431..cbcf8d12393 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -37,10 +37,6 @@ spark-core ${project.version} - - org.apache.hadoop - hadoop-client - org.eclipse.jetty jetty-server diff --git a/examples/pom.xml b/examples/pom.xml index 023ad8cb458..0db52b8691c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -47,10 +47,6 @@ spark-mllib ${project.version} - - org.apache.hadoop - hadoop-client - org.apache.hbase hbase diff --git a/mllib/pom.xml b/mllib/pom.xml index 3292f6dad03..ab31d5734e4 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -37,10 +37,6 @@ spark-core ${project.version} - - org.apache.hadoop - hadoop-client - org.eclipse.jetty jetty-server diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index f132c44fb9e..919e35f2404 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -55,11 +55,6 @@ ${project.version} runtime - - org.apache.hadoop - hadoop-client - runtime - diff --git a/repl/pom.xml b/repl/pom.xml index 82e26defbc7..5bc9a99c5c4 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -48,21 +48,6 @@ ${project.version} runtime - - org.apache.hadoop - hadoop-client - provided - - - org.apache.avro - avro - provided - - - org.apache.avro - avro-ipc - provided - org.eclipse.jetty jetty-server diff --git a/streaming/pom.xml b/streaming/pom.xml index 18609901223..5c0582d6fb6 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -45,11 +45,6 @@ spark-core ${project.version} - - org.apache.hadoop - hadoop-client - provided - org.eclipse.jetty jetty-server diff --git a/tools/pom.xml b/tools/pom.xml index 9177d85b2f4..95b5e80e5b1 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -41,11 +41,6 @@ spark-streaming ${project.version} - - org.apache.hadoop - hadoop-client - provided - org.scalatest scalatest_${scala.version} From 90a04dab8d9a2a9a372cea7cdf46cc0fd0f2f76c Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 28 Jul 2013 22:02:01 -0400 Subject: [PATCH 0773/2521] Initial work towards scheduler refactoring: - Replace use of hostPort vs host in Task.preferredLocations with a TaskLocation class that contains either an executorId and a host or just a host. This is part of a bigger effort to eliminate hostPort based data structures and just use executorID, since the hostPort vs host stuff is confusing (and not checkable with static typing, leading to ugly debug code), and hostPorts are not provided by Mesos. - Replaced most hostPort-based data structures and fields as above. - Simplified ClusterTaskSetManager to deal with preferred locations in a more concise way and generally be more concise. - Updated the way ClusterTaskSetManager handles racks: instead of enqueueing a task to a separate queue for all the hosts in the rack, which would create lots of large queues, have one queue per rack name. - Removed non-local fallback stuff in ClusterScheduler that tried to launch less-local tasks on a node once the local ones were all assigned. This change didn't work because many cluster schedulers send offers for just one node at a time (even the standalone and YARN ones do so as nodes join the cluster one by one). Thus, lots of non-local tasks would be assigned even though a node with locality for them would be able to receive tasks just a short time later. - Renamed MapOutputTracker "generations" to "epochs". --- .../cluster/YarnClusterScheduler.scala | 7 - .../main/scala/spark/MapOutputTracker.scala | 62 +-- core/src/main/scala/spark/RDD.scala | 4 +- .../main/scala/spark/executor/Executor.scala | 12 +- core/src/main/scala/spark/rdd/BlockRDD.scala | 7 +- .../scala/spark/rdd/ZippedPartitionsRDD.scala | 2 + .../scala/spark/scheduler/DAGScheduler.scala | 79 +-- .../spark/scheduler/DAGSchedulerEvent.scala | 4 +- .../scala/spark/scheduler/ResultTask.scala | 15 +- .../spark/scheduler/ShuffleMapTask.scala | 15 +- .../src/main/scala/spark/scheduler/Task.scala | 4 +- .../scala/spark/scheduler/TaskLocation.scala | 32 ++ .../scheduler/TaskSchedulerListener.scala | 2 +- .../scheduler/cluster/ClusterScheduler.scala | 283 +++------- .../cluster/ClusterTaskSetManager.scala | 510 ++++++------------ .../cluster/StandaloneSchedulerBackend.scala | 15 +- .../spark/scheduler/cluster/TaskInfo.scala | 4 +- .../scheduler/cluster/TaskLocality.scala | 32 ++ .../scheduler/cluster/TaskSetManager.scala | 13 +- .../spark/scheduler/cluster/WorkerOffer.scala | 3 +- .../scheduler/local/LocalScheduler.scala | 3 +- .../scheduler/local/LocalTaskSetManager.scala | 21 +- .../scala/spark/storage/BlockManager.scala | 60 +-- .../main/scala/spark/ui/jobs/StagePage.scala | 2 +- .../scala/spark/MapOutputTrackerSuite.scala | 12 +- .../scheduler/ClusterSchedulerSuite.scala | 6 +- .../spark/scheduler/DAGSchedulerSuite.scala | 26 +- 27 files changed, 484 insertions(+), 751 deletions(-) create mode 100644 core/src/main/scala/spark/scheduler/TaskLocation.scala create mode 100644 core/src/main/scala/spark/scheduler/cluster/TaskLocality.scala 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 307d96111c8..bb58353e0cc 100644 --- a/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala +++ b/core/src/hadoop2-yarn/scala/spark/scheduler/cluster/YarnClusterScheduler.scala @@ -41,13 +41,6 @@ private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) if (retval != null) Some(retval) else None } - // By default, if rack is unknown, return nothing - override def getCachedHostsForRack(rack: String): Option[Set[String]] = { - if (rack == None || rack == null) return None - - YarnAllocationHandler.fetchCachedHostsForRack(rack) - } - override def postStartHook() { val sparkContextInitialized = ApplicationMaster.sparkContextInitialized(sc) if (sparkContextInitialized){ diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala index 2c417e31dba..0cd0341a724 100644 --- a/core/src/main/scala/spark/MapOutputTracker.scala +++ b/core/src/main/scala/spark/MapOutputTracker.scala @@ -64,11 +64,11 @@ private[spark] class MapOutputTracker extends Logging { // Incremented every time a fetch fails so that client nodes know to clear // their cache of map output locations if this happens. - private var generation: Long = 0 - private val generationLock = new java.lang.Object + private var epoch: Long = 0 + private val epochLock = new java.lang.Object // Cache a serialized version of the output statuses for each shuffle to send them out faster - var cacheGeneration = generation + var cacheEpoch = epoch private val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]] val metadataCleaner = new MetadataCleaner("MapOutputTracker", this.cleanup) @@ -108,10 +108,10 @@ private[spark] class MapOutputTracker extends Logging { def registerMapOutputs( shuffleId: Int, statuses: Array[MapStatus], - changeGeneration: Boolean = false) { + changeEpoch: Boolean = false) { mapStatuses.put(shuffleId, Array[MapStatus]() ++ statuses) - if (changeGeneration) { - incrementGeneration() + if (changeEpoch) { + incrementEpoch() } } @@ -124,7 +124,7 @@ private[spark] class MapOutputTracker extends Logging { array(mapId) = null } } - incrementGeneration() + incrementEpoch() } else { throw new SparkException("unregisterMapOutput called for nonexistent shuffle ID") } @@ -206,58 +206,58 @@ private[spark] class MapOutputTracker extends Logging { trackerActor = null } - // Called on master to increment the generation number - def incrementGeneration() { - generationLock.synchronized { - generation += 1 - logDebug("Increasing generation to " + generation) + // Called on master to increment the epoch number + def incrementEpoch() { + epochLock.synchronized { + epoch += 1 + logDebug("Increasing epoch to " + epoch) } } - // Called on master or workers to get current generation number - def getGeneration: Long = { - generationLock.synchronized { - return generation + // Called on master or workers to get current epoch number + def getEpoch: Long = { + epochLock.synchronized { + return epoch } } - // Called on workers to update the generation number, potentially clearing old outputs - // because of a fetch failure. (Each Mesos task calls this with the latest generation + // Called on workers to update the epoch number, potentially clearing old outputs + // because of a fetch failure. (Each worker task calls this with the latest epoch // number on the master at the time it was created.) - def updateGeneration(newGen: Long) { - generationLock.synchronized { - if (newGen > generation) { - logInfo("Updating generation to " + newGen + " and clearing cache") + def updateEpoch(newEpoch: Long) { + epochLock.synchronized { + if (newEpoch > epoch) { + logInfo("Updating epoch to " + newEpoch + " and clearing cache") // mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]] mapStatuses.clear() - generation = newGen + epoch = newEpoch } } } def getSerializedLocations(shuffleId: Int): Array[Byte] = { var statuses: Array[MapStatus] = null - var generationGotten: Long = -1 - generationLock.synchronized { - if (generation > cacheGeneration) { + var epochGotten: Long = -1 + epochLock.synchronized { + if (epoch > cacheEpoch) { cachedSerializedStatuses.clear() - cacheGeneration = generation + cacheEpoch = epoch } cachedSerializedStatuses.get(shuffleId) match { case Some(bytes) => return bytes case None => statuses = mapStatuses(shuffleId) - generationGotten = generation + epochGotten = epoch } } // If we got here, we failed to find the serialized locations in the cache, so we pulled // out a snapshot of the locations as "locs"; let's serialize and return that val bytes = serializeStatuses(statuses) logInfo("Size of output statuses for shuffle %d is %d bytes".format(shuffleId, bytes.length)) - // Add them into the table only if the generation hasn't changed while we were working - generationLock.synchronized { - if (generation == generationGotten) { + // Add them into the table only if the epoch hasn't changed while we were working + epochLock.synchronized { + if (epoch == epochGotten) { cachedSerializedStatuses(shuffleId) = bytes } } diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 503ea6ccbf4..f5767a38586 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -221,8 +221,8 @@ abstract class RDD[T: ClassManifest]( } /** - * Get the preferred location of a split, taking into account whether the - * RDD is checkpointed or not. + * Get the preferred locations of a partition (as hostnames), taking into account whether the + * RDD is checkpointed. */ final def preferredLocations(split: Partition): Seq[String] = { checkpointRDD.map(_.getPreferredLocations(split)).getOrElse { diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 05a960d7c50..036c7191adc 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -32,8 +32,12 @@ import spark._ /** * The Mesos executor for Spark. */ -private[spark] class Executor(executorId: String, slaveHostname: String, properties: Seq[(String, String)]) extends Logging { - +private[spark] class Executor( + executorId: String, + slaveHostname: String, + properties: Seq[(String, String)]) + extends Logging +{ // Application dependencies (added through SparkContext) that we've fetched so far on this node. // Each map holds the master's timestamp for the version of that file or JAR we got. private val currentFiles: HashMap[String, Long] = new HashMap[String, Long]() @@ -125,8 +129,8 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert 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) + logInfo("Its epoch is " + task.epoch) + env.mapOutputTracker.updateEpoch(task.epoch) taskStart = System.currentTimeMillis() val value = task.run(taskId.toInt) val taskFinish = System.currentTimeMillis() diff --git a/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/spark/rdd/BlockRDD.scala index 0ebb722d736..03800584ae0 100644 --- a/core/src/main/scala/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/spark/rdd/BlockRDD.scala @@ -28,13 +28,12 @@ private[spark] class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[String]) extends RDD[T](sc, Nil) { - @transient lazy val locations_ = BlockManager.blockIdsToExecutorLocations(blockIds, SparkEnv.get) + @transient lazy val locations_ = BlockManager.blockIdsToHosts(blockIds, SparkEnv.get) override def getPartitions: Array[Partition] = (0 until blockIds.size).map(i => { new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition] }).toArray - override def compute(split: Partition, context: TaskContext): Iterator[T] = { val blockManager = SparkEnv.get.blockManager val blockId = split.asInstanceOf[BlockRDDPartition].blockId @@ -45,8 +44,8 @@ class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[St } } - override def getPreferredLocations(split: Partition): Seq[String] = + override def getPreferredLocations(split: Partition): Seq[String] = { locations_(split.asInstanceOf[BlockRDDPartition].blockId) - + } } diff --git a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala index 6a4fa13ad61..51f5cc32517 100644 --- a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala @@ -55,6 +55,8 @@ abstract class ZippedPartitionsBaseRDD[V: ClassManifest]( } override def getPreferredLocations(s: Partition): Seq[String] = { + // TODO(matei): Fix this for hostPort + // 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 diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index fbf3f4c8071..2f7e6d98f84 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -72,8 +72,8 @@ class DAGScheduler( } // Called by TaskScheduler when a host is added - override def executorGained(execId: String, hostPort: String) { - eventQueue.put(ExecutorGained(execId, hostPort)) + override def executorGained(execId: String, host: String) { + eventQueue.put(ExecutorGained(execId, host)) } // Called by TaskScheduler to cancel an entire TaskSet due to repeated failures. @@ -104,15 +104,16 @@ class DAGScheduler( private val listenerBus = new SparkListenerBus() - var cacheLocs = new HashMap[Int, Array[List[String]]] + // Contains the locations that each RDD's partitions are cached on + private val cacheLocs = new HashMap[Int, Array[Seq[TaskLocation]]] - // For tracking failed nodes, we use the MapOutputTracker's generation number, which is - // sent with every task. When we detect a node failing, we note the current generation number - // and failed executor, increment it for new tasks, and use this to ignore stray ShuffleMapTask - // results. - // TODO: Garbage collect information about failure generations when we know there are no more + // For tracking failed nodes, we use the MapOutputTracker's epoch number, which is sent with + // every task. When we detect a node failing, we note the current epoch number and failed + // executor, increment it for new tasks, and use this to ignore stray ShuffleMapTask results. + // + // TODO: Garbage collect information about failure epochs when we know there are no more // stray messages to detect. - val failedGeneration = new HashMap[String, Long] + val failedEpoch = new HashMap[String, Long] val idToActiveJob = new HashMap[Int, ActiveJob] @@ -141,11 +142,13 @@ class DAGScheduler( listenerBus.addListener(listener) } - private def getCacheLocs(rdd: RDD[_]): Array[List[String]] = { + private def getCacheLocs(rdd: RDD[_]): Array[Seq[TaskLocation]] = { 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, blockManagerMaster) - cacheLocs(rdd.id) = blockIds.map(locs.getOrElse(_, Nil)) + val locs = BlockManager.blockIdsToBlockManagers(blockIds, env, blockManagerMaster) + cacheLocs(rdd.id) = blockIds.map { id => + locs.getOrElse(id, Nil).map(bm => TaskLocation(bm.host, bm.executorId)) + } } cacheLocs(rdd.id) } @@ -345,8 +348,8 @@ class DAGScheduler( submitStage(finalStage) } - case ExecutorGained(execId, hostPort) => - handleExecutorGained(execId, hostPort) + case ExecutorGained(execId, host) => + handleExecutorGained(execId, host) case ExecutorLost(execId) => handleExecutorLost(execId) @@ -508,7 +511,7 @@ class DAGScheduler( } else { // This is a final stage; figure out its job's missing partitions val job = resultStageToJob(stage) - for (id <- 0 until job.numPartitions if (!job.finished(id))) { + for (id <- 0 until job.numPartitions if !job.finished(id)) { val partition = job.partitions(id) val locs = getPreferredLocs(stage.rdd, partition) tasks += new ResultTask(stage.id, stage.rdd, job.func, partition, locs, id) @@ -518,7 +521,7 @@ class DAGScheduler( // should be "StageSubmitted" first and then "JobEnded" val properties = idToActiveJob(stage.priority).properties listenerBus.post(SparkListenerStageSubmitted(stage, tasks.size, properties)) - + 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 @@ -599,7 +602,7 @@ class DAGScheduler( val status = event.result.asInstanceOf[MapStatus] val execId = status.location.executorId logDebug("ShuffleMapTask finished on " + execId) - if (failedGeneration.contains(execId) && smt.generation <= failedGeneration(execId)) { + if (failedEpoch.contains(execId) && smt.epoch <= failedEpoch(execId)) { logInfo("Ignoring possibly bogus ShuffleMapTask completion from " + execId) } else { stage.addOutputLoc(smt.partition, status) @@ -611,11 +614,11 @@ class DAGScheduler( logInfo("waiting: " + waiting) logInfo("failed: " + failed) if (stage.shuffleDep != None) { - // We supply true to increment the generation number here in case this is a + // We supply true to increment the epoch number here in case this is a // recomputation of the map outputs. In that case, some nodes may have cached // locations with holes (from when we detected the error) and will need the - // generation incremented to refetch them. - // TODO: Only increment the generation number if this is not the first time + // epoch incremented to refetch them. + // TODO: Only increment the epoch number if this is not the first time // we registered these map outputs. mapOutputTracker.registerMapOutputs( stage.shuffleDep.get.shuffleId, @@ -674,7 +677,7 @@ class DAGScheduler( lastFetchFailureTime = System.currentTimeMillis() // TODO: Use pluggable clock // TODO: mark the executor as failed only if there were lots of fetch failures on it if (bmAddress != null) { - handleExecutorLost(bmAddress.executorId, Some(task.generation)) + handleExecutorLost(bmAddress.executorId, Some(task.epoch)) } case ExceptionFailure(className, description, stackTrace, metrics) => @@ -690,14 +693,14 @@ class DAGScheduler( * Responds to an executor being lost. This is called inside the event loop, so it assumes it can * modify the scheduler's internal state. Use executorLost() to post a loss event from outside. * - * Optionally the generation during which the failure was caught can be passed to avoid allowing + * Optionally the epoch during which the failure was caught can be passed to avoid allowing * stray fetch failures from possibly retriggering the detection of a node as lost. */ - private def handleExecutorLost(execId: String, maybeGeneration: Option[Long] = None) { - val currentGeneration = maybeGeneration.getOrElse(mapOutputTracker.getGeneration) - if (!failedGeneration.contains(execId) || failedGeneration(execId) < currentGeneration) { - failedGeneration(execId) = currentGeneration - logInfo("Executor lost: %s (generation %d)".format(execId, currentGeneration)) + private def handleExecutorLost(execId: String, maybeEpoch: Option[Long] = None) { + val currentEpoch = maybeEpoch.getOrElse(mapOutputTracker.getEpoch) + if (!failedEpoch.contains(execId) || failedEpoch(execId) < currentEpoch) { + failedEpoch(execId) = currentEpoch + logInfo("Executor lost: %s (epoch %d)".format(execId, currentEpoch)) blockManagerMaster.removeExecutor(execId) // TODO: This will be really slow if we keep accumulating shuffle map stages for ((shuffleId, stage) <- shuffleToMapStage) { @@ -706,20 +709,20 @@ class DAGScheduler( mapOutputTracker.registerMapOutputs(shuffleId, locs, true) } if (shuffleToMapStage.isEmpty) { - mapOutputTracker.incrementGeneration() + mapOutputTracker.incrementEpoch() } clearCacheLocs() } else { logDebug("Additional executor lost message for " + execId + - "(generation " + currentGeneration + ")") + "(epoch " + currentEpoch + ")") } } - private def handleExecutorGained(execId: String, hostPort: String) { - // remove from failedGeneration(execId) ? - if (failedGeneration.contains(execId)) { - logInfo("Host gained which was in lost list earlier: " + hostPort) - failedGeneration -= execId + private def handleExecutorGained(execId: String, host: String) { + // remove from failedEpoch(execId) ? + if (failedEpoch.contains(execId)) { + logInfo("Host gained which was in lost list earlier: " + host) + failedEpoch -= execId } } @@ -774,16 +777,16 @@ class DAGScheduler( visitedRdds.contains(target.rdd) } - private def getPreferredLocs(rdd: RDD[_], partition: Int): List[String] = { + private def getPreferredLocs(rdd: RDD[_], partition: Int): Seq[TaskLocation] = { // If the partition is cached, return the cache locations val cached = getCacheLocs(rdd)(partition) - if (cached != Nil) { + if (!cached.isEmpty) { return cached } // If the RDD has some placement preferences (as is the case for input RDDs), get those val rddPrefs = rdd.preferredLocations(rdd.partitions(partition)).toList - if (rddPrefs != Nil) { - return rddPrefs + if (!rddPrefs.isEmpty) { + return rddPrefs.map(host => TaskLocation(host)) } // If the RDD has narrow dependencies, pick the first partition of the first narrow dep // that has any placement preferences. Ideally we would choose based on transfer sizes, diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala index 3b4ee6287aa..b8ba0e92394 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala @@ -54,9 +54,7 @@ private[spark] case class CompletionEvent( taskMetrics: TaskMetrics) extends DAGSchedulerEvent -private[spark] case class ExecutorGained(execId: String, hostPort: String) extends DAGSchedulerEvent { - Utils.checkHostPort(hostPort, "Required hostport") -} +private[spark] case class ExecutorGained(execId: String, host: String) extends DAGSchedulerEvent private[spark] case class ExecutorLost(execId: String) extends DAGSchedulerEvent diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala index 832ca18b8ca..d066df5dc1d 100644 --- a/core/src/main/scala/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/spark/scheduler/ResultTask.scala @@ -73,7 +73,7 @@ private[spark] class ResultTask[T, U]( var rdd: RDD[T], var func: (TaskContext, Iterator[T]) => U, var partition: Int, - @transient locs: Seq[String], + @transient locs: Seq[TaskLocation], val outputId: Int) extends Task[U](stageId) with Externalizable { @@ -85,11 +85,8 @@ private[spark] class ResultTask[T, U]( rdd.partitions(partition) } - private val preferredLocs: Seq[String] = if (locs == null) Nil else locs.toSet.toSeq - - { - // DEBUG code - preferredLocs.foreach (hostPort => Utils.checkHost(Utils.parseHostPort(hostPort)._1, "preferredLocs : " + preferredLocs)) + @transient private val preferredLocs: Seq[TaskLocation] = { + if (locs == null) Nil else locs.toSet.toSeq } override def run(attemptId: Long): U = { @@ -102,7 +99,7 @@ private[spark] class ResultTask[T, U]( } } - override def preferredLocations: Seq[String] = preferredLocs + override def preferredLocations: Seq[TaskLocation] = preferredLocs override def toString = "ResultTask(" + stageId + ", " + partition + ")" @@ -116,7 +113,7 @@ private[spark] class ResultTask[T, U]( out.write(bytes) out.writeInt(partition) out.writeInt(outputId) - out.writeLong(generation) + out.writeLong(epoch) out.writeObject(split) } } @@ -131,7 +128,7 @@ private[spark] class ResultTask[T, U]( func = func_.asInstanceOf[(TaskContext, Iterator[T]) => U] partition = in.readInt() val outputId = in.readInt() - generation = in.readLong() + epoch = in.readLong() split = in.readObject().asInstanceOf[Partition] } } diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index e3bb6d1e609..2dbaef24ac5 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -88,18 +88,15 @@ private[spark] class ShuffleMapTask( var rdd: RDD[_], var dep: ShuffleDependency[_,_], var partition: Int, - @transient private var locs: Seq[String]) + @transient private var locs: Seq[TaskLocation]) extends Task[MapStatus](stageId) with Externalizable with Logging { protected def this() = this(0, null, null, 0, null) - @transient private val preferredLocs: Seq[String] = if (locs == null) Nil else locs.toSet.toSeq - - { - // DEBUG code - preferredLocs.foreach (hostPort => Utils.checkHost(Utils.parseHostPort(hostPort)._1, "preferredLocs : " + preferredLocs)) + @transient private val preferredLocs: Seq[TaskLocation] = { + if (locs == null) Nil else locs.toSet.toSeq } var split = if (rdd == null) null else rdd.partitions(partition) @@ -112,7 +109,7 @@ private[spark] class ShuffleMapTask( out.writeInt(bytes.length) out.write(bytes) out.writeInt(partition) - out.writeLong(generation) + out.writeLong(epoch) out.writeObject(split) } } @@ -126,7 +123,7 @@ private[spark] class ShuffleMapTask( rdd = rdd_ dep = dep_ partition = in.readInt() - generation = in.readLong() + epoch = in.readLong() split = in.readObject().asInstanceOf[Partition] } @@ -186,7 +183,7 @@ private[spark] class ShuffleMapTask( } } - override def preferredLocations: Seq[String] = preferredLocs + override def preferredLocations: Seq[TaskLocation] = preferredLocs override def toString = "ShuffleMapTask(%d, %d)".format(stageId, partition) } diff --git a/core/src/main/scala/spark/scheduler/Task.scala b/core/src/main/scala/spark/scheduler/Task.scala index 50768d43e02..0ab2ae6cfe9 100644 --- a/core/src/main/scala/spark/scheduler/Task.scala +++ b/core/src/main/scala/spark/scheduler/Task.scala @@ -30,9 +30,9 @@ import spark.executor.TaskMetrics */ private[spark] abstract class Task[T](val stageId: Int) extends Serializable { def run(attemptId: Long): T - def preferredLocations: Seq[String] = Nil + def preferredLocations: Seq[TaskLocation] = Nil - var generation: Long = -1 // Map output tracker generation. Will be set by TaskScheduler. + var epoch: Long = -1 // Map output tracker epoch. Will be set by TaskScheduler. var metrics: Option[TaskMetrics] = None diff --git a/core/src/main/scala/spark/scheduler/TaskLocation.scala b/core/src/main/scala/spark/scheduler/TaskLocation.scala new file mode 100644 index 00000000000..0e97c61188b --- /dev/null +++ b/core/src/main/scala/spark/scheduler/TaskLocation.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR 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 + +/** + * A location where a task should run. This can either be a host or a (host, executorID) pair. + * In the latter case, we will prefer to launch the task on that executorID, but our next level + * of preference will be executors on the same host if this is not possible. + */ +private[spark] +class TaskLocation private (val host: String, val executorId: Option[String]) extends Serializable + +private[spark] object TaskLocation { + def apply(host: String, executorId: String) = new TaskLocation(host, Some(executorId)) + + def apply(host: String) = new TaskLocation(host, None) +} diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala index 2cdeb1c8c0c..64be50b2d02 100644 --- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala +++ b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala @@ -35,7 +35,7 @@ private[spark] trait TaskSchedulerListener { taskInfo: TaskInfo, taskMetrics: TaskMetrics): Unit // A node was added to the cluster. - def executorGained(execId: String, hostPort: String): Unit + def executorGained(execId: String, host: String): Unit // A node was lost from the cluster. def executorLost(execId: String): Unit diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 96568e0d276..036e36bca07 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -37,18 +37,22 @@ import java.util.{TimerTask, Timer} */ private[spark] class ClusterScheduler(val sc: SparkContext) extends TaskScheduler - with Logging { - + with Logging +{ // How often to check for speculative tasks val SPECULATION_INTERVAL = System.getProperty("spark.speculation.interval", "100").toLong + // Threshold above which we warn user initial TaskSet may be starved val STARVATION_TIMEOUT = System.getProperty("spark.starvation.timeout", "15000").toLong + // How often to revive offers in case there are pending tasks - that is how often to try to get // tasks scheduled in case there are nodes available : default 0 is to disable it - to preserve existing behavior - // Note that this is required due to delayed scheduling due to data locality waits, etc. - // TODO: rename property ? - val TASK_REVIVAL_INTERVAL = System.getProperty("spark.tasks.revive.interval", "0").toLong + // Note that this is required due to delay scheduling due to data locality waits, etc. + // TODO(matei): move to StandaloneSchedulerBackend? + val TASK_REVIVAL_INTERVAL = System.getProperty("spark.scheduler.revival.interval", "1000").toLong + // TODO(matei): replace this with something that only affects levels past PROCESS_LOCAL; + // basically it can be a "cliff" for locality /* 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 @@ -71,7 +75,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) 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", "NODE_LOCAL")) + val TASK_SCHEDULING_AGGRESSION = TaskLocality.withName( + System.getProperty("spark.tasks.schedule.aggression", "NODE_LOCAL")) val activeTaskSets = new HashMap[String, TaskSetManager] @@ -89,16 +94,11 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // Which executor IDs we have executors on 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 process local and node local task locality. - private val hostPortsAlive = new HashSet[String] - private val hostToAliveHostPorts = new HashMap[String, HashSet[String]] - // 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 - private val executorsByHostPort = new HashMap[String, HashSet[String]] + private val executorsByHost = new HashMap[String, HashSet[String]] - private val executorIdToHostPort = new HashMap[String, String] + private val executorIdToHost = new HashMap[String, String] // JAR server, if any JARs were added by the user to the SparkContext var jarServer: HttpServer = null @@ -138,7 +138,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) schedulableBuilder.buildPools() // resolve executorId to hostPort mapping. def executorToHostPort(executorId: String, defaultHostPort: String): String = { - executorIdToHostPort.getOrElse(executorId, defaultHostPort) + executorIdToHost.getOrElse(executorId, defaultHostPort) } // Unfortunately, this means that SparkEnv is indirectly referencing ClusterScheduler @@ -146,13 +146,12 @@ private[spark] class ClusterScheduler(val sc: SparkContext) SparkEnv.get.executorIdToHostPort = Some(executorToHostPort) } - def newTaskId(): Long = nextTaskId.getAndIncrement() override def start() { backend.start() - if (JBoolean.getBoolean("spark.speculation")) { + if (System.getProperty("spark.speculation", "false").toBoolean) { new Thread("ClusterScheduler speculation check") { setDaemon(true) @@ -172,6 +171,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // Change to always run with some default if TASK_REVIVAL_INTERVAL <= 0 ? + // TODO(matei): remove this thread if (TASK_REVIVAL_INTERVAL > 0) { new Thread("ClusterScheduler task offer revival check") { setDaemon(true) @@ -201,7 +201,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) taskSetTaskIds(taskSet.id) = new HashSet[Long]() - if (hasReceivedTask == false) { + if (!hasReceivedTask) { starvationTimer.scheduleAtFixedRate(new TimerTask() { override def run() { if (!hasLaunchedTask) { @@ -214,7 +214,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } }, STARVATION_TIMEOUT, STARVATION_TIMEOUT) } - hasReceivedTask = true; + hasReceivedTask = true } backend.reviveOffers() } @@ -235,172 +235,53 @@ private[spark] class ClusterScheduler(val sc: SparkContext) * 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 { - SparkEnv.set(sc.env) - // Mark each slave as alive and remember its hostname - for (o <- offers) { - // DEBUG Code - Utils.checkHostPort(o.hostPort) - - executorIdToHostPort(o.executorId) = o.hostPort - if (! executorsByHostPort.contains(o.hostPort)) { - executorsByHostPort(o.hostPort) = new HashSet[String]() - } - - hostPortsAlive += o.hostPort - hostToAliveHostPorts.getOrElseUpdate(Utils.parseHostPort(o.hostPort)._1, new HashSet[String]).add(o.hostPort) - executorGained(o.executorId, o.hostPort) - } - // Build a list of tasks to assign to each slave - val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) - // merge availableCpus into nodeToAvailableCpus block ? - val availableCpus = offers.map(o => o.cores).toArray - val nodeToAvailableCpus = { - 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 + 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.host + if (!executorsByHost.contains(o.host)) { + executorsByHost(o.host) = new HashSet[String]() + executorGained(o.executorId, o.host) } - var launchedTask = false - val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() - - for (manager <- sortedTaskSetQueue) { - logDebug("parentName:%s, name:%s, runningTasks:%s".format( - manager.parent.name, manager.name, manager.runningTasks)) - } - - for (manager <- sortedTaskSetQueue) { + } - // 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]]() + // 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 + val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() + for (manager <- sortedTaskSetQueue) { + logDebug("parentName: %s, name: %s, runningTasks: %s".format( + manager.parent.name, manager.name, manager.runningTasks)) + } + var launchedTask = false + for (manager <- sortedTaskSetQueue; offer <- offers) { + do { + launchedTask = false for (i <- 0 until offers.size) { - val hostPort = offers(i).hostPort - // DEBUG code - Utils.checkHostPort(hostPort) - - 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 execId = offers(i).executorId + val host = offers(i).host + for (task <- manager.resourceOffer(execId, host, availableCpus(i))) { + 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 } - - val host = Utils.parseHostPort(hostPort)._1 - 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 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 (numNodeLocalTasks <= 0 && numRackLocalTasks <= 0){ - // add to others list - spread even this across cluster. - val list = otherOffers.getOrElseUpdate(host, new ArrayBuffer[Int]) - list += i - } - } - - val offersPriorityList = new ArrayBuffer[Int]( - processLocalOffers.size + nodeLocalOffers.size + rackLocalOffers.size + otherOffers.size) - - // First process local, then host local, then rack, then others - - // numNodeLocalOffers contains count of both process local and host offers. - val numNodeLocalOffers = { - val processLocalPriorityList = ClusterScheduler.prioritizeContainers(processLocalOffers) - offersPriorityList ++= processLocalPriorityList - - val nodeLocalPriorityList = ClusterScheduler.prioritizeContainers(nodeLocalOffers) - offersPriorityList ++= nodeLocalPriorityList - - processLocalPriorityList.size + nodeLocalPriorityList.size - } - val numRackLocalOffers = { - val rackLocalPriorityList = ClusterScheduler.prioritizeContainers(rackLocalOffers) - offersPriorityList ++= rackLocalPriorityList - rackLocalPriorityList.size } - offersPriorityList ++= ClusterScheduler.prioritizeContainers(otherOffers) - - var lastLoop = false - val lastLoopIndex = TASK_SCHEDULING_AGGRESSION match { - case TaskLocality.NODE_LOCAL => numNodeLocalOffers - case TaskLocality.RACK_LOCAL => numRackLocalOffers + numNodeLocalOffers - case TaskLocality.ANY => offersPriorityList.size - } - - do { - launchedTask = false - var loopCount = 0 - for (i <- offersPriorityList) { - val execId = offers(i).executorId - val hostPort = offers(i).hostPort - - // If last loop and within the lastLoopIndex, expand scope - else use null (which will use default/existing) - val overrideLocality = if (lastLoop && loopCount < lastLoopIndex) TASK_SCHEDULING_AGGRESSION else null - - // If last loop, override waiting for host locality - we scheduled all local tasks already and there might be more available ... - loopCount += 1 - - manager.slaveOffer(execId, hostPort, availableCpus(i), overrideLocality) 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 - executorsByHostPort(hostPort) += execId - availableCpus(i) -= 1 - launchedTask = true - - case None => {} - } - } - // Loop once more - when lastLoop = true, then we try to schedule task on all nodes irrespective of - // data locality (we still go in order of priority : but that would not change anything since - // if data local tasks had been available, we would have scheduled them already) - if (lastLoop) { - // prevent more looping - launchedTask = false - } else if (!lastLoop && !launchedTask) { - // 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 - lastLoop = true - } - } - } while (launchedTask) - } + } while (launchedTask) + } - if (tasks.size > 0) { - hasLaunchedTask = true - } - return tasks + if (tasks.size > 0) { + hasLaunchedTask = true } + return tasks } def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { @@ -514,7 +395,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) synchronized { if (activeExecutorIds.contains(executorId)) { - val hostPort = executorIdToHostPort(executorId) + val hostPort = executorIdToHost(executorId) logError("Lost executor %s on %s: %s".format(executorId, hostPort, reason)) removeExecutor(executorId) failedExecutor = Some(executorId) @@ -535,52 +416,37 @@ private[spark] class ClusterScheduler(val sc: SparkContext) /** Remove an executor from all our data structures and mark it as lost */ private def removeExecutor(executorId: String) { + // TODO(matei): remove HostPort activeExecutorIds -= executorId - val hostPort = executorIdToHostPort(executorId) - if (hostPortsAlive.contains(hostPort)) { - // DEBUG Code - Utils.checkHostPort(hostPort) + val host = executorIdToHost(executorId) - hostPortsAlive -= hostPort - hostToAliveHostPorts.getOrElseUpdate(Utils.parseHostPort(hostPort)._1, new HashSet[String]).remove(hostPort) - } - - val execs = executorsByHostPort.getOrElse(hostPort, new HashSet) + val execs = executorsByHost.getOrElse(host, new HashSet) execs -= executorId if (execs.isEmpty) { - executorsByHostPort -= hostPort + executorsByHost -= host } - executorIdToHostPort -= executorId - rootPool.executorLost(executorId, hostPort) + executorIdToHost -= executorId + rootPool.executorLost(executorId, host) } - def executorGained(execId: String, hostPort: String) { - listener.executorGained(execId, hostPort) + def executorGained(execId: String, host: String) { + listener.executorGained(execId, host) } - def getExecutorsAliveOnHost(host: String): Option[Set[String]] = { - Utils.checkHost(host) - - val retval = hostToAliveHostPorts.get(host) - if (retval.isDefined) { - return Some(retval.get.toSet) - } + def getExecutorsAliveOnHost(host: String): Option[Set[String]] = synchronized { + executorsByHost.get(host).map(_.toSet) + } - None + def hasExecutorsAliveOnHost(host: String): Boolean = synchronized { + executorsByHost.contains(host) } - 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) + def isExecutorAlive(execId: String): Boolean = synchronized { + activeExecutorIds.contains(execId) } // By default, rack is unknown def getRackForHost(value: String): Option[String] = None - - // By default, (cached) hosts for rack is unknown - def getCachedHostsForRack(rack: String): Option[Set[String]] = None } @@ -610,6 +476,7 @@ object ClusterScheduler { // order keyList based on population of value in map val keyList = _keyList.sortWith( + // TODO(matei): not sure why we're using getOrElse if keyList = map.keys... see if it matters (left, right) => map.get(left).getOrElse(Set()).size > map.get(right).getOrElse(Set()).size ) @@ -617,7 +484,7 @@ object ClusterScheduler { var index = 0 var found = true - while (found){ + while (found) { found = false for (key <- keyList) { val containerList: ArrayBuffer[T] = map.get(key).getOrElse(null) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 7f855cd345b..1947c516dbc 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -29,49 +29,13 @@ import scala.math.min import spark.{FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskState, Utils} import spark.{ExceptionFailure, SparkException, TaskResultTooBigFailure} import spark.TaskState.TaskState -import spark.scheduler.{ShuffleMapTask, Task, TaskResult, TaskSet} +import spark.scheduler._ +import scala.Some +import spark.FetchFailed +import spark.ExceptionFailure +import spark.TaskResultTooBigFailure -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. */ @@ -113,28 +77,26 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: // 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 + // Set of pending tasks for each executor. 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]] + private val pendingTasksForExecutor = new HashMap[String, ArrayBuffer[Int]] - // List of pending tasks for each node. - // Essentially, similar to pendingTasksForHostPort, except at host level + // Set of pending tasks for each host. Similar to pendingTasksForExecutor, + // but 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]] + // Set of pending tasks for each rack -- similar to the above. + private val pendingTasksForRack = new HashMap[String, ArrayBuffer[Int]] - // List containing pending tasks with no locality preferences + // Set containing pending tasks with no locality preferences. val pendingTasksWithNoPrefs = new ArrayBuffer[Int] - // List containing all pending tasks (also used as a stack, as above) + // Set 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 @@ -144,13 +106,14 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: // Task index, start and finish time for each task attempt (indexed by task ID) val taskInfos = new HashMap[Long, TaskInfo] - // Did the job fail? + // Did the TaskSet 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 @@ -158,11 +121,11 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: // 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) + // Figure out the current map output tracker epoch and set it on all tasks + val epoch = sched.mapOutputTracker.getEpoch + logDebug("Epoch for " + taskSet.id + ": " + epoch) for (t <- tasks) { - t.generation = generation + t.epoch = epoch } // Add all our tasks to the pending lists. We do this in reverse order @@ -171,166 +134,74 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: 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 - } - } + /** + * Add a task to all the pending-task lists that it should be on. If readding is set, we are + * re-adding the task so only include it in each list if it's not already there. + */ + private def addPendingTask(index: Int, readding: Boolean = false) { + // Utility method that adds `index` to a list only if readding=false or it's not already there + def addTo(list: ArrayBuffer[Int]) { + if (!readding || !list.contains(index)) { + list += index } - - 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 + var hadAliveLocations = false + for (loc <- tasks(index).preferredLocations) { + for (execId <- loc.executorId) { + if (sched.isExecutorAlive(execId)) { + addTo(pendingTasksForExecutor.getOrElseUpdate(execId, new ArrayBuffer)) + hadAliveLocations = true + } } - } - - val retval = new HashSet[String] - scheduler.synchronized { - for (prefLocation <- taskPreferredLocations) { - val aliveLocationsOpt = scheduler.getExecutorsAliveOnHost(Utils.parseHostPort(prefLocation)._1) - if (aliveLocationsOpt.isDefined) { - retval ++= aliveLocationsOpt.get + if (sched.hasExecutorsAliveOnHost(loc.host)) { + addTo(pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer)) + for (rack <- sched.getRackForHost(loc.host)) { + addTo(pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer)) } + hadAliveLocations = true } } - 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 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. - 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 - } + if (!hadAliveLocations) { + // Even though the task might've had preferred locations, all of those hosts or executors + // are dead; put it in the no-prefs list so we can schedule it elsewhere right away. + addTo(pendingTasksWithNoPrefs) } - allPendingTasks += index + addTo(allPendingTasks) } - // 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 executor ID, or an empty list if + * there is no map entry for that host + */ + private def getPendingTasksForExecutor(executorId: String): ArrayBuffer[Int] = { + pendingTasksForExecutor.getOrElse(executorId, 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 + /** + * 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(host: String): ArrayBuffer[Int] = { 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) - 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) - 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 - override def numRackLocalPendingTasksForHost(hostPort: String): Int = { - getRackLocalPendingTasksForHost(hostPort).count { index => - copiesRunning(index) == 0 && !finished(index) - } + /** + * Return the pending rack-local task list for a given rack, or an empty list if + * there is no map entry for that rack + */ + private def getPendingTasksForRack(rack: String): ArrayBuffer[Int] = { + pendingTasksForRack.getOrElse(rack, ArrayBuffer()) } - - // 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. + /** + * 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 @@ -342,176 +213,145 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: 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] = { + /** Check whether a task is currently running an attempt on a given host */ + private def hasAttemptOnHost(taskIndex: Int, host: String): Boolean = { + !taskAttempts(taskIndex).exists(_.host == host) + } - assert (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) + /** + * Return a speculative task for a given executor if any are available. The task should not have + * an attempt running on this host, in case the host is slow. In addition, the task should meet + * the given locality constraint. + */ + private def findSpeculativeTask(execId: String, host: String, locality: TaskLocality.Value) + : Option[(Int, TaskLocality.Value)] = + { 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 (!speculatableTasks.isEmpty) { + // Check for process-local or preference-less tasks; note that tasks can be process-local + // on multiple nodes when we replicate cached blocks, as in Spark Streaming + for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) { + val prefs = tasks(index).preferredLocations + val executors = prefs.flatMap(_.executorId) + if (prefs.size == 0 || executors.contains(execId)) { + speculatableTasks -= index + return Some((index, TaskLocality.PROCESS_LOCAL)) + } } - if (localTask != None) { - speculatableTasks -= localTask.get - return localTask + // Check for node-local tasks + if (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) { + for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) { + val locations = tasks(index).preferredLocations.map(_.host) + if (locations.contains(host)) { + speculatableTasks -= index + return Some((index, TaskLocality.NODE_LOCAL)) + } + } } - // check for rack locality + // Check for rack-local tasks 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 + for (rack <- sched.getRackForHost(host)) { + for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) { + val racks = tasks(index).preferredLocations.map(_.host).map(sched.getRackForHost) + if (racks.contains(rack)) { + speculatableTasks -= index + return Some((index, TaskLocality.RACK_LOCAL)) + } + } } } - // Any task ... + // Check for non-local tasks 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 + for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) { + speculatableTasks -= index + return Some((index, TaskLocality.ANY)) } } } + 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 + /** + * Dequeue a pending task for a given node and return its index and locality level. + * Only search for tasks matching the given locality constraint. + */ + private def findTask(execId: String, host: String, locality: TaskLocality.Value) + : Option[(Int, TaskLocality.Value)] = + { + for (index <- findTaskFromList(getPendingTasksForExecutor(execId))) { + return Some((index, TaskLocality.PROCESS_LOCAL)) } - val localTask = findTaskFromList(getPendingTasksForHost(hostPort)) - if (localTask != None) { - return localTask + if (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) { + for (index <- findTaskFromList(getPendingTasksForHost(host))) { + return Some((index, TaskLocality.NODE_LOCAL)) + } } if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { - val rackLocalTask = findTaskFromList(getRackLocalPendingTasksForHost(hostPort)) - if (rackLocalTask != None) { - return rackLocalTask + for { + rack <- sched.getRackForHost(host) + index <- findTaskFromList(getPendingTasksForRack(rack)) + } { + return Some((index, TaskLocality.RACK_LOCAL)) } } - // 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 + // Look for no-pref tasks after rack-local tasks since they can run anywhere. + for (index <- findTaskFromList(pendingTasksWithNoPrefs)) { + return Some((index, TaskLocality.PROCESS_LOCAL)) } if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { - val nonLocalTask = findTaskFromList(allPendingTasks) - if (nonLocalTask != None) { - return nonLocalTask + for (index <- findTaskFromList(allPendingTasks)) { + return Some((index, TaskLocality.ANY)) } } // 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) + return findSpeculativeTask(execId, host, locality) } - // Respond to an offer of a single slave from the scheduler by finding a task - override def slaveOffer( - execId: String, - hostPort: String, - availableCpus: Double, - overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = + /** + * Respond to an offer of a single slave from the scheduler by finding a task + */ + override def resourceOffer(execId: String, host: String, availableCpus: Double) + : Option[TaskDescription] = { if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { + val curTime = System.currentTimeMillis + // If explicitly specified, use that - val locality = if (overrideLocality != null) overrideLocality else { + val locality = { // 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) { + // TODO(matei): Multi-level delay scheduling + if (curTime - 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 + findTask(execId, host, locality) match { + case Some((index, taskLocality)) => { + // Found a task; do some bookkeeping and return a task description 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)) + taskSet.id, index, taskId, execId, host, taskLocality)) // Do various bookkeeping copiesRunning(index) += 1 - val time = System.currentTimeMillis - val info = new TaskInfo(taskId, index, time, execId, hostPort, taskLocality) + val info = new TaskInfo(taskId, index, curTime, execId, host, taskLocality) taskInfos(taskId) = info taskAttempts(index) = info :: taskAttempts(index) if (taskLocality == TaskLocality.PROCESS_LOCAL || taskLocality == TaskLocality.NODE_LOCAL) { - lastPreferredLaunchTime = time + lastPreferredLaunchTime = curTime } // Serialize and return the task val startTime = System.currentTimeMillis @@ -534,6 +374,7 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: return None } + /** Called by cluster scheduler when one of our tasks changes state */ override def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { SparkEnv.set(env) state match { @@ -566,7 +407,7 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: if (!finished(index)) { tasksFinished += 1 logInfo("Finished TID %s in %d ms on %s (progress: %d/%d)".format( - tid, info.duration, info.hostPort, tasksFinished, numTasks)) + tid, info.duration, info.host, tasksFinished, numTasks)) // Deserialize task result and pass it to the scheduler try { val result = ser.deserialize[TaskResult[_]](serializedData) @@ -698,44 +539,33 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: } } - // TODO(xiajunluan): 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 addSchedulable(schedulable: Schedulable) {} - override def removeSchedulable(schedulable:Schedulable) { - // nothing - } + override def removeSchedulable(schedulable: Schedulable) {} override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = { - var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager] + var sortedTaskSetQueue = ArrayBuffer[TaskSetManager](this) sortedTaskSetQueue += this return sortedTaskSetQueue } - override def executorLost(execId: String, hostPort: String) { + /** Called by cluster scheduler when an executor is lost so we can re-enqueue our tasks */ + override def executorLost(execId: String, host: 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.NODE_LOCAL) - if (newLocs.isEmpty) { - pendingTasksWithNoPrefs += index - } + // Re-enqueue pending tasks for this host based on the status of the cluster -- for example, a + // task that used to have locations on only this host might now go to the no-prefs list. Note + // that it's okay if we add a task to the same queue twice (if it had multiple preferred + // locations), because findTaskFromList will skip already-running tasks. + for (index <- getPendingTasksForExecutor(execId)) { + addPendingTask(index, readding=true) + } + for (index <- getPendingTasksForHost(host)) { + addPendingTask(index, readding=true) } // Re-enqueue any tasks that ran on the failed executor if this is a shuffle map stage @@ -789,7 +619,7 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: !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)) + taskSet.id, index, info.host, threshold)) speculatableTasks += index foundTasks = true } diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 075a7cbf7e2..3b49af12585 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -37,7 +37,9 @@ import spark.scheduler.cluster.StandaloneClusterMessages._ */ private[spark] class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: ActorSystem) - extends SchedulerBackend with Logging { + extends SchedulerBackend with Logging +{ + // TODO(matei): periodically revive offers as in MesosScheduler // Use an atomic variable to track total number of cores in the cluster for simplicity and speed var totalCoreCount = new AtomicInteger(0) @@ -45,7 +47,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor { private val executorActor = new HashMap[String, ActorRef] private val executorAddress = new HashMap[String, Address] - private val executorHostPort = new HashMap[String, String] + private val executorHost = new HashMap[String, String] private val freeCores = new HashMap[String, Int] private val actorToExecutorId = new HashMap[ActorRef, String] private val addressToExecutorId = new HashMap[Address, String] @@ -65,7 +67,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor sender ! RegisteredExecutor(sparkProperties) context.watch(sender) executorActor(executorId) = sender - executorHostPort(executorId) = hostPort + executorHost(executorId) = Utils.parseHostPort(hostPort)._1 freeCores(executorId) = cores executorAddress(executorId) = sender.path.address actorToExecutorId(sender) = executorId @@ -105,13 +107,13 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor // Make fake resource offers on all executors def makeOffers() { launchTasks(scheduler.resourceOffers( - executorHostPort.toArray.map {case (id, hostPort) => new WorkerOffer(id, hostPort, freeCores(id))})) + executorHost.toArray.map {case (id, host) => new WorkerOffer(id, host, freeCores(id))})) } // Make fake resource offers on just one executor def makeOffers(executorId: String) { launchTasks(scheduler.resourceOffers( - Seq(new WorkerOffer(executorId, executorHostPort(executorId), freeCores(executorId))))) + Seq(new WorkerOffer(executorId, executorHost(executorId), freeCores(executorId))))) } // Launch tasks returned by a set of resource offers @@ -130,9 +132,8 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor actorToExecutorId -= executorActor(executorId) addressToExecutorId -= executorAddress(executorId) executorActor -= executorId - executorHostPort -= executorId + executorHost -= executorId freeCores -= executorId - executorHostPort -= executorId totalCoreCount.addAndGet(-numCores) scheduler.executorLost(executorId, SlaveLost(reason)) } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala index c693b722ac3..c2c5522686f 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala @@ -28,11 +28,9 @@ class TaskInfo( val index: Int, val launchTime: Long, val executorId: String, - val hostPort: String, + val host: String, val taskLocality: TaskLocality.TaskLocality) { - Utils.checkHostPort(hostPort, "Expected hostport") - var finishTime: Long = 0 var failed = false diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskLocality.scala b/core/src/main/scala/spark/scheduler/cluster/TaskLocality.scala new file mode 100644 index 00000000000..1c33e41f87e --- /dev/null +++ b/core/src/main/scala/spark/scheduler/cluster/TaskLocality.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR 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 + + +private[spark] object TaskLocality + extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY") +{ + // 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 = { + condition <= constraint + } +} diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 1a92a5ed6fa..277654edc07 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -29,17 +29,8 @@ 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 resourceOffer(execId: String, hostPort: String, availableCpus: Double) + : Option[TaskDescription] def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) diff --git a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala index 06d1203f70b..1d09bd9b034 100644 --- a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala +++ b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala @@ -21,5 +21,4 @@ package spark.scheduler.cluster * Represents free resources available on an executor. */ private[spark] -class WorkerOffer(val executorId: String, val hostPort: String, val cores: Int) { -} +class WorkerOffer(val executorId: String, val host: String, val cores: Int) diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index 6c43928bc85..a4f5f46777a 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -141,7 +141,8 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: for (manager <- sortedTaskSetQueue) { do { launchTask = false - manager.slaveOffer(null, null, freeCpuCores) match { + // TODO(matei): don't pass null here? + manager.resourceOffer(null, null, freeCpuCores) match { case Some(task) => tasks += task taskIdToTaskSetId(task.taskId) = manager.taskSet.id diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index c38eeb9e11e..698c777bec6 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -98,14 +98,11 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas return None } - override def slaveOffer( - execId: String, - hostPort: String, - availableCpus: Double, - overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = + override def resourceOffer(execId: String, host: String, availableCpus: Double) + : Option[TaskDescription] = { SparkEnv.set(sched.env) - logDebug("availableCpus:%d,numFinished:%d,numTasks:%d".format( + logDebug("availableCpus:%d, numFinished:%d, numTasks:%d".format( availableCpus.toInt, numFinished, numTasks)) if (availableCpus > 0 && numFinished < numTasks) { findTask() match { @@ -131,18 +128,6 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas return None } - override def numPendingTasksForHostPort(hostPort: String): Int = { - return 0 - } - - override def numRackLocalPendingTasksForHost(hostPort :String): Int = { - return 0 - } - - override def numPendingTasksForHost(hostPort: String): Int = { - return 0 - } - override def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { SparkEnv.set(env) state match { diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala index 3a72474419a..2a6ec2a55dd 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/spark/storage/BlockManager.scala @@ -1004,43 +1004,43 @@ private[spark] object BlockManager extends Logging { } } - def blockIdsToExecutorLocations(blockIds: Array[String], env: SparkEnv, blockManagerMaster: BlockManagerMaster = null): HashMap[String, List[String]] = { + def blockIdsToBlockManagers( + blockIds: Array[String], + env: SparkEnv, + blockManagerMaster: BlockManagerMaster = null) + : Map[String, Seq[BlockManagerId]] = + { // env == null and blockManagerMaster != null is used in tests assert (env != null || blockManagerMaster != null) - val locationBlockIds: Seq[Seq[BlockManagerId]] = - if (env != null) { - env.blockManager.getLocationBlockIds(blockIds) - } else { - blockManagerMaster.getLocations(blockIds) - } + val blockLocations: Seq[Seq[BlockManagerId]] = if (env != null) { + env.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]]() + val blockManagers = new HashMap[String, Seq[BlockManagerId]] for (i <- 0 until blockIds.length) { - val blockId = blockIds(i) - val blockLocations = locationBlockIds(i) - - val executors = new HashSet[String]() - - 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) + blockManagers(blockIds(i)) = blockLocations(i) } + blockManagers.toMap + } - executorLocations + def blockIdsToExecutorIds( + blockIds: Array[String], + env: SparkEnv, + blockManagerMaster: BlockManagerMaster = null) + : Map[String, Seq[String]] = + { + blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.executorId)) } + def blockIdsToHosts( + blockIds: Array[String], + env: SparkEnv, + blockManagerMaster: BlockManagerMaster = null) + : Map[String, Seq[String]] = + { + blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.host)) + } } diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 797513f2663..6948ea4dd90 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -156,7 +156,7 @@ private[spark] class StagePage(parent: JobProgressUI) {

    - + + + + + + + + + + + + + + + + From 4004cf775d9397efbb5945768aaf05ba682c715c Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 16 Aug 2013 14:01:12 -0700 Subject: [PATCH 0777/2521] Added some comments on threading in scheduler code --- .../scala/spark/scheduler/DAGScheduler.scala | 20 +++++++++++++++---- .../scheduler/cluster/ClusterScheduler.scala | 12 ++++++++++- .../cluster/ClusterTaskSetManager.scala | 9 ++++++++- 3 files changed, 35 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 2f7e6d98f84..35b31f45a7c 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -32,10 +32,22 @@ import spark.storage.{BlockManager, BlockManagerMaster} import spark.util.{MetadataCleaner, TimeStampedHashMap} /** - * 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 - * schedule to run the job. Subclasses only need to implement the code to send a task to the cluster - * and to report fetch failures (the submitTasks method, and code to add CompletionEvents). + * The high-level scheduling layer 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 finds a + * minimal schedule to run the job. It then submits stages as TaskSets to an underlying + * TaskScheduler implementation that runs them on the cluster. + * + * In addition to coming up with a DAG of stages, this class also determines the preferred + * locations to run each task on, based on the current cache status, and passes these to the + * low-level TaskScheduler. Furthermore, it handles failures due to shuffle output files being + * lost, in which case old stages may need to be resubmitted. Failures *within* a stage that are + * not caused by shuffie file loss are handled by the TaskScheduler, which will retry each task + * a small number of times before cancelling the whole stage. + * + * THREADING: This class runs all its logic in a single thread executing the run() method, to which + * events are submitted using a synchonized queue (eventQueue). The public API methods, such as + * runJob, taskEnded and executorLost, post events asynchronously to this queue. All other methods + * should be private. */ private[spark] class DAGScheduler( diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index e88edc5b2ac..679d899b472 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -33,7 +33,17 @@ import java.util.{TimerTask, Timer} /** * The main TaskScheduler implementation, for running tasks on a cluster. Clients should first call - * start(), then submit task sets through the runTasks method. + * initialize() and start(), then submit task sets through the runTasks method. + * + * This class can work with multiple types of clusters by acting through a SchedulerBackend. + * It handles common logic, like determining a scheduling order across jobs, waking up to launch + * speculative tasks, etc. + * + * THREADING: SchedulerBackends and task-submitting clients can call this class from multiple + * threads, so it needs locks in public API methods to maintain its state. In addition, some + * SchedulerBackends sycnchronize on themselves when they want to send events here, and then + * acquire a lock on us, so we need to make sure that we don't try to lock the backend while + * we are holding a lock on ourselves. */ private[spark] class ClusterScheduler(val sc: SparkContext) extends TaskScheduler diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 91de25254c0..1d57732f5dd 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -37,7 +37,14 @@ import spark.TaskResultTooBigFailure /** - * Schedules the tasks within a single TaskSet in the ClusterScheduler. + * Schedules the tasks within a single TaskSet in the ClusterScheduler. This class keeps track of + * the status of each task, retries tasks if they fail (up to a limited number of times), and + * handles locality-aware scheduling for this TaskSet via delay scheduling. The main interfaces + * to it are resourceOffer, which asks the TaskSet whether it wants to run a task on one node, + * and statusUpdate, which tells it that one of its tasks changed state (e.g. finished). + * + * THREADING: This class is designed to only be called from code with a lock on the + * ClusterScheduler (e.g. its event handlers). It should not be called from other threads. */ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: TaskSet) extends TaskSetManager with Logging { From 8ac3d1e2636ec71ab9a14bed68f138e3a365603e Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 18 Aug 2013 19:36:34 -0700 Subject: [PATCH 0778/2521] Added unit tests for ClusterTaskSetManager, and fix a bug found with resetting locality level after a non-local launch --- .../scala/spark/scheduler/TaskLocation.scala | 4 +- .../scala/spark/scheduler/TaskResult.scala | 4 +- .../cluster/ClusterTaskSetManager.scala | 27 +- .../scheduler/cluster/TaskDescription.scala | 3 + .../scheduler/cluster/TaskSetManager.scala | 9 + .../scheduler/local/LocalTaskSetManager.scala | 2 +- core/src/main/scala/spark/util/Clock.scala | 29 ++ .../cluster/ClusterSchedulerSuite.scala | 21 +- .../cluster/ClusterTaskSetManagerSuite.scala | 273 ++++++++++++++++++ .../spark/scheduler/cluster/FakeTask.scala | 26 ++ .../src/test/scala/spark/util/FakeClock.scala | 26 ++ 11 files changed, 396 insertions(+), 28 deletions(-) create mode 100644 core/src/main/scala/spark/util/Clock.scala create mode 100644 core/src/test/scala/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala create mode 100644 core/src/test/scala/spark/scheduler/cluster/FakeTask.scala create mode 100644 core/src/test/scala/spark/util/FakeClock.scala diff --git a/core/src/main/scala/spark/scheduler/TaskLocation.scala b/core/src/main/scala/spark/scheduler/TaskLocation.scala index 0e97c61188b..fea117e956d 100644 --- a/core/src/main/scala/spark/scheduler/TaskLocation.scala +++ b/core/src/main/scala/spark/scheduler/TaskLocation.scala @@ -23,7 +23,9 @@ package spark.scheduler * of preference will be executors on the same host if this is not possible. */ private[spark] -class TaskLocation private (val host: String, val executorId: Option[String]) extends Serializable +class TaskLocation private (val host: String, val executorId: Option[String]) extends Serializable { + override def toString: String = "TaskLocation(" + host + ", " + executorId + ")" +} private[spark] object TaskLocation { def apply(host: String, executorId: String) = new TaskLocation(host, Some(executorId)) diff --git a/core/src/main/scala/spark/scheduler/TaskResult.scala b/core/src/main/scala/spark/scheduler/TaskResult.scala index 89793e0e828..fc4856756b1 100644 --- a/core/src/main/scala/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/spark/scheduler/TaskResult.scala @@ -28,7 +28,9 @@ import java.nio.ByteBuffer // TODO: Use of distributed cache to return result is a hack to get around // what seems to be a bug with messages over 60KB in libprocess; fix it private[spark] -class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var metrics: TaskMetrics) extends Externalizable { +class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var metrics: TaskMetrics) + extends Externalizable +{ def this() = this(null.asInstanceOf[T], null, null) override def writeExternal(out: ObjectOutput) { diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 1d57732f5dd..a4d6880abb5 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -34,6 +34,7 @@ import scala.Some import spark.FetchFailed import spark.ExceptionFailure import spark.TaskResultTooBigFailure +import spark.util.{SystemClock, Clock} /** @@ -46,9 +47,13 @@ import spark.TaskResultTooBigFailure * THREADING: This class is designed to only be called from code with a lock on the * ClusterScheduler (e.g. its event handlers). It should not be called from other threads. */ -private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: TaskSet) - extends TaskSetManager with Logging { - +private[spark] class ClusterTaskSetManager( + sched: ClusterScheduler, + val taskSet: TaskSet, + clock: Clock = SystemClock) + extends TaskSetManager + with Logging +{ // CPUs to request per task val CPUS_PER_TASK = System.getProperty("spark.task.cpus", "1").toInt @@ -142,7 +147,7 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: // last launched a task at that level, and move up a level when localityWaits[curLevel] expires. // We then move down if we manage to launch a "more local" task. var currentLocalityIndex = 0 // Index of our current locality level in validLocalityLevels - var lastLaunchTime = System.currentTimeMillis() // Time we last launched a task at this level + var lastLaunchTime = clock.getTime() // Time we last launched a task at this level /** * Add a task to all the pending-task lists that it should be on. If readding is set, we are @@ -340,7 +345,7 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: : Option[TaskDescription] = { if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { - val curTime = System.currentTimeMillis() + val curTime = clock.getTime() var allowedLocality = getAllowedLocalityLevel(curTime) if (allowedLocality > maxLocality) { @@ -361,22 +366,22 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: taskInfos(taskId) = info taskAttempts(index) = info :: taskAttempts(index) // Update our locality level for delay scheduling - currentLocalityIndex = getLocalityIndex(allowedLocality) + currentLocalityIndex = getLocalityIndex(taskLocality) lastLaunchTime = curTime // Serialize and return the task - val startTime = System.currentTimeMillis() + val startTime = clock.getTime() // 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 + val timeTaken = clock.getTime() - 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) if (taskAttempts(index).size == 1) taskStarted(task,info) - return Some(new TaskDescription(taskId, execId, taskName, serializedTask)) + return Some(new TaskDescription(taskId, execId, taskName, index, serializedTask)) } case _ => } @@ -505,7 +510,7 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: case ef: ExceptionFailure => sched.listener.taskEnded(tasks(index), ef, null, null, info, ef.metrics.getOrElse(null)) val key = ef.description - val now = System.currentTimeMillis() + val now = clock.getTime() val (printFull, dupCount) = { if (recentExceptions.contains(key)) { val (dupCount, printTime) = recentExceptions(key) @@ -643,7 +648,7 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: val minFinishedForSpeculation = (SPECULATION_QUANTILE * numTasks).floor.toInt logDebug("Checking for speculative tasks: minFinished = " + minFinishedForSpeculation) if (tasksFinished >= minFinishedForSpeculation) { - val time = System.currentTimeMillis() + val time = clock.getTime() val durations = taskInfos.values.filter(_.successful).map(_.duration).toArray Arrays.sort(durations) val medianDuration = durations(min((0.5 * numTasks).round.toInt, durations.size - 1)) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala index 761fdf69198..187553233fb 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala @@ -24,6 +24,7 @@ private[spark] class TaskDescription( val taskId: Long, val executorId: String, val name: String, + val index: Int, // Index within this task's TaskSet _serializedTask: ByteBuffer) extends Serializable { @@ -31,4 +32,6 @@ private[spark] class TaskDescription( private val buffer = new SerializableBuffer(_serializedTask) def serializedTask: ByteBuffer = buffer.value + + override def toString: String = "TaskDescription(TID=%d, index=%d)".format(taskId, index) } diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 5ab6ab9aadb..0248830b7a2 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -22,6 +22,15 @@ import java.nio.ByteBuffer import spark.TaskState.TaskState import spark.scheduler.TaskSet +/** + * Tracks and schedules the tasks within a single TaskSet. This class keeps track of the status of + * each task and is responsible for retries on failure and locality. The main interfaces to it + * are resourceOffer, which asks the TaskSet whether it wants to run a task on one node, and + * statusUpdate, which tells it that one of its tasks changed state (e.g. finished). + * + * THREADING: This class is designed to only be called from code with a lock on the TaskScheduler + * (e.g. its event handlers). It should not be called from other threads. + */ private[spark] trait TaskSetManager extends Schedulable { def schedulableQueue = null diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index 3ef636ff075..e237f289e36 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -125,7 +125,7 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas copiesRunning(index) += 1 increaseRunningTasks(1) taskStarted(task, info) - return Some(new TaskDescription(taskId, null, taskName, bytes)) + return Some(new TaskDescription(taskId, null, taskName, index, bytes)) case None => {} } } diff --git a/core/src/main/scala/spark/util/Clock.scala b/core/src/main/scala/spark/util/Clock.scala new file mode 100644 index 00000000000..aa71a5b4424 --- /dev/null +++ b/core/src/main/scala/spark/util/Clock.scala @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.util + +/** + * An interface to represent clocks, so that they can be mocked out in unit tests. + */ +private[spark] trait Clock { + def getTime(): Long +} + +private[spark] object SystemClock extends Clock { + def getTime(): Long = System.currentTimeMillis() +} diff --git a/core/src/test/scala/spark/scheduler/cluster/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/cluster/ClusterSchedulerSuite.scala index aeeed147864..abfdabf5fe6 100644 --- a/core/src/test/scala/spark/scheduler/cluster/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/cluster/ClusterSchedulerSuite.scala @@ -27,7 +27,7 @@ import scala.collection.mutable.ArrayBuffer import java.util.Properties -class DummyTaskSetManager( +class FakeTaskSetManager( initPriority: Int, initStageId: Int, initNumTasks: Int, @@ -81,7 +81,7 @@ class DummyTaskSetManager( { if (tasksFinished + runningTasks < numTasks) { increaseRunningTasks(1) - return Some(new TaskDescription(0, execId, "task 0:0", null)) + return Some(new TaskDescription(0, execId, "task 0:0", 0, null)) } return None } @@ -104,17 +104,10 @@ class DummyTaskSetManager( } } -class DummyTask(stageId: Int) extends Task[Int](stageId) -{ - def run(attemptId: Long): Int = { - return 0 - } -} - 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) + def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: ClusterScheduler, taskSet: TaskSet): FakeTaskSetManager = { + new FakeTaskSetManager(priority, stage, numTasks, cs , taskSet) } def resourceOffer(rootPool: Pool): Int = { @@ -141,7 +134,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging sc = new SparkContext("local", "ClusterSchedulerSuite") val clusterScheduler = new ClusterScheduler(sc) var tasks = ArrayBuffer[Task[_]]() - val task = new DummyTask(0) + val task = new FakeTask(0) tasks += task val taskSet = new TaskSet(tasks.toArray,0,0,0,null) @@ -168,7 +161,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging sc = new SparkContext("local", "ClusterSchedulerSuite") val clusterScheduler = new ClusterScheduler(sc) var tasks = ArrayBuffer[Task[_]]() - val task = new DummyTask(0) + val task = new FakeTask(0) tasks += task val taskSet = new TaskSet(tasks.toArray,0,0,0,null) @@ -225,7 +218,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging sc = new SparkContext("local", "ClusterSchedulerSuite") val clusterScheduler = new ClusterScheduler(sc) var tasks = ArrayBuffer[Task[_]]() - val task = new DummyTask(0) + val task = new FakeTask(0) tasks += task val taskSet = new TaskSet(tasks.toArray,0,0,0,null) diff --git a/core/src/test/scala/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala b/core/src/test/scala/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala new file mode 100644 index 00000000000..5a0b949ef5f --- /dev/null +++ b/core/src/test/scala/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala @@ -0,0 +1,273 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR 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 +import scala.collection.mutable + +import org.scalatest.FunSuite + +import spark._ +import spark.scheduler._ +import spark.executor.TaskMetrics +import java.nio.ByteBuffer +import spark.util.FakeClock + +/** + * A mock ClusterScheduler implementation that just remembers information about tasks started and + * feedback received from the TaskSetManagers. Note that it's important to initialize this with + * a list of "live" executors and their hostnames for isExecutorAlive and hasExecutorsAliveOnHost + * to work, and these are required for locality in ClusterTaskSetManager. + */ +class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /* execId, host */) + extends ClusterScheduler(sc) +{ + val startedTasks = new ArrayBuffer[Long] + val endedTasks = new mutable.HashMap[Long, TaskEndReason] + val finishedManagers = new ArrayBuffer[TaskSetManager] + + val executors = new mutable.HashMap[String, String] ++ liveExecutors + + listener = new TaskSchedulerListener { + def taskStarted(task: Task[_], taskInfo: TaskInfo) { + startedTasks += taskInfo.index + } + + def taskEnded( + task: Task[_], + reason: TaskEndReason, + result: Any, + accumUpdates: mutable.Map[Long, Any], + taskInfo: TaskInfo, + taskMetrics: TaskMetrics) + { + endedTasks(taskInfo.index) = reason + } + + def executorGained(execId: String, host: String) {} + + def executorLost(execId: String) {} + + def taskSetFailed(taskSet: TaskSet, reason: String) {} + } + + def removeExecutor(execId: String): Unit = executors -= execId + + override def taskSetFinished(manager: TaskSetManager): Unit = finishedManagers += manager + + override def isExecutorAlive(execId: String): Boolean = executors.contains(execId) + + override def hasExecutorsAliveOnHost(host: String): Boolean = executors.values.exists(_ == host) +} + +class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { + import TaskLocality.{ANY, PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL} + + val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong + + test("TaskSet with no preferences") { + sc = new SparkContext("local", "test") + val sched = new FakeClusterScheduler(sc, ("exec1", "host1")) + val taskSet = createTaskSet(1) + val manager = new ClusterTaskSetManager(sched, taskSet) + + // Offer a host with no CPUs + assert(manager.resourceOffer("exec1", "host1", 0, ANY) === None) + + // Offer a host with process-local as the constraint; this should work because the TaskSet + // above won't have any locality preferences + val taskOption = manager.resourceOffer("exec1", "host1", 2, TaskLocality.PROCESS_LOCAL) + assert(taskOption.isDefined) + val task = taskOption.get + assert(task.executorId === "exec1") + assert(sched.startedTasks.contains(0)) + + // Re-offer the host -- now we should get no more tasks + assert(manager.resourceOffer("exec1", "host1", 2, PROCESS_LOCAL) === None) + + // Tell it the task has finished + manager.statusUpdate(0, TaskState.FINISHED, createTaskResult(0)) + assert(sched.endedTasks(0) === Success) + assert(sched.finishedManagers.contains(manager)) + } + + test("multiple offers with no preferences") { + sc = new SparkContext("local", "test") + val sched = new FakeClusterScheduler(sc, ("exec1", "host1")) + val taskSet = createTaskSet(3) + val manager = new ClusterTaskSetManager(sched, taskSet) + + // First three offers should all find tasks + for (i <- 0 until 3) { + val taskOption = manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL) + assert(taskOption.isDefined) + val task = taskOption.get + assert(task.executorId === "exec1") + } + assert(sched.startedTasks.toSet === Set(0, 1, 2)) + + // Re-offer the host -- now we should get no more tasks + assert(manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL) === None) + + // Finish the first two tasks + manager.statusUpdate(0, TaskState.FINISHED, createTaskResult(0)) + manager.statusUpdate(1, TaskState.FINISHED, createTaskResult(1)) + assert(sched.endedTasks(0) === Success) + assert(sched.endedTasks(1) === Success) + assert(!sched.finishedManagers.contains(manager)) + + // Finish the last task + manager.statusUpdate(2, TaskState.FINISHED, createTaskResult(2)) + assert(sched.endedTasks(2) === Success) + assert(sched.finishedManagers.contains(manager)) + } + + test("basic delay scheduling") { + sc = new SparkContext("local", "test") + val sched = new FakeClusterScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) + val taskSet = createTaskSet(4, + Seq(TaskLocation("host1", "exec1")), + Seq(TaskLocation("host2", "exec2")), + Seq(TaskLocation("host1"), TaskLocation("host2", "exec2")), + Seq() // Last task has no locality prefs + ) + val clock = new FakeClock + val manager = new ClusterTaskSetManager(sched, taskSet, clock) + + // First offer host1, exec1: first task should be chosen + assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0) + + // Offer host1, exec1 again: the last task, which has no prefs, should be chosen + assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 3) + + // Offer host1, exec1 again, at PROCESS_LOCAL level: nothing should get chosen + assert(manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL) === None) + + clock.advance(LOCALITY_WAIT) + + // Offer host1, exec1 again, at PROCESS_LOCAL level: nothing should get chosen + assert(manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL) === None) + + // Offer host1, exec1 again, at NODE_LOCAL level: we should choose task 2 + assert(manager.resourceOffer("exec1", "host1", 1, NODE_LOCAL).get.index == 2) + + // Offer host1, exec1 again, at NODE_LOCAL level: nothing should get chosen + assert(manager.resourceOffer("exec1", "host1", 1, NODE_LOCAL) === None) + + // Offer host1, exec1 again, at ANY level: nothing should get chosen + assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None) + + clock.advance(LOCALITY_WAIT) + + // Offer host1, exec1 again, at ANY level: task 1 should get chosen + assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 1) + + // Offer host1, exec1 again, at ANY level: nothing should be chosen as we've launched all tasks + assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None) + } + + test("delay scheduling with fallback") { + sc = new SparkContext("local", "test") + val sched = new FakeClusterScheduler(sc, + ("exec1", "host1"), ("exec2", "host2"), ("exec3", "host3")) + val taskSet = createTaskSet(5, + Seq(TaskLocation("host1")), + Seq(TaskLocation("host2")), + Seq(TaskLocation("host2")), + Seq(TaskLocation("host3")), + Seq(TaskLocation("host2")) + ) + val clock = new FakeClock + val manager = new ClusterTaskSetManager(sched, taskSet, clock) + + // First offer host1: first task should be chosen + assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0) + + // Offer host1 again: nothing should get chosen + assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None) + + clock.advance(LOCALITY_WAIT) + + // Offer host1 again: second task (on host2) should get chosen + assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 1) + + // Offer host1 again: third task (on host2) should get chosen + assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 2) + + // Offer host2: fifth task (also on host2) should get chosen + assert(manager.resourceOffer("exec2", "host2", 1, ANY).get.index === 4) + + // Now that we've launched a local task, we should no longer launch the task for host3 + assert(manager.resourceOffer("exec2", "host2", 1, ANY) === None) + + clock.advance(LOCALITY_WAIT) + + // After another delay, we can go ahead and launch that task non-locally + assert(manager.resourceOffer("exec2", "host2", 1, ANY).get.index === 3) + } + + test("delay scheduling with failed hosts") { + sc = new SparkContext("local", "test") + val sched = new FakeClusterScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) + val taskSet = createTaskSet(3, + Seq(TaskLocation("host1")), + Seq(TaskLocation("host2")), + Seq(TaskLocation("host3")) + ) + val clock = new FakeClock + val manager = new ClusterTaskSetManager(sched, taskSet, clock) + + // First offer host1: first task should be chosen + assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0) + + // Offer host1 again: third task should be chosen immediately because host3 is not up + assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 2) + + // After this, nothing should get chosen + assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None) + + // Now mark host2 as dead + sched.removeExecutor("exec2") + manager.executorLost("exec2", "host2") + + // Task 1 should immediately be launched on host1 because its original host is gone + assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 1) + + // Now that all tasks have launched, nothing new should be launched anywhere else + assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None) + assert(manager.resourceOffer("exec2", "host2", 1, ANY) === None) + } + + /** + * Utility method to create a TaskSet, potentially setting a particular sequence of preferred + * locations for each task (given as varargs) if this sequence is not empty. + */ + def createTaskSet(numTasks: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { + if (prefLocs.size != 0 && prefLocs.size != numTasks) { + throw new IllegalArgumentException("Wrong number of task locations") + } + val tasks = Array.tabulate[Task[_]](numTasks) { i => + new FakeTask(i, if (prefLocs.size != 0) prefLocs(i) else Nil) + } + new TaskSet(tasks, 0, 0, 0, null) + } + + def createTaskResult(id: Int): ByteBuffer = { + ByteBuffer.wrap(Utils.serialize(new TaskResult[Int](id, mutable.Map.empty, new TaskMetrics))) + } +} diff --git a/core/src/test/scala/spark/scheduler/cluster/FakeTask.scala b/core/src/test/scala/spark/scheduler/cluster/FakeTask.scala new file mode 100644 index 00000000000..de9e66be20a --- /dev/null +++ b/core/src/test/scala/spark/scheduler/cluster/FakeTask.scala @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.scheduler.cluster + +import spark.scheduler.{TaskLocation, Task} + +class FakeTask(stageId: Int, prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId) { + override def run(attemptId: Long): Int = 0 + + override def preferredLocations: Seq[TaskLocation] = prefLocs +} diff --git a/core/src/test/scala/spark/util/FakeClock.scala b/core/src/test/scala/spark/util/FakeClock.scala new file mode 100644 index 00000000000..236706317e3 --- /dev/null +++ b/core/src/test/scala/spark/util/FakeClock.scala @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package spark.util + +class FakeClock extends Clock { + private var time = 0L + + def advance(millis: Long): Unit = time += millis + + def getTime(): Long = time +} From 82bf4c0339808f51c9cdffa6a0a829cb5981d92d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 18 Aug 2013 20:25:45 -0700 Subject: [PATCH 0779/2521] Allow subclasses of Product2 in all key-value related classes (ShuffleDependency, PairRDDFunctions, etc). --- core/src/main/scala/spark/Aggregator.scala | 8 +- core/src/main/scala/spark/Dependency.scala | 4 +- .../main/scala/spark/PairRDDFunctions.scala | 104 +++++------------- core/src/main/scala/spark/Partitioner.scala | 2 +- core/src/main/scala/spark/RDD.scala | 5 +- core/src/main/scala/spark/SparkContext.scala | 9 +- .../scala/spark/api/java/JavaPairRDD.scala | 13 ++- .../main/scala/spark/rdd/CoGroupedRDD.scala | 6 +- .../main/scala/spark/rdd/ShuffledRDD.scala | 6 +- .../main/scala/spark/rdd/SubtractedRDD.scala | 6 +- .../tools/JavaAPICompletenessChecker.scala | 6 +- 11 files changed, 60 insertions(+), 109 deletions(-) diff --git a/core/src/main/scala/spark/Aggregator.scala b/core/src/main/scala/spark/Aggregator.scala index 136b4da61e5..3920f8511c8 100644 --- a/core/src/main/scala/spark/Aggregator.scala +++ b/core/src/main/scala/spark/Aggregator.scala @@ -28,11 +28,11 @@ import scala.collection.JavaConversions._ * @param mergeCombiners function to merge outputs from multiple mergeValue function. */ case class Aggregator[K, V, C] ( - val createCombiner: V => C, - val mergeValue: (C, V) => C, - val mergeCombiners: (C, C) => C) { + createCombiner: V => C, + mergeValue: (C, V) => C, + mergeCombiners: (C, C) => C) { - def combineValuesByKey(iter: Iterator[(K, V)]) : Iterator[(K, C)] = { + def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]) : Iterator[(K, C)] = { val combiners = new JHashMap[K, C] for ((k, v) <- iter) { val oldC = combiners.get(k) diff --git a/core/src/main/scala/spark/Dependency.scala b/core/src/main/scala/spark/Dependency.scala index b1edaa06f80..d5a96065709 100644 --- a/core/src/main/scala/spark/Dependency.scala +++ b/core/src/main/scala/spark/Dependency.scala @@ -44,10 +44,10 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { * @param serializerClass class name of the serializer to use */ class ShuffleDependency[K, V]( - @transient rdd: RDD[(K, V)], + @transient rdd: RDD[_ <: Product2[K, V]], val partitioner: Partitioner, val serializerClass: String = null) - extends Dependency(rdd) { + extends Dependency(rdd.asInstanceOf[RDD[Product2[K, V]]]) { 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 0be4b4feb8c..3ae703ce1ae 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -21,9 +21,8 @@ import java.nio.ByteBuffer import java.util.{Date, HashMap => JHashMap} import java.text.SimpleDateFormat -import scala.collection.Map +import scala.collection.{mutable, Map} import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ import org.apache.hadoop.conf.Configuration @@ -38,7 +37,6 @@ 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} -import org.apache.hadoop.security.UserGroupInformation import spark.partial.BoundedDouble import spark.partial.PartialResult @@ -50,8 +48,7 @@ import spark.Partitioner._ * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. * Import `spark.SparkContext._` at the top of your program to use these functions. */ -class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( - self: RDD[(K, V)]) +class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[_ <: Product2[K, V]]) extends Logging with HadoopMapReduceUtil with Serializable { @@ -85,18 +82,17 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( } val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners) if (self.partitioner == Some(partitioner)) { - self.mapPartitions(aggregator.combineValuesByKey, true) + self.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true) } else if (mapSideCombine) { - val mapSideCombined = self.mapPartitions(aggregator.combineValuesByKey, true) - val partitioned = new ShuffledRDD[K, C](mapSideCombined, partitioner) - .setSerializer(serializerClass) - partitioned.mapPartitions(aggregator.combineCombinersByKey, true) + val combined = self.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true) + val partitioned = new ShuffledRDD[K, C](combined, partitioner).setSerializer(serializerClass) + partitioned.mapPartitions(aggregator.combineCombinersByKey, preservesPartitioning = 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).setSerializer(serializerClass) - values.mapPartitions(aggregator.combineValuesByKey, true) + values.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true) } } @@ -166,7 +162,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( throw new SparkException("reduceByKeyLocally() does not support array keys") } - def reducePartition(iter: Iterator[(K, V)]): Iterator[JHashMap[K, V]] = { + def reducePartition(iter: Iterator[Product2[K, V]]): Iterator[JHashMap[K, V]] = { val map = new JHashMap[K, V] for ((k, v) <- iter) { val old = map.get(k) @@ -180,7 +176,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( val old = m1.get(k) m1.put(k, if (old == null) v else func(old, v)) } - return m1 + m1 } self.mapPartitions(reducePartition).reduce(mergeMaps) @@ -378,7 +374,13 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( /** * Return the key-value pairs in this RDD to the master as a Map. */ - def collectAsMap(): Map[K, V] = HashMap(self.collect(): _*) + def collectAsMap(): Map[K, V] = { + val data = self.toArray() + val map = new mutable.HashMap[K, V] + map.sizeHint(data.length) + data.foreach { case(k, v) => map.put(k, v) } + map + } /** * Pass each value in the key-value pair RDD through a map function without changing the keys; @@ -406,13 +408,10 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } - val cg = new CoGroupedRDD[K]( - Seq(self.asInstanceOf[RDD[(K, _)]], other.asInstanceOf[RDD[(K, _)]]), - partitioner) + val cg = new CoGroupedRDD[K](Seq(self, other), partitioner) val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest) - prfs.mapValues { - case Seq(vs, ws) => - (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]]) + prfs.mapValues { case Seq(vs, ws) => + (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]]) } } @@ -425,15 +424,10 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } - val cg = new CoGroupedRDD[K]( - Seq(self.asInstanceOf[RDD[(K, _)]], - other1.asInstanceOf[RDD[(K, _)]], - other2.asInstanceOf[RDD[(K, _)]]), - partitioner) + val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner) val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest) - prfs.mapValues { - case Seq(vs, w1s, w2s) => - (vs.asInstanceOf[Seq[V]], w1s.asInstanceOf[Seq[W1]], w2s.asInstanceOf[Seq[W2]]) + prfs.mapValues { case Seq(vs, w1s, w2s) => + (vs.asInstanceOf[Seq[V]], w1s.asInstanceOf[Seq[W1]], w2s.asInstanceOf[Seq[W2]]) } } @@ -507,7 +501,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( self.partitioner match { case Some(p) => val index = p.getPartition(key) - def process(it: Iterator[(K, V)]): Seq[V] = { + def process(it: Iterator[Product2[K, V]]): Seq[V] = { val buf = new ArrayBuffer[V] for ((k, v) <- it if k == key) { buf += v @@ -565,7 +559,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( val formatter = new SimpleDateFormat("yyyyMMddHHmm") val jobtrackerID = formatter.format(new Date()) val stageId = self.id - def writeShard(context: spark.TaskContext, iter: Iterator[(K,V)]): Int = { + def writeShard(context: spark.TaskContext, iter: Iterator[Product2[K,V]]): Int = { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt @@ -664,7 +658,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( val writer = new HadoopWriter(conf) writer.preSetup() - def writeToFile(context: TaskContext, iter: Iterator[(K,V)]) { + def writeToFile(context: TaskContext, iter: Iterator[Product2[K,V]]) { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt @@ -703,54 +697,6 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( private[spark] def getValueClass() = implicitly[ClassManifest[V]].erasure } -/** - * Extra functions available on RDDs of (key, value) pairs where the key is sortable through - * an implicit conversion. Import `spark.SparkContext._` at the top of your program to use these - * functions. They will work with any key type that has a `scala.math.Ordered` implementation. - */ -class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest]( - self: RDD[(K, V)]) - extends Logging - with Serializable { - - /** - * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling - * `collect` or `save` on the resulting RDD will return or output an ordered list of records - * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in - * order of the keys). - */ - def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size): RDD[(K,V)] = { - val shuffled = - new ShuffledRDD[K, V](self, new RangePartitioner(numPartitions, self, ascending)) - shuffled.mapPartitions(iter => { - val buf = iter.toArray - if (ascending) { - buf.sortWith((x, y) => x._1 < y._1).iterator - } else { - buf.sortWith((x, y) => x._1 > y._1).iterator - } - }, true) - } -} - -private[spark] -class MappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => U) extends RDD[(K, U)](prev) { - override def getPartitions = firstParent[(K, V)].partitions - override val partitioner = firstParent[(K, V)].partitioner - override def compute(split: Partition, context: TaskContext) = - firstParent[(K, V)].iterator(split, context).map{ case (k, v) => (k, f(v)) } -} - -private[spark] -class FlatMappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => TraversableOnce[U]) - extends RDD[(K, U)](prev) { - - override def getPartitions = firstParent[(K, V)].partitions - override val partitioner = firstParent[(K, V)].partitioner - override def compute(split: Partition, context: TaskContext) = { - firstParent[(K, V)].iterator(split, context).flatMap { case (k, v) => f(v).map(x => (k, x)) } - } -} private[spark] object Manifests { val seqSeqManifest = classManifest[Seq[Seq[_]]] diff --git a/core/src/main/scala/spark/Partitioner.scala b/core/src/main/scala/spark/Partitioner.scala index 6035bc075ef..65da8235d75 100644 --- a/core/src/main/scala/spark/Partitioner.scala +++ b/core/src/main/scala/spark/Partitioner.scala @@ -84,7 +84,7 @@ class HashPartitioner(partitions: Int) extends Partitioner { */ class RangePartitioner[K <% Ordered[K]: ClassManifest, V]( partitions: Int, - @transient rdd: RDD[(K,V)], + @transient rdd: RDD[_ <: Product2[K,V]], private val ascending: Boolean = true) extends Partitioner { diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 503ea6ccbf4..04b37df2129 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -287,7 +287,10 @@ abstract class RDD[T: ClassManifest]( def coalesce(numPartitions: Int, shuffle: Boolean = false): RDD[T] = { if (shuffle) { // include a shuffle step so that our upstream tasks are still distributed - new CoalescedRDD(new ShuffledRDD(map(x => (x, null)), new HashPartitioner(numPartitions)), numPartitions).keys + new CoalescedRDD( + new ShuffledRDD(map(x => (x, null)), + new HashPartitioner(numPartitions)), + numPartitions).keys } else { new CoalescedRDD(this, numPartitions) } diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 80c65dfebd2..c049bd3fa90 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -60,7 +60,8 @@ 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.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD, + OrderedRDDFunctions} import spark.scheduler.{DAGScheduler, DAGSchedulerSource, ResultTask, ShuffleMapTask, SparkListener, SplitInfo, Stage, StageInfo, TaskScheduler, ActiveJob} import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, @@ -833,11 +834,11 @@ class SparkContext( /** Default min number of partitions for Hadoop RDDs when not given by user */ def defaultMinSplits: Int = math.min(defaultParallelism, 2) - private var nextShuffleId = new AtomicInteger(0) + private val nextShuffleId = new AtomicInteger(0) private[spark] def newShuffleId(): Int = nextShuffleId.getAndIncrement() - private var nextRddId = new AtomicInteger(0) + private val nextRddId = new AtomicInteger(0) /** Register a new RDD, returning its RDD ID */ private[spark] def newRddId(): Int = nextRddId.getAndIncrement() @@ -886,7 +887,7 @@ object SparkContext { implicit def rddToOrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest]( rdd: RDD[(K, V)]) = - new OrderedRDDFunctions(rdd) + new OrderedRDDFunctions(rdd.asInstanceOf[RDD[Product2[K, V]]]) implicit def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]) = new DoubleRDDFunctions(rdd) diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index c2995b836a1..f5632428e79 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -30,17 +30,18 @@ import org.apache.hadoop.mapred.OutputFormat import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.hadoop.conf.Configuration -import spark.api.java.function.{Function2 => JFunction2} -import spark.api.java.function.{Function => JFunction} -import spark.partial.BoundedDouble -import spark.partial.PartialResult -import spark.OrderedRDDFunctions -import spark.storage.StorageLevel import spark.HashPartitioner import spark.Partitioner import spark.Partitioner._ import spark.RDD import spark.SparkContext.rddToPairRDDFunctions +import spark.api.java.function.{Function2 => JFunction2} +import spark.api.java.function.{Function => JFunction} +import spark.partial.BoundedDouble +import spark.partial.PartialResult +import spark.rdd.OrderedRDDFunctions +import spark.storage.StorageLevel + class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManifest[K], implicit val vManifest: ClassManifest[V]) extends JavaRDDLike[(K, V), JavaPairRDD[K, V]] { diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index c2d95dc0604..06e15bb73c1 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -60,7 +60,7 @@ class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep]) * @param rdds parent RDDs. * @param part partitioner used to partition the shuffle output. */ -class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner) +class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { private var serializerClass: String = null @@ -71,13 +71,13 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner) } override def getDependencies: Seq[Dependency[_]] = { - rdds.map { rdd: RDD[(K, _)] => + rdds.map { rdd: RDD[_ <: Product2[K, _]] => if (rdd.partitioner == Some(part)) { logInfo("Adding one-to-one dependency with " + rdd) new OneToOneDependency(rdd) } else { logInfo("Adding shuffle dependency with " + rdd) - new ShuffleDependency[Any, Any](rdd.asInstanceOf[RDD[(Any, Any)]], part, serializerClass) + new ShuffleDependency[Any, Any](rdd, part, serializerClass) } } } diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala index bcf7d0d89c4..2eac62f9c05 100644 --- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala @@ -17,9 +17,7 @@ package spark.rdd -import spark._ -import scala.Some -import scala.Some +import spark.{Dependency, Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext} private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @@ -35,7 +33,7 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { * @tparam V the value class. */ class ShuffledRDD[K, V]( - @transient var prev: RDD[(K, V)], + @transient var prev: RDD[_ <: Product2[K, V]], part: Partitioner) extends RDD[(K, V)](prev.context, Nil) { diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala index 46b8cafaac3..200e85d4327 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -47,8 +47,8 @@ import spark.OneToOneDependency * out of memory because of the size of `rdd2`. */ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassManifest]( - @transient var rdd1: RDD[(K, V)], - @transient var rdd2: RDD[(K, W)], + @transient var rdd1: RDD[_ <: Product2[K, V]], + @transient var rdd2: RDD[_ <: Product2[K, W]], part: Partitioner) extends RDD[(K, V)](rdd1.context, Nil) { @@ -66,7 +66,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, serializerClass) + new ShuffleDependency(rdd, part, serializerClass) } } } diff --git a/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala index 30fded12f02..b22e6c25df5 100644 --- a/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala +++ b/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala @@ -17,13 +17,15 @@ package spark.tools -import spark._ import java.lang.reflect.Method + import scala.collection.mutable.ArrayBuffer + +import spark._ import spark.api.java._ +import spark.rdd.OrderedRDDFunctions 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) From 2a7b99c08b29d3002183a8d7ed3acd14fbf5dc41 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 18 Aug 2013 20:39:29 -0700 Subject: [PATCH 0780/2521] Added the missing RDD files and cleaned up SparkContext. --- core/src/main/scala/spark/SparkContext.scala | 17 ++----- .../scala/spark/rdd/FlatMappedValuesRDD.scala | 36 +++++++++++++ .../scala/spark/rdd/MappedValuesRDD.scala | 34 +++++++++++++ .../scala/spark/rdd/OrderedRDDFunctions.scala | 51 +++++++++++++++++++ 4 files changed, 126 insertions(+), 12 deletions(-) create mode 100644 core/src/main/scala/spark/rdd/FlatMappedValuesRDD.scala create mode 100644 core/src/main/scala/spark/rdd/MappedValuesRDD.scala create mode 100644 core/src/main/scala/spark/rdd/OrderedRDDFunctions.scala diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index c049bd3fa90..5db17671460 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -20,19 +20,14 @@ package spark import java.io._ import java.net.URI import java.util.Properties -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.mutable.ArrayBuffer import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap import scala.util.DynamicVariable -import scala.collection.mutable.{ConcurrentMap, HashMap} - -import akka.actor.Actor._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -54,7 +49,6 @@ 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 @@ -63,15 +57,14 @@ import spark.partial.{ApproximateEvaluator, PartialResult} import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD, OrderedRDDFunctions} import spark.scheduler.{DAGScheduler, DAGSchedulerSource, ResultTask, ShuffleMapTask, SparkListener, - SplitInfo, Stage, StageInfo, TaskScheduler, ActiveJob} + SplitInfo, Stage, StageInfo, TaskScheduler} import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler, Schedulable, SchedulingMode} import spark.scheduler.local.LocalScheduler import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import spark.storage.{StorageStatus, StorageUtils, RDDInfo, BlockManagerSource} +import spark.ui.SparkUI import spark.util.{MetadataCleaner, TimeStampedHashMap} -import ui.{SparkUI} -import spark.metrics._ /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -887,7 +880,7 @@ object SparkContext { implicit def rddToOrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest]( rdd: RDD[(K, V)]) = - new OrderedRDDFunctions(rdd.asInstanceOf[RDD[Product2[K, V]]]) + new OrderedRDDFunctions(rdd) implicit def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]) = new DoubleRDDFunctions(rdd) diff --git a/core/src/main/scala/spark/rdd/FlatMappedValuesRDD.scala b/core/src/main/scala/spark/rdd/FlatMappedValuesRDD.scala new file mode 100644 index 00000000000..05fdfd82c16 --- /dev/null +++ b/core/src/main/scala/spark/rdd/FlatMappedValuesRDD.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR 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.{TaskContext, Partition, RDD} + + +private[spark] +class FlatMappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => TraversableOnce[U]) + extends RDD[(K, U)](prev) { + + override def getPartitions = firstParent[Product2[K, V]].partitions + + override val partitioner = firstParent[Product2[K, V]].partitioner + + override def compute(split: Partition, context: TaskContext) = { + firstParent[Product2[K, V]].iterator(split, context).flatMap { case (k, v) => + f(v).map(x => (k, x)) + } + } +} diff --git a/core/src/main/scala/spark/rdd/MappedValuesRDD.scala b/core/src/main/scala/spark/rdd/MappedValuesRDD.scala new file mode 100644 index 00000000000..21ae97daa93 --- /dev/null +++ b/core/src/main/scala/spark/rdd/MappedValuesRDD.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR 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.{TaskContext, Partition, RDD} + +private[spark] +class MappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => U) + extends RDD[(K, U)](prev) { + + override def getPartitions = firstParent[Product2[K, U]].partitions + + override val partitioner = firstParent[Product2[K, U]].partitioner + + override def compute(split: Partition, context: TaskContext): Iterator[(K, U)] = { + firstParent[Product2[K, V]].iterator(split, context).map { case(k ,v) => (k, f(v)) } + } +} diff --git a/core/src/main/scala/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/spark/rdd/OrderedRDDFunctions.scala new file mode 100644 index 00000000000..6328c6a4ac1 --- /dev/null +++ b/core/src/main/scala/spark/rdd/OrderedRDDFunctions.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR 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.{RangePartitioner, Logging, RDD} + +/** + * Extra functions available on RDDs of (key, value) pairs where the key is sortable through + * an implicit conversion. Import `spark.SparkContext._` at the top of your program to use these + * functions. They will work with any key type that has a `scala.math.Ordered` implementation. + */ +class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest]( + self: RDD[_ <: Product2[K, V]]) + extends Logging with Serializable { + + /** + * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling + * `collect` or `save` on the resulting RDD will return or output an ordered list of records + * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in + * order of the keys). + */ + def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size) + : RDD[(K, V)] = + { + val part = new RangePartitioner(numPartitions, self.asInstanceOf[RDD[Product2[K,V]]], ascending) + val shuffled = new ShuffledRDD[K, V](self, part) + shuffled.mapPartitions(iter => { + val buf = iter.toArray + if (ascending) { + buf.sortWith((x, y) => x._1 < y._1).iterator + } else { + buf.sortWith((x, y) => x._1 > y._1).iterator + } + }, preservesPartitioning = true) + } +} From 71d705a66eb8782e5cd5c77853fdd99fd8155334 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 19 Aug 2013 00:40:43 -0700 Subject: [PATCH 0781/2521] Made PairRDDFunctions taking only Tuple2, but made the rest of the shuffle code path working with general Product2. --- core/src/main/scala/spark/Aggregator.scala | 10 ++-- .../spark/BlockStoreShuffleFetcher.scala | 11 ++-- .../main/scala/spark/PairRDDFunctions.scala | 50 +++++++++---------- core/src/main/scala/spark/RDD.scala | 9 ++-- .../src/main/scala/spark/ShuffleFetcher.scala | 5 +- core/src/main/scala/spark/SparkContext.scala | 2 +- .../scala/spark/api/java/JavaPairRDD.scala | 2 +- .../main/scala/spark/rdd/CoGroupedRDD.scala | 12 ++--- .../scala/spark/rdd/FlatMappedValuesRDD.scala | 2 +- .../scala/spark/rdd/MappedValuesRDD.scala | 2 +- .../scala/spark/rdd/OrderedRDDFunctions.scala | 14 +++--- .../main/scala/spark/rdd/ShuffledRDD.scala | 12 ++--- .../main/scala/spark/rdd/SubtractedRDD.scala | 14 +++--- .../spark/scheduler/ShuffleMapTask.scala | 2 +- .../main/scala/spark/util/MutablePair.scala | 34 +++++++++++++ .../test/scala/spark/CheckpointSuite.scala | 2 +- .../scala/spark/PairRDDFunctionsSuite.scala | 7 +-- core/src/test/scala/spark/RDDSuite.scala | 2 +- core/src/test/scala/spark/ShuffleSuite.scala | 31 +++++++++--- .../tools/JavaAPICompletenessChecker.scala | 2 +- 20 files changed, 133 insertions(+), 92 deletions(-) create mode 100644 core/src/main/scala/spark/util/MutablePair.scala diff --git a/core/src/main/scala/spark/Aggregator.scala b/core/src/main/scala/spark/Aggregator.scala index 3920f8511c8..9af401986d3 100644 --- a/core/src/main/scala/spark/Aggregator.scala +++ b/core/src/main/scala/spark/Aggregator.scala @@ -34,12 +34,12 @@ case class Aggregator[K, V, C] ( def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]) : Iterator[(K, C)] = { val combiners = new JHashMap[K, C] - for ((k, v) <- iter) { - val oldC = combiners.get(k) + for (kv <- iter) { + val oldC = combiners.get(kv._1) if (oldC == null) { - combiners.put(k, createCombiner(v)) + combiners.put(kv._1, createCombiner(kv._2)) } else { - combiners.put(k, mergeValue(oldC, v)) + combiners.put(kv._1, mergeValue(oldC, kv._2)) } } combiners.iterator @@ -47,7 +47,7 @@ case class Aggregator[K, V, C] ( def combineCombinersByKey(iter: Iterator[(K, C)]) : Iterator[(K, C)] = { val combiners = new JHashMap[K, C] - for ((k, c) <- iter) { + iter.foreach { case(k, c) => val oldC = combiners.get(k) if (oldC == null) { combiners.put(k, c) diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala index 8f6953b1f52..1ec95ed9b88 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala @@ -28,8 +28,9 @@ import spark.util.CompletionIterator private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { - override def fetch[K, V]( - shuffleId: Int, reduceId: Int, metrics: TaskMetrics, serializer: Serializer) = { + override def fetch[T](shuffleId: Int, reduceId: Int, metrics: TaskMetrics, serializer: Serializer) + : Iterator[T] = + { logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId)) val blockManager = SparkEnv.get.blockManager @@ -49,12 +50,12 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin (address, splits.map(s => ("shuffle_%d_%d_%d".format(shuffleId, s._1, reduceId), s._2))) } - def unpackBlock(blockPair: (String, Option[Iterator[Any]])) : Iterator[(K, V)] = { + def unpackBlock(blockPair: (String, Option[Iterator[Any]])) : Iterator[T] = { val blockId = blockPair._1 val blockOption = blockPair._2 blockOption match { case Some(block) => { - block.asInstanceOf[Iterator[(K, V)]] + block.asInstanceOf[Iterator[T]] } case None => { val regex = "shuffle_([0-9]*)_([0-9]*)_([0-9]*)".r @@ -73,7 +74,7 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer) val itr = blockFetcherItr.flatMap(unpackBlock) - CompletionIterator[(K,V), Iterator[(K,V)]](itr, { + CompletionIterator[T, Iterator[T]](itr, { val shuffleMetrics = new ShuffleReadMetrics shuffleMetrics.shuffleFinishTime = System.currentTimeMillis shuffleMetrics.remoteFetchTime = blockFetcherItr.remoteFetchTime diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 3ae703ce1ae..f8900d39214 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -48,7 +48,7 @@ import spark.Partitioner._ * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. * Import `spark.SparkContext._` at the top of your program to use these functions. */ -class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[_ <: Product2[K, V]]) +class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) extends Logging with HadoopMapReduceUtil with Serializable { @@ -85,13 +85,14 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[_ <: Produc self.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true) } else if (mapSideCombine) { val combined = self.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true) - val partitioned = new ShuffledRDD[K, C](combined, partitioner).setSerializer(serializerClass) + val partitioned = new ShuffledRDD[K, C, (K, C)](combined, partitioner) + .setSerializer(serializerClass) partitioned.mapPartitions(aggregator.combineCombinersByKey, preservesPartitioning = 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).setSerializer(serializerClass) + val values = new ShuffledRDD[K, V, (K, V)](self, partitioner).setSerializer(serializerClass) values.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true) } } @@ -162,7 +163,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[_ <: Produc throw new SparkException("reduceByKeyLocally() does not support array keys") } - def reducePartition(iter: Iterator[Product2[K, V]]): Iterator[JHashMap[K, V]] = { + def reducePartition(iter: Iterator[(K, V)]): Iterator[JHashMap[K, V]] = { val map = new JHashMap[K, V] for ((k, v) <- iter) { val old = map.get(k) @@ -236,7 +237,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[_ <: Produc if (getKeyClass().isArray && partitioner.isInstanceOf[HashPartitioner]) { throw new SparkException("Default partitioner cannot partition array keys.") } - new ShuffledRDD[K, V](self, partitioner) + new ShuffledRDD[K, V, (K, V)](self, partitioner) } /** @@ -245,9 +246,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[_ <: Produc * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. */ def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { - this.cogroup(other, partitioner).flatMapValues { - case (vs, ws) => - for (v <- vs.iterator; w <- ws.iterator) yield (v, w) + this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => + for (v <- vs.iterator; w <- ws.iterator) yield (v, w) } } @@ -258,13 +258,12 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[_ <: Produc * partition the output RDD. */ def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { - this.cogroup(other, partitioner).flatMapValues { - case (vs, ws) => - if (ws.isEmpty) { - vs.iterator.map(v => (v, None)) - } else { - for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w)) - } + this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => + if (ws.isEmpty) { + vs.iterator.map(v => (v, None)) + } else { + for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w)) + } } } @@ -276,13 +275,12 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[_ <: Produc */ def rightOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner) : RDD[(K, (Option[V], W))] = { - this.cogroup(other, partitioner).flatMapValues { - case (vs, ws) => - if (vs.isEmpty) { - ws.iterator.map(w => (None, w)) - } else { - for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w) - } + this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => + if (vs.isEmpty) { + ws.iterator.map(w => (None, w)) + } else { + for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w) + } } } @@ -378,7 +376,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[_ <: Produc val data = self.toArray() val map = new mutable.HashMap[K, V] map.sizeHint(data.length) - data.foreach { case(k, v) => map.put(k, v) } + data.foreach { case (k, v) => map.put(k, v) } map } @@ -501,7 +499,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[_ <: Produc self.partitioner match { case Some(p) => val index = p.getPartition(key) - def process(it: Iterator[Product2[K, V]]): Seq[V] = { + def process(it: Iterator[(K, V)]): Seq[V] = { val buf = new ArrayBuffer[V] for ((k, v) <- it if k == key) { buf += v @@ -559,7 +557,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[_ <: Produc val formatter = new SimpleDateFormat("yyyyMMddHHmm") val jobtrackerID = formatter.format(new Date()) val stageId = self.id - def writeShard(context: spark.TaskContext, iter: Iterator[Product2[K,V]]): Int = { + def writeShard(context: spark.TaskContext, iter: Iterator[(K,V)]): Int = { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt @@ -658,7 +656,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[_ <: Produc val writer = new HadoopWriter(conf) writer.preSetup() - def writeToFile(context: TaskContext, iter: Iterator[Product2[K,V]]) { + def writeToFile(context: TaskContext, iter: Iterator[(K, V)]) { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 04b37df2129..c9a044afabb 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -31,9 +31,8 @@ import org.apache.hadoop.mapred.TextOutputFormat import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} -import spark.api.java.JavaRDD -import spark.broadcast.Broadcast import spark.Partitioner._ +import spark.api.java.JavaRDD import spark.partial.BoundedDouble import spark.partial.CountEvaluator import spark.partial.GroupedCountEvaluator @@ -288,7 +287,7 @@ abstract class RDD[T: ClassManifest]( if (shuffle) { // include a shuffle step so that our upstream tasks are still distributed new CoalescedRDD( - new ShuffledRDD(map(x => (x, null)), + new ShuffledRDD[T, Null, (T, Null)](map(x => (x, null)), new HashPartitioner(numPartitions)), numPartitions).keys } else { @@ -305,8 +304,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 = this.count() + val multiplier = 3.0 + val initialCount = this.count() var maxSelected = 0 if (num < 0) { diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala index dcced035e74..a6839cf7a40 100644 --- a/core/src/main/scala/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/spark/ShuffleFetcher.scala @@ -22,12 +22,13 @@ 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, - serializer: Serializer = SparkEnv.get.serializerManager.default): Iterator[(K,V)] + def fetch[T](shuffleId: Int, reduceId: Int, metrics: TaskMetrics, + serializer: Serializer = SparkEnv.get.serializerManager.default): Iterator[T] /** Stop the fetcher */ def stop() {} diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 5db17671460..185c76366fc 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -880,7 +880,7 @@ object SparkContext { implicit def rddToOrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest]( rdd: RDD[(K, V)]) = - new OrderedRDDFunctions(rdd) + new OrderedRDDFunctions[K, V, (K, V)](rdd) implicit def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]) = new DoubleRDDFunctions(rdd) diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index f5632428e79..effe6e5e0d6 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -564,7 +564,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif override def compare(b: K) = comp.compare(a, b) } implicit def toOrdered(x: K): Ordered[K] = new KeyOrdering(x) - fromRDD(new OrderedRDDFunctions(rdd).sortByKey(ascending)) + fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending)) } /** diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 06e15bb73c1..01b6c23dcc2 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -73,10 +73,10 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: override def getDependencies: Seq[Dependency[_]] = { rdds.map { rdd: RDD[_ <: Product2[K, _]] => if (rdd.partitioner == Some(part)) { - logInfo("Adding one-to-one dependency with " + rdd) + logDebug("Adding one-to-one dependency with " + rdd) new OneToOneDependency(rdd) } else { - logInfo("Adding shuffle dependency with " + rdd) + logDebug("Adding shuffle dependency with " + rdd) new ShuffleDependency[Any, Any](rdd, part, serializerClass) } } @@ -122,15 +122,15 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: for ((dep, depNum) <- split.deps.zipWithIndex) dep match { case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { // Read them from the parent - for ((k, v) <- rdd.iterator(itsSplit, context)) { - getSeq(k.asInstanceOf[K])(depNum) += v + rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]].foreach { kv => + getSeq(kv._1)(depNum) += kv._2 } } case ShuffleCoGroupSplitDep(shuffleId) => { // Read map outputs of shuffle val fetcher = SparkEnv.get.shuffleFetcher - fetcher.fetch[K, Any](shuffleId, split.index, context.taskMetrics, ser).foreach { - case (key, value) => getSeq(key)(depNum) += value + fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context.taskMetrics, ser).foreach { + kv => getSeq(kv._1)(depNum) += kv._2 } } } diff --git a/core/src/main/scala/spark/rdd/FlatMappedValuesRDD.scala b/core/src/main/scala/spark/rdd/FlatMappedValuesRDD.scala index 05fdfd82c16..a6bdce89d8a 100644 --- a/core/src/main/scala/spark/rdd/FlatMappedValuesRDD.scala +++ b/core/src/main/scala/spark/rdd/FlatMappedValuesRDD.scala @@ -29,7 +29,7 @@ class FlatMappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => Trave override val partitioner = firstParent[Product2[K, V]].partitioner override def compute(split: Partition, context: TaskContext) = { - firstParent[Product2[K, V]].iterator(split, context).flatMap { case (k, v) => + firstParent[Product2[K, V]].iterator(split, context).flatMap { case Product2(k, v) => f(v).map(x => (k, x)) } } diff --git a/core/src/main/scala/spark/rdd/MappedValuesRDD.scala b/core/src/main/scala/spark/rdd/MappedValuesRDD.scala index 21ae97daa93..8334e3b557d 100644 --- a/core/src/main/scala/spark/rdd/MappedValuesRDD.scala +++ b/core/src/main/scala/spark/rdd/MappedValuesRDD.scala @@ -29,6 +29,6 @@ class MappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => U) override val partitioner = firstParent[Product2[K, U]].partitioner override def compute(split: Partition, context: TaskContext): Iterator[(K, U)] = { - firstParent[Product2[K, V]].iterator(split, context).map { case(k ,v) => (k, f(v)) } + firstParent[Product2[K, V]].iterator(split, context).map { case Product2(k ,v) => (k, f(v)) } } } diff --git a/core/src/main/scala/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/spark/rdd/OrderedRDDFunctions.scala index 6328c6a4ac1..9154b760359 100644 --- a/core/src/main/scala/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/spark/rdd/OrderedRDDFunctions.scala @@ -24,8 +24,10 @@ import spark.{RangePartitioner, Logging, RDD} * an implicit conversion. Import `spark.SparkContext._` at the top of your program to use these * functions. They will work with any key type that has a `scala.math.Ordered` implementation. */ -class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest]( - self: RDD[_ <: Product2[K, V]]) +class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, + V: ClassManifest, + P <: Product2[K, V] : ClassManifest]( + self: RDD[P]) extends Logging with Serializable { /** @@ -34,11 +36,9 @@ class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest]( * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in * order of the keys). */ - def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size) - : RDD[(K, V)] = - { - val part = new RangePartitioner(numPartitions, self.asInstanceOf[RDD[Product2[K,V]]], ascending) - val shuffled = new ShuffledRDD[K, V](self, part) + def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size): RDD[P] = { + val part = new RangePartitioner(numPartitions, self, ascending) + val shuffled = new ShuffledRDD[K, V, P](self, part) shuffled.mapPartitions(iter => { val buf = iter.toArray if (ascending) { diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala index 2eac62f9c05..51c05af064e 100644 --- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala @@ -32,14 +32,14 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { * @tparam K the key class. * @tparam V the value class. */ -class ShuffledRDD[K, V]( - @transient var prev: RDD[_ <: Product2[K, V]], +class ShuffledRDD[K, V, P <: Product2[K, V] : ClassManifest]( + @transient var prev: RDD[P], part: Partitioner) - extends RDD[(K, V)](prev.context, Nil) { + extends RDD[P](prev.context, Nil) { private var serializerClass: String = null - def setSerializer(cls: String): ShuffledRDD[K, V] = { + def setSerializer(cls: String): ShuffledRDD[K, V, P] = { serializerClass = cls this } @@ -54,9 +54,9 @@ class ShuffledRDD[K, V]( Array.tabulate[Partition](part.numPartitions)(i => new ShuffledRDDPartition(i)) } - override def compute(split: Partition, context: TaskContext): Iterator[(K, V)] = { + override def compute(split: Partition, context: TaskContext): Iterator[P] = { val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId - SparkEnv.get.shuffleFetcher.fetch[K, V](shuffledId, split.index, context.taskMetrics, + SparkEnv.get.shuffleFetcher.fetch[P](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 200e85d4327..dadef5e17de 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala @@ -62,10 +62,10 @@ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassM override def getDependencies: Seq[Dependency[_]] = { Seq(rdd1, rdd2).map { rdd => if (rdd.partitioner == Some(part)) { - logInfo("Adding one-to-one dependency with " + rdd) + logDebug("Adding one-to-one dependency with " + rdd) new OneToOneDependency(rdd) } else { - logInfo("Adding shuffle dependency with " + rdd) + logDebug("Adding shuffle dependency with " + rdd) new ShuffleDependency(rdd, part, serializerClass) } } @@ -103,16 +103,14 @@ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassM seq } } - def integrate(dep: CoGroupSplitDep, op: ((K, V)) => Unit) = dep match { + def integrate(dep: CoGroupSplitDep, op: Product2[K, V] => Unit) = dep match { case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { - for (t <- rdd.iterator(itsSplit, context)) - op(t.asInstanceOf[(K, V)]) + rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, V]]].foreach(op) } case ShuffleCoGroupSplitDep(shuffleId) => { - val iter = SparkEnv.get.shuffleFetcher.fetch(shuffleId, partition.index, + val iter = SparkEnv.get.shuffleFetcher.fetch[Product2[K, V]](shuffleId, partition.index, context.taskMetrics, serializer) - for (t <- iter) - op(t.asInstanceOf[(K, V)]) + iter.foreach(op) } } // the first dep is rdd1; add all values to the map diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala index e3bb6d1e609..121ff31121d 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala @@ -148,7 +148,7 @@ private[spark] class ShuffleMapTask( // Write the map output to its associated buckets. for (elem <- rdd.iterator(split, taskContext)) { - val pair = elem.asInstanceOf[(Any, Any)] + val pair = elem.asInstanceOf[Product2[Any, Any]] val bucketId = dep.partitioner.getPartition(pair._1) buckets.writers(bucketId).write(pair) } diff --git a/core/src/main/scala/spark/util/MutablePair.scala b/core/src/main/scala/spark/util/MutablePair.scala new file mode 100644 index 00000000000..117218bf477 --- /dev/null +++ b/core/src/main/scala/spark/util/MutablePair.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR 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 + + +/** A tuple of 2 elements. + * @param _1 Element 1 of this MutablePair + * @param _2 Element 2 of this MutablePair + */ +case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T1, + @specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T2] + (var _1: T1,var _2: T2) + extends Product2[T1, T2] +{ + + override def toString = "(" + _1 + "," + _2 + ")" + + def canEqual(that: Any): Boolean = that.isInstanceOf[MutablePair[T1, T2]] +} diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala index a84c89e3c95..966dede2be1 100644 --- a/core/src/test/scala/spark/CheckpointSuite.scala +++ b/core/src/test/scala/spark/CheckpointSuite.scala @@ -99,7 +99,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { test("ShuffledRDD") { testCheckpointing(rdd => { // Creating ShuffledRDD directly as PairRDDFunctions.combineByKey produces a MapPartitionedRDD - new ShuffledRDD(rdd.map(x => (x % 2, 1)), partitioner) + new ShuffledRDD[Int, Int, (Int, Int)](rdd.map(x => (x % 2, 1)), partitioner) }) } diff --git a/core/src/test/scala/spark/PairRDDFunctionsSuite.scala b/core/src/test/scala/spark/PairRDDFunctionsSuite.scala index b102eaf4e65..328b3b5497e 100644 --- a/core/src/test/scala/spark/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/spark/PairRDDFunctionsSuite.scala @@ -21,16 +21,11 @@ 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))) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index cbddf4e523d..75778de1cc9 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -170,7 +170,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { // we can optionally shuffle to keep the upstream parallel val coalesced5 = data.coalesce(1, shuffle = true) - assert(coalesced5.dependencies.head.rdd.dependencies.head.rdd.asInstanceOf[ShuffledRDD[_, _]] != + assert(coalesced5.dependencies.head.rdd.dependencies.head.rdd.asInstanceOf[ShuffledRDD[_, _, _]] != null) } diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index c686b8cc5af..c319a57fdd1 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -22,6 +22,9 @@ import org.scalatest.matchers.ShouldMatchers import spark.rdd.ShuffledRDD import spark.SparkContext._ +import spark.ShuffleSuite.NonJavaSerializableClass +import spark.util.MutablePair + class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { test("groupByKey without compression") { @@ -46,12 +49,12 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { val a = sc.parallelize(1 to 10, 2) val b = a.map { x => - (x, new ShuffleSuite.NonJavaSerializableClass(x * 2)) + (x, new 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(NUM_BLOCKS)) - .setSerializer(classOf[spark.KryoSerializer].getName) + val c = new ShuffledRDD[Int, NonJavaSerializableClass, (Int, NonJavaSerializableClass)]( + b, new HashPartitioner(NUM_BLOCKS)).setSerializer(classOf[spark.KryoSerializer].getName) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId assert(c.count === 10) @@ -68,12 +71,12 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { 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)) + (x, new 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)) - .setSerializer(classOf[spark.KryoSerializer].getName) + val c = new ShuffledRDD[Int, NonJavaSerializableClass, (Int, NonJavaSerializableClass)]( + b, new HashPartitioner(3)).setSerializer(classOf[spark.KryoSerializer].getName) assert(c.count === 10) } @@ -88,7 +91,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { // NOTE: The default Java serializer doesn't create zero-sized blocks. // So, use Kryo - val c = new ShuffledRDD(b, new HashPartitioner(10)) + val c = new ShuffledRDD[Int, Int, (Int, Int)](b, new HashPartitioner(10)) .setSerializer(classOf[spark.KryoSerializer].getName) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId @@ -114,7 +117,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { val b = a.map(x => (x, x*2)) // NOTE: The default Java serializer should create zero-sized blocks - val c = new ShuffledRDD(b, new HashPartitioner(10)) + val c = new ShuffledRDD[Int, Int, (Int, Int)](b, new HashPartitioner(10)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId assert(c.count === 4) @@ -128,6 +131,18 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { // We should have at most 4 non-zero sized partitions assert(nonEmptyBlocks.size <= 4) } + + test("shuffle using mutable pairs") { + // 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") + def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) + val data = Array(p(1, 1), p(1, 2), p(1, 3), p(2, 1)) + val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data) + val results = new ShuffledRDD[Int, Int, MutablePair[Int, Int]](pairs, new HashPartitioner(2)) + .collect() + + data.foreach { pair => results should contain (pair) } + } } object ShuffleSuite { diff --git a/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala index b22e6c25df5..f45d0b281cf 100644 --- a/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala +++ b/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala @@ -338,7 +338,7 @@ object JavaAPICompletenessChecker { println() println("Missing OrderedRDD methods") - printMissingMethods(classOf[OrderedRDDFunctions[_, _]], classOf[JavaPairRDD[_, _]]) + printMissingMethods(classOf[OrderedRDDFunctions[_, _, _]], classOf[JavaPairRDD[_, _]]) println() println("Missing SparkContext methods") From acc4aa1f4701235be6eae25a9b940f36a87ea685 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 19 Aug 2013 11:02:10 -0700 Subject: [PATCH 0782/2521] Added a test for sorting using MutablePair's. --- core/src/test/scala/spark/ShuffleSuite.scala | 20 ++++++++++++++++++-- 1 file changed, 18 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index c319a57fdd1..f1361546a32 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -20,9 +20,10 @@ package spark import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers -import spark.rdd.ShuffledRDD import spark.SparkContext._ import spark.ShuffleSuite.NonJavaSerializableClass +import spark.rdd.OrderedRDDFunctions +import spark.rdd.ShuffledRDD import spark.util.MutablePair @@ -137,12 +138,27 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { sc = new SparkContext("local-cluster[2,1,512]", "test") def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) val data = Array(p(1, 1), p(1, 2), p(1, 3), p(2, 1)) - val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data) + val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data, 2) val results = new ShuffledRDD[Int, Int, MutablePair[Int, Int]](pairs, new HashPartitioner(2)) .collect() data.foreach { pair => results should contain (pair) } } + + test("sorting using mutable pairs") { + // This is not in SortingSuite because of the local cluster setup. + // 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") + def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) + val data = Array(p(1, 11), p(3, 33), p(100, 100), p(2, 22)) + val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data, 2) + val results = new OrderedRDDFunctions[Int, Int, MutablePair[Int, Int]](pairs) + .sortByKey().collect() + results(0) should be (p(1, 11)) + results(1) should be (p(2, 22)) + results(2) should be (p(3, 33)) + results(3) should be (p(100, 100)) + } } object ShuffleSuite { From 6f6944c8079bffdd088ddb0a84fbf83356e294ea Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Mon, 19 Aug 2013 12:33:13 -0700 Subject: [PATCH 0783/2521] Update SBT build to use simpler fix for Hadoop 0.23.9 --- project/SparkBuild.scala | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index cea982b8860..282b0cbed57 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -190,22 +190,13 @@ object SparkBuild extends Build { "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test", "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm), + "org.apache.avro" % "avro" % "1.7.4", + "org.apache.avro" % "avro-ipc" % "1.7.4" excludeAll(excludeNetty), "com.codahale.metrics" % "metrics-core" % "3.0.0", "com.codahale.metrics" % "metrics-jvm" % "3.0.0", "com.codahale.metrics" % "metrics-json" % "3.0.0", "com.twitter" % "chill_2.9.3" % "0.3.1", "com.twitter" % "chill-java" % "0.3.1" - ) ++ ( - if (isYarnMode) { - // This kludge is needed for 0.23.x - Seq( - "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm), - "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm), - "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm) - ) - } else { - Seq() - } ) ) ++ assemblySettings ++ extraAssemblySettings From 5054abd41b4bac4b7c8159dc23c7ee15aeb7ef2a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 19 Aug 2013 12:58:02 -0700 Subject: [PATCH 0784/2521] Code review feedback. (added tests for cogroup and substract; added more documentation on MutablePair) --- .../main/scala/spark/PairRDDFunctions.scala | 4 +- .../main/scala/spark/util/MutablePair.scala | 16 +++---- core/src/test/scala/spark/ShuffleSuite.scala | 42 ++++++++++++++++++- 3 files changed, 51 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index f8900d39214..e7d4a7f562c 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -165,7 +165,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) def reducePartition(iter: Iterator[(K, V)]): Iterator[JHashMap[K, V]] = { val map = new JHashMap[K, V] - for ((k, v) <- iter) { + iter.foreach { case (k, v) => val old = map.get(k) map.put(k, if (old == null) v else func(old, v)) } @@ -173,7 +173,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) } def mergeMaps(m1: JHashMap[K, V], m2: JHashMap[K, V]): JHashMap[K, V] = { - for ((k, v) <- m2) { + m2.foreach { case (k, v) => val old = m1.get(k) m1.put(k, if (old == null) v else func(old, v)) } diff --git a/core/src/main/scala/spark/util/MutablePair.scala b/core/src/main/scala/spark/util/MutablePair.scala index 117218bf477..3063806e83e 100644 --- a/core/src/main/scala/spark/util/MutablePair.scala +++ b/core/src/main/scala/spark/util/MutablePair.scala @@ -18,17 +18,19 @@ package spark.util -/** A tuple of 2 elements. - * @param _1 Element 1 of this MutablePair - * @param _2 Element 2 of this MutablePair - */ +/** + * A tuple of 2 elements. This can be used as an alternative to Scala's Tuple2 when we want to + * minimize object allocation. + * + * @param _1 Element 1 of this MutablePair + * @param _2 Element 2 of this MutablePair + */ case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T1, @specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T2] - (var _1: T1,var _2: T2) + (var _1: T1, var _2: T2) extends Product2[T1, T2] { - override def toString = "(" + _1 + "," + _2 + ")" - def canEqual(that: Any): Boolean = that.isInstanceOf[MutablePair[T1, T2]] + override def canEqual(that: Any): Boolean = that.isInstanceOf[MutablePair[T1, T2]] } diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index f1361546a32..8745689c70b 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -22,8 +22,7 @@ import org.scalatest.matchers.ShouldMatchers import spark.SparkContext._ import spark.ShuffleSuite.NonJavaSerializableClass -import spark.rdd.OrderedRDDFunctions -import spark.rdd.ShuffledRDD +import spark.rdd.{SubtractedRDD, CoGroupedRDD, OrderedRDDFunctions, ShuffledRDD} import spark.util.MutablePair @@ -159,6 +158,45 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { results(2) should be (p(3, 33)) results(3) should be (p(100, 100)) } + + test("cogroup using mutable pairs") { + // 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") + def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) + val data1 = Seq(p(1, 1), p(1, 2), p(1, 3), p(2, 1)) + val data2 = Seq(p(1, "11"), p(1, "12"), p(2, "22"), p(3, "3")) + val pairs1: RDD[MutablePair[Int, Int]] = sc.parallelize(data1, 2) + val pairs2: RDD[MutablePair[Int, String]] = sc.parallelize(data2, 2) + val results = new CoGroupedRDD[Int](Seq(pairs1, pairs2), new HashPartitioner(2)).collectAsMap() + + assert(results(1)(0).length === 3) + assert(results(1)(0).contains(1)) + assert(results(1)(0).contains(2)) + assert(results(1)(0).contains(3)) + assert(results(1)(1).length === 2) + assert(results(1)(1).contains("11")) + assert(results(1)(1).contains("12")) + assert(results(2)(0).length === 1) + assert(results(2)(0).contains(1)) + assert(results(2)(1).length === 1) + assert(results(2)(1).contains("22")) + assert(results(3)(0).length === 0) + assert(results(3)(1).contains("3")) + } + + test("subtract mutable pairs") { + // 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") + def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) + val data1 = Seq(p(1, 1), p(1, 2), p(1, 3), p(2, 1), p(3, 33)) + val data2 = Seq(p(1, "11"), p(1, "12"), p(2, "22")) + val pairs1: RDD[MutablePair[Int, Int]] = sc.parallelize(data1, 2) + val pairs2: RDD[MutablePair[Int, String]] = sc.parallelize(data2, 2) + val results = new SubtractedRDD(pairs1, pairs2, new HashPartitioner(2)).collect() + results should have length (1) + // substracted rdd return results as Tuple2 + results(0) should be ((3, 33)) + } } object ShuffleSuite { From 023702c90da7c421e1e339e14878bccebbb26e24 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 19 Aug 2013 13:32:54 -0700 Subject: [PATCH 0785/2521] Updating documentation. --- core/src/main/scala/spark/rdd/IndexedRDD.scala | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index b2a2180b8f1..d19d220ec24 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -35,18 +35,20 @@ import spark.Partitioner._ /** - * And index RDD + * The BlockIndex is the internal map structure used inside the index + * of the IndexedRDD. */ class BlockIndex[@specialized K: ClassManifest] extends JHashMap[K,Int] -//type BlockIndex[@specialized K: ClassManifest] = JHashMap[K,Int] /** - * An IndexedRDD is an RDD[(K,V)] where each K is unique. + * An IndexedRDD[K,V] extends the RDD[(K,V)] by pre-indexing the keys and + * organizing the values to enable faster join operations. + * + * In addition to providing the basic RDD[(K,V)] functionality the IndexedRDD + * exposes an index member which can be used to "key" other IndexedRDDs * - * The IndexedRDD contains an index datastructure that can - * be used to accelerate join and aggregation operations. */ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val index: RDD[ BlockIndex[K] ], From 55c6e73bfbd29358226d7fb6b7f753d1b05cd9bc Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Mon, 19 Aug 2013 17:35:51 -0700 Subject: [PATCH 0786/2521] Wrapping the index in an opaque type. --- .../main/scala/spark/PairRDDFunctions.scala | 2 +- core/src/main/scala/spark/RDD.scala | 4 +- .../src/main/scala/spark/rdd/IndexedRDD.scala | 75 +++++++++++++------ 3 files changed, 55 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 6751da72af7..03f25040a11 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -706,7 +706,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( IndexedRDD(self.partitionBy(partitioner)) - def indexed(existingIndex: RDD[BlockIndex[K]] = null): IndexedRDD[K,V] = + def indexed(existingIndex: RDDIndex[K] = null): IndexedRDD[K,V] = IndexedRDD(self, existingIndex) private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 1c5095fa865..16644715246 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -55,7 +55,7 @@ import spark.rdd.ZippedPartitionsRDD2 import spark.rdd.ZippedPartitionsRDD3 import spark.rdd.ZippedPartitionsRDD4 -import spark.rdd.{IndexedRDD, BlockIndex} +import spark.rdd.{IndexedRDD, RDDIndex} import spark.storage.StorageLevel import spark.util.BoundedPriorityQueue @@ -774,7 +774,7 @@ abstract class RDD[T: ClassManifest]( } - def makeIndex(partitioner: Option[Partitioner] = None): RDD[BlockIndex[T]] = + def makeIndex(partitioner: Option[Partitioner] = None): RDDIndex[T] = IndexedRDD.makeIndex(this, partitioner) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index d19d220ec24..ab1e460aeb6 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -31,6 +31,10 @@ import spark.rdd._ import spark.SparkContext._ import spark.Partitioner._ +import spark.storage.StorageLevel + + + @@ -41,6 +45,20 @@ import spark.Partitioner._ class BlockIndex[@specialized K: ClassManifest] extends JHashMap[K,Int] +/** + * The RDDIndex is an opaque type used to represent the organization + * of values in an RDD + */ +class RDDIndex[@specialized K: ClassManifest](private[spark] val rdd: RDD[BlockIndex[K]]) { + def persist(newLevel: StorageLevel): RDDIndex[K] = { + rdd.persist(newLevel) + return this + } +} + + + + /** * An IndexedRDD[K,V] extends the RDD[(K,V)] by pre-indexing the keys and @@ -51,30 +69,41 @@ class BlockIndex[@specialized K: ClassManifest] extends JHashMap[K,Int] * */ class IndexedRDD[K: ClassManifest, V: ClassManifest]( - val index: RDD[ BlockIndex[K] ], + val index: RDDIndex[K], val valuesRDD: RDD[ Seq[Seq[V]] ]) - extends RDD[(K, V)](index.context, - List(new OneToOneDependency(index), new OneToOneDependency(valuesRDD)) ) { + extends RDD[(K, V)](index.rdd.context, + List(new OneToOneDependency(index.rdd), new OneToOneDependency(valuesRDD)) ) { /** - * An internal representation of the maps and block managers + * An internal representation which joins the block indices with the values */ - protected val tuples = new ZippedRDD(index.context, index, valuesRDD) + protected val tuples = new ZippedRDD(index.rdd.context, index.rdd, valuesRDD) - override val partitioner = index.partitioner + /** + * The partitioner is defined by the index + */ + override val partitioner = index.rdd.partitioner + /** + * The actual partitions are defined by the tuples. + */ override def getPartitions: Array[Partition] = tuples.getPartitions - + /** + * The preferred locations are computed based on the preferred locations of the tuples. + */ override def getPreferredLocations(s: Partition): Seq[String] = tuples.getPreferredLocations(s) - override def cache: IndexedRDD[K,V] = { - index.cache - valuesRDD.cache + /** + * Caching an IndexedRDD causes the index and values to be cached separately. + */ + override def persist(newLevel: StorageLevel): RDD[(K,V)] = { + index.persist(newLevel) + valuesRDD.persist(newLevel) return this } @@ -90,7 +119,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * this also retains the original RDD's partitioning. */ def mapValues[U: ClassManifest](f: V => U): IndexedRDD[K, U] = { - val cleanF = index.context.clean(f) + val cleanF = index.rdd.context.clean(f) val newValues = valuesRDD.mapPartitions(_.map(values => values.map{ case null => null case row => row.map(x => f(x)) @@ -104,7 +133,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( * keys; this also retains the original RDD's partitioning. */ def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): IndexedRDD[K,U] = { - val cleanF = index.context.clean(f) + val cleanF = index.rdd.context.clean(f) val newValues = valuesRDD.mapPartitions(_.map(values => values.map{ case null => null case row => row.flatMap(x => f(x)) @@ -211,12 +240,12 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( }) new IndexedRDD[K, (Seq[V], Seq[W])](index, newValues) } - case other: IndexedRDD[_, _] if other.index.partitioner == index.partitioner => { + case other: IndexedRDD[_, _] if other.index.rdd.partitioner == index.rdd.partitioner => { // If both RDDs are indexed using different indices but with the same partitioners // then we we need to first merge the indicies and then use the merged index to // merge the values. val newIndex = - index.zipPartitions(other.index)( + index.rdd.zipPartitions(other.index.rdd)( (thisIter, otherIter) => { val thisIndex = thisIter.next() assert(!thisIter.hasNext()) @@ -264,11 +293,11 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } List(newValues.toSeq).iterator }) - new IndexedRDD(newIndex, newValues) + new IndexedRDD(new RDDIndex(newIndex), newValues) } case _ => { // Get the partitioner from the index - val partitioner = index.partitioner match { + val partitioner = index.rdd.partitioner match { case Some(p) => p case None => throw new SparkException("An index must have a partitioner.") } @@ -328,7 +357,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - new IndexedRDD[K, (Seq[V], Seq[W])](newIndex, newValues) + new IndexedRDD[K, (Seq[V], Seq[W])](new RDDIndex(newIndex), newValues) } } } @@ -440,7 +469,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( object IndexedRDD { def apply[K: ClassManifest, V: ClassManifest]( tbl: RDD[(K,V)], - existingIndex: RDD[BlockIndex[K]] = null ): IndexedRDD[K, V] = { + existingIndex: RDDIndex[K] = null ): IndexedRDD[K, V] = { if (existingIndex == null) { // Shuffle the table (if necessary) @@ -466,10 +495,10 @@ object IndexedRDD { // extract the index and the values val index = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) val values = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - new IndexedRDD[K,V](index, values) + new IndexedRDD[K,V](new RDDIndex(index), values) } else { val index = existingIndex - val partitioner = index.partitioner match { + val partitioner = index.rdd.partitioner match { case Some(p) => p case None => throw new SparkException("An index must have a partitioner.") } @@ -483,7 +512,7 @@ object IndexedRDD { } // Use the index to build the new values table - val values = index.zipPartitions(shuffledTbl)( + val values = index.rdd.zipPartitions(shuffledTbl)( (indexIter, tblIter) => { // There is only one map val index = indexIter.next() @@ -511,7 +540,7 @@ object IndexedRDD { * Construct and index of the unique values in a given RDD. */ def makeIndex[K: ClassManifest](keys: RDD[K], - partitioner: Option[Partitioner] = None): RDD[BlockIndex[K]] = { + partitioner: Option[Partitioner] = None): RDDIndex[K] = { // Ugly hack :-(. In order to partition the keys they must have values. @@ -539,7 +568,7 @@ object IndexedRDD { } List(indexMap).iterator }, true).cache - index + new RDDIndex(index) } } From 498a26189b197bdaf4be47e6a8baca7b97fe9064 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 19 Aug 2013 18:17:49 -0700 Subject: [PATCH 0787/2521] Small fixes to web UI: - Use SPARK_PUBLIC_DNS environment variable if set (for EC2) - Use a non-ephemeral port (3030 instead of 33000) by default - Updated test to use non-ephemeral port too --- core/src/main/scala/spark/ui/SparkUI.scala | 10 +++++----- core/src/test/scala/spark/ui/UISuite.scala | 2 +- ec2/spark_ec2.py | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 1fd5a0989e1..23ded44ba38 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -30,7 +30,7 @@ import spark.ui.JettyUtils._ /** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { - val host = Utils.localHostName() + val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(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 @@ -58,9 +58,9 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { server = Some(srv) boundPort = Some(usedPort) } catch { - case e: Exception => - logError("Failed to create Spark JettyUtils", e) - System.exit(1) + case e: Exception => + logError("Failed to create Spark JettyUtils", e) + System.exit(1) } } @@ -82,6 +82,6 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { } private[spark] object SparkUI { - val DEFAULT_PORT = "33000" + val DEFAULT_PORT = "3030" val STATIC_RESOURCE_DIR = "spark/ui/static" } diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index 56c1fed6ad0..41cb0e0a358 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -24,7 +24,7 @@ import org.eclipse.jetty.server.Server class UISuite extends FunSuite { test("jetty port increases under contention") { - val startPort = 33333 + val startPort = 3030 val server = new Server(startPort) server.start() val (jettyServer1, boundPort1) = JettyUtils.startJettyServer("localhost", startPort, Seq()) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 740ec08542f..4087c3bc2bf 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -178,7 +178,7 @@ def launch_cluster(conn, opts, cluster_name): master_group.authorize('tcp', 50030, 50030, '0.0.0.0/0') master_group.authorize('tcp', 50070, 50070, '0.0.0.0/0') master_group.authorize('tcp', 60070, 60070, '0.0.0.0/0') - master_group.authorize('tcp', 33000, 33010, '0.0.0.0/0') + master_group.authorize('tcp', 3030, 3030, '0.0.0.0/0') if opts.cluster_type == "mesos": master_group.authorize('tcp', 38090, 38090, '0.0.0.0/0') if opts.ganglia: From 793a722f8e14552b8d36f46cca39d336dc2df9dd Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 19 Aug 2013 18:35:47 -0700 Subject: [PATCH 0788/2521] Allow some wiggle room in UISuite port test and in EC2 ports --- core/src/test/scala/spark/ui/UISuite.scala | 5 +++-- ec2/spark_ec2.py | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/spark/ui/UISuite.scala index 41cb0e0a358..735a794396f 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/spark/ui/UISuite.scala @@ -30,8 +30,9 @@ class UISuite extends FunSuite { val (jettyServer1, boundPort1) = JettyUtils.startJettyServer("localhost", startPort, Seq()) val (jettyServer2, boundPort2) = JettyUtils.startJettyServer("localhost", startPort, Seq()) - assert(boundPort1 === startPort + 1) - assert(boundPort2 === startPort + 2) + // Allow some wiggle room in case ports on the machine are under contention + assert(boundPort1 > startPort && boundPort1 < startPort + 10) + assert(boundPort2 > boundPort1 && boundPort2 < boundPort1 + 10) } test("jetty binds to port 0 correctly") { diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 4087c3bc2bf..6127c10ced9 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -178,7 +178,7 @@ def launch_cluster(conn, opts, cluster_name): master_group.authorize('tcp', 50030, 50030, '0.0.0.0/0') master_group.authorize('tcp', 50070, 50070, '0.0.0.0/0') master_group.authorize('tcp', 60070, 60070, '0.0.0.0/0') - master_group.authorize('tcp', 3030, 3030, '0.0.0.0/0') + master_group.authorize('tcp', 3030, 3035, '0.0.0.0/0') if opts.cluster_type == "mesos": master_group.authorize('tcp', 38090, 38090, '0.0.0.0/0') if opts.ganglia: From ad18410427190572f90754624469a7e806c78971 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 16 Aug 2013 12:14:52 -0700 Subject: [PATCH 0789/2521] Renamed 'priority' to 'jobId' and assorted minor changes --- .../scala/spark/scheduler/ActiveJob.scala | 2 +- .../scala/spark/scheduler/DAGScheduler.scala | 84 +++++++++---------- .../spark/scheduler/DAGSchedulerSource.scala | 2 +- .../scala/spark/scheduler/JobLogger.scala | 24 +++--- .../main/scala/spark/scheduler/Stage.scala | 7 +- 5 files changed, 60 insertions(+), 59 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/ActiveJob.scala b/core/src/main/scala/spark/scheduler/ActiveJob.scala index 71cc94edb68..fecc3e9648a 100644 --- a/core/src/main/scala/spark/scheduler/ActiveJob.scala +++ b/core/src/main/scala/spark/scheduler/ActiveJob.scala @@ -25,7 +25,7 @@ import java.util.Properties * Tracks information about an active job in the DAGScheduler. */ private[spark] class ActiveJob( - val runId: Int, + val jobId: Int, val finalStage: Stage, val func: (TaskContext, Iterator[_]) => _, val partitions: Array[Int], diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 35b31f45a7c..7823d0c8cf6 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -104,11 +104,11 @@ class DAGScheduler( private val eventQueue = new LinkedBlockingQueue[DAGSchedulerEvent] - val nextRunId = new AtomicInteger(0) + val nextJobId = new AtomicInteger(0) val nextStageId = new AtomicInteger(0) - val idToStage = new TimeStampedHashMap[Int, Stage] + val stageIdToStage = new TimeStampedHashMap[Int, Stage] val shuffleToMapStage = new TimeStampedHashMap[Int, Stage] @@ -171,14 +171,14 @@ class DAGScheduler( /** * Get or create a shuffle map stage for the given shuffle dependency's map side. - * 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). + * The jobId value passed in will be used if the stage doesn't already exist with + * a lower jobId (jobId always increases across jobs.) */ - private def getShuffleMapStage(shuffleDep: ShuffleDependency[_,_], priority: Int): Stage = { + private def getShuffleMapStage(shuffleDep: ShuffleDependency[_,_], jobId: Int): 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), jobId) shuffleToMapStage(shuffleDep.shuffleId) = stage stage } @@ -186,13 +186,13 @@ class DAGScheduler( /** * Create a Stage for the given RDD, either as a shuffle map stage (for a ShuffleDependency) or - * as a result stage for the final RDD used directly in an action. The stage will also be given - * the provided priority. + * as a result stage for the final RDD used directly in an action. The stage will also be + * associated with the provided jobId. */ private def newStage( rdd: RDD[_], shuffleDep: Option[ShuffleDependency[_,_]], - priority: Int, + jobId: Int, callSite: Option[String] = None) : Stage = { @@ -203,17 +203,17 @@ 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, callSite) - idToStage(id) = stage + val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, jobId), jobId, callSite) + stageIdToStage(id) = stage stageToInfos(stage) = StageInfo(stage) stage } /** * 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. + * provided jobId if they haven't already been created with a lower jobId. */ - private def getParentStages(rdd: RDD[_], priority: Int): List[Stage] = { + private def getParentStages(rdd: RDD[_], jobId: Int): List[Stage] = { val parents = new HashSet[Stage] val visited = new HashSet[RDD[_]] def visit(r: RDD[_]) { @@ -224,7 +224,7 @@ class DAGScheduler( for (dep <- r.dependencies) { dep match { case shufDep: ShuffleDependency[_,_] => - parents += getShuffleMapStage(shufDep, priority) + parents += getShuffleMapStage(shufDep, jobId) case _ => visit(dep.rdd) } @@ -245,7 +245,7 @@ class DAGScheduler( for (dep <- rdd.dependencies) { dep match { case shufDep: ShuffleDependency[_,_] => - val mapStage = getShuffleMapStage(shufDep, stage.priority) + val mapStage = getShuffleMapStage(shufDep, stage.jobId) if (!mapStage.isAvailable) { missing += mapStage } @@ -282,7 +282,7 @@ class DAGScheduler( val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties) - return (toSubmit, waiter) + (toSubmit, waiter) } def runJob[T, U: ClassManifest]( @@ -329,8 +329,8 @@ class DAGScheduler( 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, properties)) - return listener.awaitResult() // Will throw an exception if the job fails + eventQueue.put(JobSubmitted(rdd, func2, partitions, allowLocal = false, callSite, listener, properties)) + listener.awaitResult() // Will throw an exception if the job fails } /** @@ -340,11 +340,11 @@ class DAGScheduler( private[scheduler] def processEvent(event: DAGSchedulerEvent): Boolean = { event match { case JobSubmitted(finalRDD, func, partitions, allowLocal, callSite, listener, properties) => - val runId = nextRunId.getAndIncrement() - val finalStage = newStage(finalRDD, None, runId, Some(callSite)) - val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener, properties) + val jobId = nextJobId.getAndIncrement() + val finalStage = newStage(finalRDD, None, jobId, Some(callSite)) + val job = new ActiveJob(jobId, finalStage, func, partitions, callSite, listener, properties) clearCacheLocs() - logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length + + logInfo("Got job " + job.jobId + " (" + callSite + ") with " + partitions.length + " output partitions (allowLocal=" + allowLocal + ")") logInfo("Final stage: " + finalStage + " (" + finalStage.name + ")") logInfo("Parents of final stage: " + finalStage.parents) @@ -354,7 +354,7 @@ class DAGScheduler( runLocally(job) } else { listenerBus.post(SparkListenerJobStart(job, properties)) - idToActiveJob(runId) = job + idToActiveJob(jobId) = job activeJobs += job resultStageToJob(finalStage) = job submitStage(finalStage) @@ -375,7 +375,7 @@ class DAGScheduler( handleTaskCompletion(completion) case TaskSetFailed(taskSet, reason) => - abortStage(idToStage(taskSet.stageId), reason) + abortStage(stageIdToStage(taskSet.stageId), reason) case StopDAGScheduler => // Cancel any active jobs @@ -386,7 +386,7 @@ class DAGScheduler( } return true } - return false + false } /** @@ -398,7 +398,7 @@ class DAGScheduler( clearCacheLocs() val failed2 = failed.toArray failed.clear() - for (stage <- failed2.sortBy(_.priority)) { + for (stage <- failed2.sortBy(_.jobId)) { submitStage(stage) } } @@ -416,7 +416,7 @@ class DAGScheduler( logTrace("failed: " + failed) val waiting2 = waiting.toArray waiting.clear() - for (stage <- waiting2.sortBy(_.priority)) { + for (stage <- waiting2.sortBy(_.jobId)) { submitStage(stage) } } @@ -463,7 +463,7 @@ class DAGScheduler( */ protected def runLocally(job: ActiveJob) { logInfo("Computing the requested partition locally") - new Thread("Local computation of job " + job.runId) { + new Thread("Local computation of job " + job.jobId) { override def run() { runLocallyWithinThread(job) } @@ -531,7 +531,7 @@ class DAGScheduler( } // must be run listener before possible NotSerializableException // should be "StageSubmitted" first and then "JobEnded" - val properties = idToActiveJob(stage.priority).properties + val properties = idToActiveJob(stage.jobId).properties listenerBus.post(SparkListenerStageSubmitted(stage, tasks.size, properties)) if (tasks.size > 0) { @@ -552,7 +552,7 @@ class DAGScheduler( myPending ++= tasks logDebug("New pending tasks: " + myPending) taskSched.submitTasks( - new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.priority, properties)) + new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties)) if (!stage.submissionTime.isDefined) { stage.submissionTime = Some(System.currentTimeMillis()) } @@ -569,7 +569,7 @@ class DAGScheduler( */ private def handleTaskCompletion(event: CompletionEvent) { val task = event.task - val stage = idToStage(task.stageId) + val stage = stageIdToStage(task.stageId) def markStageAsFinished(stage: Stage) = { val serviceTime = stage.submissionTime match { @@ -598,7 +598,7 @@ class DAGScheduler( job.numFinished += 1 // If the whole job has finished, remove it if (job.numFinished == job.numPartitions) { - idToActiveJob -= stage.priority + idToActiveJob -= stage.jobId activeJobs -= job resultStageToJob -= stage markStageAsFinished(stage) @@ -635,7 +635,7 @@ class DAGScheduler( mapOutputTracker.registerMapOutputs( stage.shuffleDep.get.shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, - true) + changeGeneration = true) } clearCacheLocs() if (stage.outputLocs.count(_ == Nil) != 0) { @@ -669,7 +669,7 @@ class DAGScheduler( case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => // Mark the stage that the reducer was in as unrunnable - val failedStage = idToStage(task.stageId) + val failedStage = stageIdToStage(task.stageId) running -= failedStage failed += failedStage // TODO: Cancel running tasks in the stage @@ -697,7 +697,7 @@ class DAGScheduler( case other => // Unrecognized failure - abort all jobs depending on this stage - abortStage(idToStage(task.stageId), task + " failed: " + other) + abortStage(stageIdToStage(task.stageId), task + " failed: " + other) } } @@ -718,7 +718,7 @@ class DAGScheduler( for ((shuffleId, stage) <- shuffleToMapStage) { stage.removeOutputsOnExecutor(execId) val locs = stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray - mapOutputTracker.registerMapOutputs(shuffleId, locs, true) + mapOutputTracker.registerMapOutputs(shuffleId, locs, changeGeneration = true) } if (shuffleToMapStage.isEmpty) { mapOutputTracker.incrementEpoch() @@ -750,7 +750,7 @@ class DAGScheduler( val error = new SparkException("Job failed: " + reason) job.listener.jobFailed(error) listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(failedStage)))) - idToActiveJob -= resultStage.priority + idToActiveJob -= resultStage.jobId activeJobs -= job resultStageToJob -= resultStage } @@ -774,7 +774,7 @@ class DAGScheduler( for (dep <- rdd.dependencies) { dep match { case shufDep: ShuffleDependency[_,_] => - val mapStage = getShuffleMapStage(shufDep, stage.priority) + val mapStage = getShuffleMapStage(shufDep, stage.jobId) if (!mapStage.isAvailable) { visitedStages += mapStage visit(mapStage.rdd) @@ -812,13 +812,13 @@ class DAGScheduler( } case _ => }) - return Nil + Nil } private def cleanup(cleanupTime: Long) { - var sizeBefore = idToStage.size - idToStage.clearOldValues(cleanupTime) - logInfo("idToStage " + sizeBefore + " --> " + idToStage.size) + var sizeBefore = stageIdToStage.size + stageIdToStage.clearOldValues(cleanupTime) + logInfo("stageIdToStage " + sizeBefore + " --> " + stageIdToStage.size) sizeBefore = shuffleToMapStage.size shuffleToMapStage.clearOldValues(cleanupTime) diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala index 87d27cc70d7..98c4fb7e59a 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala @@ -21,7 +21,7 @@ private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends }) metricRegistry.register(MetricRegistry.name("job", "allJobs", "number"), new Gauge[Int] { - override def getValue: Int = dagScheduler.nextRunId.get() + override def getValue: Int = dagScheduler.nextJobId.get() }) metricRegistry.register(MetricRegistry.name("job", "activeJobs", "number"), new Gauge[Int] { diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/spark/scheduler/JobLogger.scala index 7194fcaa49a..1bc9fabdffd 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/spark/scheduler/JobLogger.scala @@ -102,7 +102,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { stageIDToJobID.get(stageID).foreach(jobID => jobLogInfo(jobID, info, withTime)) protected def buildJobDep(jobID: Int, stage: Stage) { - if (stage.priority == jobID) { + if (stage.jobId == jobID) { jobIDToStages.get(jobID) match { case Some(stageList) => stageList += stage case None => val stageList = new ListBuffer[Stage] @@ -178,12 +178,12 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { }else{ stageInfo = "STAGE_ID=" + stage.id + " RESULT_STAGE" } - if (stage.priority == jobID) { + if (stage.jobId == 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) + jobLogInfo(jobID, indentString(indent) + stageInfo + " JOB_ID=" + stage.jobId, false) } // Record task metrics into job log files @@ -260,7 +260,7 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { override def onJobEnd(jobEnd: SparkListenerJobEnd) { val job = jobEnd.job - var info = "JOB_ID=" + job.runId + var info = "JOB_ID=" + job.jobId jobEnd.jobResult match { case JobSucceeded => info += " STATUS=SUCCESS" case JobFailed(exception, _) => @@ -268,8 +268,8 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { exception.getMessage.split("\\s+").foreach(info += _ + "_") case _ => } - jobLogInfo(job.runId, info.substring(0, info.length - 1).toUpperCase) - closeLogWriter(job.runId) + jobLogInfo(job.jobId, info.substring(0, info.length - 1).toUpperCase) + closeLogWriter(job.jobId) } protected def recordJobProperties(jobID: Int, properties: Properties) { @@ -282,11 +282,11 @@ class JobLogger(val logDirName: String) extends SparkListener with Logging { override def onJobStart(jobStart: SparkListenerJobStart) { val job = jobStart.job val properties = jobStart.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") + createLogWriter(job.jobId) + recordJobProperties(job.jobId, properties) + buildJobDep(job.jobId, job.finalStage) + recordStageDep(job.jobId) + recordStageDepGraph(job.jobId, job.finalStage) + jobLogInfo(job.jobId, "JOB_ID=" + job.jobId + " STATUS=STARTED") } } diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala index 5428daeb947..c599c00ac44 100644 --- a/core/src/main/scala/spark/scheduler/Stage.scala +++ b/core/src/main/scala/spark/scheduler/Stage.scala @@ -33,15 +33,16 @@ import spark.storage.BlockManagerId * initiated a job (e.g. count(), save(), etc). For shuffle map stages, we also track the nodes * that each output partition is on. * - * Each Stage also has a priority, which is (by default) based on the job it was submitted in. - * This allows Stages from earlier jobs to be computed first or recovered faster on failure. + * Each Stage also has a jobId, identifying the job that first submitted the stage. When FIFO + * scheduling is used, this allows Stages from earlier jobs to be computed first or recovered + * faster on failure. */ private[spark] class Stage( val id: Int, val rdd: RDD[_], val shuffleDep: Option[ShuffleDependency[_,_]], // Output shuffle if stage is a map stage val parents: List[Stage], - val priority: Int, + val jobId: Int, callSite: Option[String]) extends Logging { From 1630fbf838fff101e36324934a165f95fb324482 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 20 Aug 2013 00:17:16 -0700 Subject: [PATCH 0790/2521] changeGeneration --> changeEpoch renaming --- core/src/main/scala/spark/scheduler/DAGScheduler.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 7823d0c8cf6..9402f18a0f6 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -635,7 +635,7 @@ class DAGScheduler( mapOutputTracker.registerMapOutputs( stage.shuffleDep.get.shuffleId, stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray, - changeGeneration = true) + changeEpoch = true) } clearCacheLocs() if (stage.outputLocs.count(_ == Nil) != 0) { @@ -718,7 +718,7 @@ class DAGScheduler( for ((shuffleId, stage) <- shuffleToMapStage) { stage.removeOutputsOnExecutor(execId) val locs = stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray - mapOutputTracker.registerMapOutputs(shuffleId, locs, changeGeneration = true) + mapOutputTracker.registerMapOutputs(shuffleId, locs, changeEpoch = true) } if (shuffleToMapStage.isEmpty) { mapOutputTracker.incrementEpoch() From 07e5c8b69560e071d521bfeab1d86b5f68d5bb57 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 20 Aug 2013 15:49:52 -0700 Subject: [PATCH 0791/2521] Set default Hadoop version to 1 --- ec2/spark_ec2.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 8080cc41720..182cbbbb7c2 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -73,8 +73,8 @@ def parse_args(): parser.add_option("--spark-git-repo", default="https://github.com/mesos/spark", help="Github repo from which to checkout supplied commit hash") - parser.add_option("--hadoop-major-version", default="2", - help="Major version of Hadoop (default: 2)") + parser.add_option("--hadoop-major-version", default="1", + help="Major version of Hadoop (default: 1)") parser.add_option("-D", metavar="[ADDRESS:]PORT", dest="proxy_port", help="Use SSH dynamic port forwarding to create a SOCKS proxy at " + From 1ede102ba5863f6cee27437b0adbc4d54cedffb3 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Tue, 13 Aug 2013 20:46:22 -0700 Subject: [PATCH 0792/2521] load balancing coalescer --- .../main/scala/spark/rdd/CoalescedRDD.scala | 207 +++++++++++++++++- core/src/test/scala/spark/RDDSuite.scala | 22 ++ 2 files changed, 218 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 2b5bf18541c..09ae9a8fa64 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -19,11 +19,13 @@ package spark.rdd import spark.{Dependency, OneToOneDependency, NarrowDependency, RDD, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} +import scala.collection.mutable private[spark] case class CoalescedRDDPartition( index: Int, @transient rdd: RDD[_], - parentsIndices: Array[Int] + parentsIndices: Array[Int], + prefLoc: String = "" ) extends Partition { var parents: Seq[Partition] = parentsIndices.map(rdd.partitions(_)) @@ -33,6 +35,8 @@ private[spark] case class CoalescedRDDPartition( parents = parentsIndices.map(rdd.partitions(_)) oos.defaultWriteObject() } + + def getPreferredLocation = prefLoc } /** @@ -45,20 +49,24 @@ private[spark] case class CoalescedRDDPartition( */ class CoalescedRDD[T: ClassManifest]( @transient var prev: RDD[T], - maxPartitions: Int) + maxPartitions: Int, + balanceSlack: Double = 0.20 ) extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies override def getPartitions: Array[Partition] = { - val prevSplits = prev.partitions - if (prevSplits.length < maxPartitions) { - prevSplits.map(_.index).map{idx => new CoalescedRDDPartition(idx, prev, Array(idx)) } - } else { - (0 until maxPartitions).map { i => - val rangeStart = ((i.toLong * prevSplits.length) / maxPartitions).toInt - val rangeEnd = (((i.toLong + 1) * prevSplits.length) / maxPartitions).toInt - new CoalescedRDDPartition(i, prev, (rangeStart until rangeEnd).toArray) - }.toArray + val res = mutable.ArrayBuffer[CoalescedRDDPartition]() + val targetLen = math.min(prev.partitions.length, maxPartitions) + val packer = new PartitionCoalescer(maxPartitions, prev, balanceSlack) + + packer.setupGroups(targetLen) // setup the groups (bins) and preferred locations + packer.throwBalls() // assign partitions (balls) to each group (bins) + + for ((pg, i) <- packer.groupArr.zipWithIndex) { + val ids = pg.list.map(_.index).toArray + res += new CoalescedRDDPartition(i, prev, ids, pg.prefLoc) } + + res.toArray } override def compute(split: Partition, context: TaskContext): Iterator[T] = { @@ -79,3 +87,180 @@ class CoalescedRDD[T: ClassManifest]( prev = null } } + + +class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { + + // this class just keeps iterating and rotating infinitely over the partitions of the RDD + // next() returns the next preferred machine that a partition is replicated on + // the rotator first goes through the first replica copy of each partition, then second, then third + private class RotateLocations(prev: RDD[_]) extends Iterator[String] { + + private var it: Iterator[String] = resetIterator() + override val isEmpty = !it.hasNext + + // initializes/resets to start iterating from the beginning + private def resetIterator() = { + val i1 = prev.partitions.view.map( (p: Partition) => + { if (prev.preferredLocations(p).length > 0) Some(prev.preferredLocations(p)(0)) else None } ) + val i2 = prev.partitions.view.map( (p: Partition) => + { if (prev.preferredLocations(p).length > 1) Some(prev.preferredLocations(p)(1)) else None } ) + val i3 = prev.partitions.view.map( (p: Partition) => + { if (prev.preferredLocations(p).length > 2) Some(prev.preferredLocations(p)(2)) else None } ) + val res = List(i1,i2,i3) + res.view.flatMap(x => x).flatten.iterator // fuses the 3 iterators (1st replica, 2nd, 3rd) into one iterator + } + + // hasNext() is false iff there are no preferredLocations for any of the partitions of the RDD + def hasNext(): Boolean = !isEmpty + + // return the next preferredLocation of some partition of the RDD + def next(): String = { + if (it.hasNext) + it.next() + else { + it = resetIterator() // ran out of preferred locations, reset and rotate to the beginning + it.next() + } + } + } + + case class PartitionGroup(prefLoc: String = "") { + var list = mutable.MutableList[Partition]() + + def size = list.size + + // returns number of partitions that got locality in this group + def local = { + var loc: Int = 0 + list.foreach(p => if (prev.preferredLocations(p).contains(prefLoc)) loc += 1) + loc + } + + override def toString(): String = { + val localityStr = if (size == 0) "0" else (local*100. / size).toInt.toString + "PartitionGroup(\"" + prefLoc + "\") size: " + size + " locality: " + localityStr +"% \n" + // list.map("\t\t" + _.toString).mkString("\n") + "\n" + } + } + + def compare(o1: PartitionGroup, o2: PartitionGroup): Boolean = o1.size < o2.size + def compare(o1: Option[PartitionGroup], o2: Option[PartitionGroup]): Boolean = + if (o1 == None) false else if (o2 == None) true else compare(o1.get, o2.get) + + val rnd = new scala.util.Random(7919) // keep this class deterministic + + // each element of groupArr represents one coalesced partition + val groupArr = mutable.ArrayBuffer[PartitionGroup]() + + // hash used to check whether some machine is already in groupArr + val groupHash = mutable.Map[String, mutable.MutableList[PartitionGroup]]() + + // determines the tradeoff between load-balancing the partitions sizes and their locality + // e.g. balanceSlack=0.10 means that it allows up to 10% imbalance in favor of locality + val slack = (balanceSlack * maxPartitions).toInt + + private var noLocality = true // if true if no preferredLocations exists for parent RDD + + /** + * Sorts and gets the least element of the list associated with key in groupHash + * The returned PartitionGroup is the least loaded of all groups that represent the machine "key" + * @param key string representing a partitioned group on preferred machine key + * @return Option of PartitionGroup that has least elements for key + */ + private def getLeastGroupHash(key: String): Option[PartitionGroup] = { + groupHash.get(key).map(_.sortWith(compare).head) + } + + /** + * Initializes targetLen partition groups and assigns a preferredLocation + * This uses coupon collector to estimate how many preferredLocations it must rotate through until it has seen + * most of the preferred locations (2 * n log(n)) + * @param targetLen + */ + def setupGroups(targetLen: Int) { + val rotIt = new RotateLocations(prev) + + // deal with empty rotator case, just create targetLen partition groups with no preferred location + if (!rotIt.hasNext()) { + (1 to targetLen).foreach(x => groupArr += PartitionGroup()) + return + } + + noLocality = false + + // number of iterations needed to be certain that we've seen most preferred locations + val expectedCoupons2 = 2 * (math.log(targetLen)*targetLen + targetLen + 0.5).toInt + var numCreated = 0 + var tries = 0 + + // rotate through until either targetLen unique/distinct preferred locations have been created OR + // we've rotated expectedCoupons2, in which case we have likely seen all preferred locations, i.e. + // likely targetLen >> number of preferred locations (more buckets than there are machines) + while (numCreated < targetLen && tries < expectedCoupons2) { + tries += 1 + val nxt = rotIt.next() + if (!groupHash.contains(nxt)) { + val pgroup = PartitionGroup(nxt) + groupArr += pgroup + groupHash += (nxt -> (mutable.MutableList(pgroup))) // use list in case we have multiple groups for same machine + numCreated += 1 + } + } + + while (numCreated < targetLen) { // if we don't have enough partition groups, just create duplicates + val nxt = rotIt.next() + val pgroup = PartitionGroup(nxt) + groupArr += pgroup + groupHash.get(nxt).get += pgroup + numCreated += 1 + } + } + + /** + * Takes a parent RDD partition and decides which of the partition groups to put it in + * Takes locality into account, but also uses power of 2 choices to load balance + * It strikes a balance between the two use the balanceSlack variable + * @param p partition (ball to be thrown) + * @return partition group (bin to be put in) + */ + def pickBin(p: Partition): PartitionGroup = { + val pref = prev.preferredLocations(p).map(getLeastGroupHash(_)).sortWith(compare) // least loaded bin of replicas + val prefPart = if (pref == Nil) None else pref.head + + val r1 = rnd.nextInt(groupArr.size) + val r2 = rnd.nextInt(groupArr.size) + val minPowerOfTwo = if (groupArr(r1).size < groupArr(r2).size) groupArr(r1) else groupArr(r2) // power of 2 + if (prefPart== None) // if no preferred locations, just use basic power of two + return minPowerOfTwo + + val prefPartActual = prefPart.get + + if (minPowerOfTwo.size + slack <= prefPartActual.size) // more imbalance than the slack allows + return minPowerOfTwo // prefer balance over locality + else { + return prefPartActual // prefer locality over balance + } + } + + def throwBalls() { + if (noLocality) { // no preferredLocations in parent RDD, no randomization needed + if (maxPartitions > groupArr.size) { // just return prev.partitions + for ((p,i) <- prev.partitions.zipWithIndex) { + groupArr(i).list += p + } + } else { // old code, just splits them grouping partitions that are next to each other in the array + (0 until maxPartitions).foreach { i => + val rangeStart = ((i.toLong * prev.partitions.length) / maxPartitions).toInt + val rangeEnd = (((i.toLong + 1) * prev.partitions.length) / maxPartitions).toInt + (rangeStart until rangeEnd).foreach{ j => groupArr(i).list += prev.partitions(j) } + } + } + } else { + for (p <- prev.partitions) { // throw every partition (ball) into a partition group (bin) + pickBin(p).list += p + } + } + } + +} diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 75778de1cc9..881bdedfe53 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -173,6 +173,28 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(coalesced5.dependencies.head.rdd.dependencies.head.rdd.asInstanceOf[ShuffledRDD[_, _, _]] != null) } + test("cogrouped RDDs with locality") { + // RDD with locality preferences spread (non-randomly) over 6 machines, m0 through m5 + val data = sc.makeRDD((1 to 9).map( i => (i, (i to (i+2)).map{ j => "m" + (j%6)} ))) + val coalesced1 = data.coalesce(3) + assert(coalesced1.collect().toList.sorted === (1 to 9).toList) // no data lost (NB: order might reshuffle) + + val splits = coalesced1.glom().collect().map(_.toList).toList + assert(splits.length === 3) // ensure it indeed created 3 partitions + + assert(splits.foldLeft(true)( (x,y) => if (!x) false else y.length >= 2) === true) // descent balance (2+ per bin) + + val prefs = List(List("m1","m2","m3"), List("m4","m5","m6")) + val data2 = sc.makeRDD((1 to 100).map( i => (i, prefs(i % 2) ))) // alternate machine prefs + val coalesced2 = data2.coalesce(10) + val splits2 = coalesced2.glom().collect().map(_.toList).toList + + // this gives a list of pairs, each pair is of the form (even,odd), where even is the number of even elements... + val list = splits2.map( ls => ls.foldLeft((0,0))( (x,y) => if (y % 2 == 0) (x._1+1,x._2) else (x._1,x._2+1)) ) + val maxes = list.map( { case (a,b) => if (a>b) a else b } ) // get the maxs, this represents the locality + maxes.foreach( locality => assert( locality > 7) ) // at least 70% locality in each partition + + } test("zipped RDDs") { val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) From 66edf854aa585d23e47fc0bfb7fdd4e23c0ea592 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Wed, 14 Aug 2013 11:15:39 -0700 Subject: [PATCH 0793/2521] Bug, should compute slack wrt parent partition size, not number of bins --- core/src/main/scala/spark/rdd/CoalescedRDD.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 09ae9a8fa64..f999e9b0ec5 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -50,7 +50,7 @@ private[spark] case class CoalescedRDDPartition( class CoalescedRDD[T: ClassManifest]( @transient var prev: RDD[T], maxPartitions: Int, - balanceSlack: Double = 0.20 ) + balanceSlack: Double = 0.10 ) extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies override def getPartitions: Array[Partition] = { @@ -158,7 +158,7 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) // determines the tradeoff between load-balancing the partitions sizes and their locality // e.g. balanceSlack=0.10 means that it allows up to 10% imbalance in favor of locality - val slack = (balanceSlack * maxPartitions).toInt + val slack = (balanceSlack * prev.partitions.size).toInt private var noLocality = true // if true if no preferredLocations exists for parent RDD From 7a2a33e32dede41937570ec77cf1dfad070e963f Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Wed, 14 Aug 2013 19:40:24 -0700 Subject: [PATCH 0794/2521] Large scale load and locality tests for the coalesced partitions added --- .../main/scala/spark/rdd/CoalescedRDD.scala | 143 +++++++++++------- core/src/test/scala/spark/RDDSuite.scala | 38 +++-- 2 files changed, 118 insertions(+), 63 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index f999e9b0ec5..61c4d0c0042 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -36,7 +36,21 @@ private[spark] case class CoalescedRDDPartition( oos.defaultWriteObject() } + /** + * Gets the preferred location for this coalesced RDD partition. Most parent indices should prefer this machine. + * @return preferred location + */ def getPreferredLocation = prefLoc + + /** + * Computes how many of the parents partitions have getPreferredLocation as one of their preferredLocations + * @return locality of this coalesced partition between 0 and 1 + */ + def localFraction :Double = { + var loc: Int = 0 + parents.foreach(p => if (rdd.preferredLocations(p).contains(getPreferredLocation)) loc += 1) + if (parents.size == 0) 0.0 else (loc.toDouble / parents.size.toDouble) + } } /** @@ -55,13 +69,9 @@ class CoalescedRDD[T: ClassManifest]( override def getPartitions: Array[Partition] = { val res = mutable.ArrayBuffer[CoalescedRDDPartition]() - val targetLen = math.min(prev.partitions.length, maxPartitions) val packer = new PartitionCoalescer(maxPartitions, prev, balanceSlack) - packer.setupGroups(targetLen) // setup the groups (bins) and preferred locations - packer.throwBalls() // assign partitions (balls) to each group (bins) - - for ((pg, i) <- packer.groupArr.zipWithIndex) { + for ((pg, i) <- packer.getPartitions.zipWithIndex) { val ids = pg.list.map(_.index).toArray res += new CoalescedRDDPartition(i, prev, ids, pg.prefLoc) } @@ -86,27 +96,68 @@ class CoalescedRDD[T: ClassManifest]( super.clearDependencies() prev = null } + + /** + * Returns the preferred machine for the split. If split is of type CoalescedRDDPartition, then the preferred machine + * will be one which most parent splits prefer too. + * @param split + * @return the machine most preferred by split + */ + override def getPreferredLocations(split: Partition): Seq[String] = { + if (split.isInstanceOf[CoalescedRDDPartition]) + List(split.asInstanceOf[CoalescedRDDPartition].getPreferredLocation) + else + super.getPreferredLocations(split) + } + } class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { + private def compare(o1: PartitionGroup, o2: PartitionGroup): Boolean = o1.size < o2.size + private def compare(o1: Option[PartitionGroup], o2: Option[PartitionGroup]): Boolean = + if (o1 == None) false else if (o2 == None) true else compare(o1.get, o2.get) + + private val rnd = new scala.util.Random(7919) // keep this class deterministic + + // each element of groupArr represents one coalesced partition + private val groupArr = mutable.ArrayBuffer[PartitionGroup]() + + // hash used to check whether some machine is already in groupArr + private val groupHash = mutable.Map[String, mutable.ListBuffer[PartitionGroup]]() + + // hash used for the first maxPartitions (to avoid duplicates) + private val initialHash = mutable.Map[Partition, Boolean]() + + // determines the tradeoff between load-balancing the partitions sizes and their locality + // e.g. balanceSlack=0.10 means that it allows up to 10% imbalance in favor of locality + private val slack = (balanceSlack * prev.partitions.size).toInt + + private var noLocality = true // if true if no preferredLocations exists for parent RDD + + this.setupGroups(math.min(prev.partitions.length, maxPartitions)) // setup the groups (bins) and preferred locations + this.throwBalls() // assign partitions (balls) to each group (bins) + + def getPartitions : Array[PartitionGroup] = groupArr.filter( pg => pg.size > 0).toArray + // this class just keeps iterating and rotating infinitely over the partitions of the RDD // next() returns the next preferred machine that a partition is replicated on // the rotator first goes through the first replica copy of each partition, then second, then third - private class RotateLocations(prev: RDD[_]) extends Iterator[String] { + // the iterators return type is a tuple: (replicaString, partition) + private class RotateLocations(prev: RDD[_]) extends Iterator[(String, Partition)] { - private var it: Iterator[String] = resetIterator() + private var it: Iterator[(String, Partition)] = resetIterator() override val isEmpty = !it.hasNext // initializes/resets to start iterating from the beginning private def resetIterator() = { val i1 = prev.partitions.view.map( (p: Partition) => - { if (prev.preferredLocations(p).length > 0) Some(prev.preferredLocations(p)(0)) else None } ) + { if (prev.preferredLocations(p).length > 0) Some((prev.preferredLocations(p)(0),p)) else None } ) val i2 = prev.partitions.view.map( (p: Partition) => - { if (prev.preferredLocations(p).length > 1) Some(prev.preferredLocations(p)(1)) else None } ) + { if (prev.preferredLocations(p).length > 1) Some((prev.preferredLocations(p)(1),p)) else None } ) val i3 = prev.partitions.view.map( (p: Partition) => - { if (prev.preferredLocations(p).length > 2) Some(prev.preferredLocations(p)(2)) else None } ) + { if (prev.preferredLocations(p).length > 2) Some((prev.preferredLocations(p)(2),p)) else None } ) val res = List(i1,i2,i3) res.view.flatMap(x => x).flatten.iterator // fuses the 3 iterators (1st replica, 2nd, 3rd) into one iterator } @@ -115,7 +166,7 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) def hasNext(): Boolean = !isEmpty // return the next preferredLocation of some partition of the RDD - def next(): String = { + def next(): (String, Partition) = { if (it.hasNext) it.next() else { @@ -126,42 +177,11 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) } case class PartitionGroup(prefLoc: String = "") { - var list = mutable.MutableList[Partition]() + var list = mutable.ListBuffer[Partition]() def size = list.size - - // returns number of partitions that got locality in this group - def local = { - var loc: Int = 0 - list.foreach(p => if (prev.preferredLocations(p).contains(prefLoc)) loc += 1) - loc - } - - override def toString(): String = { - val localityStr = if (size == 0) "0" else (local*100. / size).toInt.toString - "PartitionGroup(\"" + prefLoc + "\") size: " + size + " locality: " + localityStr +"% \n" - // list.map("\t\t" + _.toString).mkString("\n") + "\n" - } } - def compare(o1: PartitionGroup, o2: PartitionGroup): Boolean = o1.size < o2.size - def compare(o1: Option[PartitionGroup], o2: Option[PartitionGroup]): Boolean = - if (o1 == None) false else if (o2 == None) true else compare(o1.get, o2.get) - - val rnd = new scala.util.Random(7919) // keep this class deterministic - - // each element of groupArr represents one coalesced partition - val groupArr = mutable.ArrayBuffer[PartitionGroup]() - - // hash used to check whether some machine is already in groupArr - val groupHash = mutable.Map[String, mutable.MutableList[PartitionGroup]]() - - // determines the tradeoff between load-balancing the partitions sizes and their locality - // e.g. balanceSlack=0.10 means that it allows up to 10% imbalance in favor of locality - val slack = (balanceSlack * prev.partitions.size).toInt - - private var noLocality = true // if true if no preferredLocations exists for parent RDD - /** * Sorts and gets the least element of the list associated with key in groupHash * The returned PartitionGroup is the least loaded of all groups that represent the machine "key" @@ -172,13 +192,21 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) groupHash.get(key).map(_.sortWith(compare).head) } + def addPartToPGroup(part : Partition, pgroup : PartitionGroup) : Boolean = { + if (!initialHash.contains(part)) { + pgroup.list += part // already preassign this element, ensures every bucket will have 1 element + initialHash += (part -> true) // needed to avoid assigning partitions to multiple buckets/groups + true + } else false + } + /** * Initializes targetLen partition groups and assigns a preferredLocation * This uses coupon collector to estimate how many preferredLocations it must rotate through until it has seen * most of the preferred locations (2 * n log(n)) * @param targetLen */ - def setupGroups(targetLen: Int) { + private def setupGroups(targetLen: Int) { val rotIt = new RotateLocations(prev) // deal with empty rotator case, just create targetLen partition groups with no preferred location @@ -199,22 +227,29 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) // likely targetLen >> number of preferred locations (more buckets than there are machines) while (numCreated < targetLen && tries < expectedCoupons2) { tries += 1 - val nxt = rotIt.next() - if (!groupHash.contains(nxt)) { - val pgroup = PartitionGroup(nxt) + val (nxt_replica, nxt_part) = rotIt.next() + if (!groupHash.contains(nxt_replica)) { + val pgroup = PartitionGroup(nxt_replica) groupArr += pgroup - groupHash += (nxt -> (mutable.MutableList(pgroup))) // use list in case we have multiple groups for same machine + addPartToPGroup(nxt_part, pgroup) + groupHash += (nxt_replica -> (mutable.ListBuffer(pgroup))) // list in case we have multiple groups per machine numCreated += 1 } } while (numCreated < targetLen) { // if we don't have enough partition groups, just create duplicates - val nxt = rotIt.next() - val pgroup = PartitionGroup(nxt) + var (nxt_replica, nxt_part) = rotIt.next() + val pgroup = PartitionGroup(nxt_replica) groupArr += pgroup - groupHash.get(nxt).get += pgroup + groupHash.get(nxt_replica).get += pgroup + var tries = 0 + while (!addPartToPGroup(nxt_part, pgroup) && tries < targetLen) { // ensure each group has at least one partition + nxt_part = rotIt.next()._2 + tries += 1 + } numCreated += 1 } + } /** @@ -224,7 +259,7 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) * @param p partition (ball to be thrown) * @return partition group (bin to be put in) */ - def pickBin(p: Partition): PartitionGroup = { + private def pickBin(p: Partition): PartitionGroup = { val pref = prev.preferredLocations(p).map(getLeastGroupHash(_)).sortWith(compare) // least loaded bin of replicas val prefPart = if (pref == Nil) None else pref.head @@ -243,7 +278,7 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) } } - def throwBalls() { + private def throwBalls() { if (noLocality) { // no preferredLocations in parent RDD, no randomization needed if (maxPartitions > groupArr.size) { // just return prev.partitions for ((p,i) <- prev.partitions.zipWithIndex) { @@ -257,7 +292,7 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) } } } else { - for (p <- prev.partitions) { // throw every partition (ball) into a partition group (bin) + for (p <- prev.partitions if (!initialHash.contains(p))) { // throw every partition into a partition group pickBin(p).list += p } } diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 881bdedfe53..c200bfe909b 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -22,7 +22,8 @@ import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts._ import org.scalatest.time.{Span, Millis} import spark.SparkContext._ -import spark.rdd.{CoalescedRDD, CoGroupedRDD, EmptyRDD, PartitionPruningRDD, ShuffledRDD} +import spark.rdd._ +import scala.collection.parallel.mutable class RDDSuite extends FunSuite with SharedSparkContext { @@ -184,16 +185,35 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(splits.foldLeft(true)( (x,y) => if (!x) false else y.length >= 2) === true) // descent balance (2+ per bin) - val prefs = List(List("m1","m2","m3"), List("m4","m5","m6")) - val data2 = sc.makeRDD((1 to 100).map( i => (i, prefs(i % 2) ))) // alternate machine prefs - val coalesced2 = data2.coalesce(10) - val splits2 = coalesced2.glom().collect().map(_.toList).toList + // If we try to coalesce into more partitions than the original RDD, it should just + // keep the original number of partitions. + val coalesced4 = data.coalesce(20) + assert(coalesced4.glom().collect().map(_.toList).toList.sortWith( + (x, y) => if (x.isEmpty) false else if (y.isEmpty) true else x(0) < y(0)) === (1 to 9).map(x => List(x)).toList) + + + // large scale experiment + import collection.mutable + val rnd = scala.util.Random + val partitions = 10000 + val numMachines = 50 + val machines = mutable.ListBuffer[String]() + (1 to numMachines).foreach(machines += "m"+_) + + val blocks = (1 to partitions).map( i => (i, (i to (i+2)).map{ j => machines(rnd.nextInt(machines.size)) } )) + + val data2 = sc.makeRDD(blocks) + val coalesced2 = data2.coalesce(numMachines*2) - // this gives a list of pairs, each pair is of the form (even,odd), where even is the number of even elements... - val list = splits2.map( ls => ls.foldLeft((0,0))( (x,y) => if (y % 2 == 0) (x._1+1,x._2) else (x._1,x._2+1)) ) - val maxes = list.map( { case (a,b) => if (a>b) a else b } ) // get the maxs, this represents the locality - maxes.foreach( locality => assert( locality > 7) ) // at least 70% locality in each partition + // test that you get over 95% locality in each group + val minLocality = coalesced2.partitions.map( part => part.asInstanceOf[CoalescedRDDPartition].localFraction ) + .foldLeft(100.)( (perc, loc) => math.min(perc,loc) ) + assert(minLocality > 0.95) + // test that the groups are load balanced with 100 +/- 15 elements in each + val maxImbalance = coalesced2.partitions.map( part => part.asInstanceOf[CoalescedRDDPartition].parents.size ) + .foldLeft(0)((dev, curr) => math.max(math.abs(100-curr),dev)) + assert(maxImbalance < 15) } test("zipped RDDs") { From c4d59910b149b8b9bbf729f38e3eef3fb64fc85b Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Wed, 14 Aug 2013 19:59:21 -0700 Subject: [PATCH 0795/2521] added goals inline as comment --- .../main/scala/spark/rdd/CoalescedRDD.scala | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 61c4d0c0042..6af55cd80c5 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -60,6 +60,27 @@ private[spark] case class CoalescedRDDPartition( * * This transformation is useful when an RDD with many partitions gets filtered into a smaller one, * or to avoid having a large number of small tasks when processing a directory with many files. + * + * If there is no locality information (no preferredLocations) in the parent RDD, then the coalescing + * is very simple: chunk parents that are close in the Array in chunks. + * If there is locality information, it proceeds to pack them with the following three goals in mind: + * + * (1) Balance the groups so they roughly have the same number of parent partitions + * (2) Achieve locality per partition, i.e. there exists one machine which most parent partitions prefer + * (3) Be efficient, i.e. O(n) algorithm for n parent partitions (underlying problem is likely NP-hard) + * (4) Balance preferred machines, i.e. avoid as much as possible picking the same preferred machine + * + * Furthermore, it is assumed that the parent RDD may have many partitions, e.g. 100 000. + * We assume the final number of desired partitions is small, e.g. less than 1000. + * + * The algorithm tries to assign unique preferred machines to each partition. If the number of desired + * partitions is greater than the number of preferred machines (can happen), it needs to start picking + * duplicate preferred machines. This is determined using coupon collector estimation (2n log(n)). + * The load balancing is done using power-of-two randomized bins-balls with one twist: it tries to + * also achieve locality. This is done by allowing a slack (balanceSlack) between two bins. + * If two bins are within the slack in terms of balance, the algorithm will assign partitions + * according to locality. (contact alig for questions) + * */ class CoalescedRDD[T: ClassManifest]( @transient var prev: RDD[T], From 937f72feb86b406056bd163ce7320f582b70ab16 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Wed, 14 Aug 2013 20:21:53 -0700 Subject: [PATCH 0796/2521] word wrap before 100 chars per line --- .../main/scala/spark/rdd/CoalescedRDD.scala | 75 ++++++++++--------- core/src/test/scala/spark/RDDSuite.scala | 17 +++-- 2 files changed, 51 insertions(+), 41 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 6af55cd80c5..beed2d5b694 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -37,13 +37,15 @@ private[spark] case class CoalescedRDDPartition( } /** - * Gets the preferred location for this coalesced RDD partition. Most parent indices should prefer this machine. + * Gets the preferred location for this coalesced RDD partition. + * Most parent indices should prefer this machine. * @return preferred location */ def getPreferredLocation = prefLoc /** - * Computes how many of the parents partitions have getPreferredLocation as one of their preferredLocations + * Computes how many of the parents partitions have getPreferredLocation + * as one of their preferredLocations * @return locality of this coalesced partition between 0 and 1 */ def localFraction :Double = { @@ -61,24 +63,24 @@ private[spark] case class CoalescedRDDPartition( * This transformation is useful when an RDD with many partitions gets filtered into a smaller one, * or to avoid having a large number of small tasks when processing a directory with many files. * - * If there is no locality information (no preferredLocations) in the parent RDD, then the coalescing + * If there is no locality information (no preferredLocations) in the parent, then the coalescing * is very simple: chunk parents that are close in the Array in chunks. - * If there is locality information, it proceeds to pack them with the following three goals in mind: + * If there is locality information, it proceeds to pack them with the following three goals: * * (1) Balance the groups so they roughly have the same number of parent partitions - * (2) Achieve locality per partition, i.e. there exists one machine which most parent partitions prefer - * (3) Be efficient, i.e. O(n) algorithm for n parent partitions (underlying problem is likely NP-hard) + * (2) Achieve locality per partition, i.e. there exists one machine which most parent splits prefer + * (3) Be efficient, i.e. O(n) algorithm for n parent partitions (problem is likely NP-hard) * (4) Balance preferred machines, i.e. avoid as much as possible picking the same preferred machine * * Furthermore, it is assumed that the parent RDD may have many partitions, e.g. 100 000. * We assume the final number of desired partitions is small, e.g. less than 1000. * - * The algorithm tries to assign unique preferred machines to each partition. If the number of desired - * partitions is greater than the number of preferred machines (can happen), it needs to start picking - * duplicate preferred machines. This is determined using coupon collector estimation (2n log(n)). - * The load balancing is done using power-of-two randomized bins-balls with one twist: it tries to - * also achieve locality. This is done by allowing a slack (balanceSlack) between two bins. - * If two bins are within the slack in terms of balance, the algorithm will assign partitions + * The algorithm tries to assign unique preferred machines to each partition. If the number of + * desired partitions is greater than the number of preferred machines (can happen), it needs to + * start picking duplicate preferred machines. This is determined using coupon collector estimation + * (2n log(n)). The load balancing is done using power-of-two randomized bins-balls with one twist: + * it tries to also achieve locality. This is done by allowing a slack (balanceSlack) between two + * bins. If two bins are within the slack in terms of balance, the algorithm will assign partitions * according to locality. (contact alig for questions) * */ @@ -119,8 +121,8 @@ class CoalescedRDD[T: ClassManifest]( } /** - * Returns the preferred machine for the split. If split is of type CoalescedRDDPartition, then the preferred machine - * will be one which most parent splits prefer too. + * Returns the preferred machine for the split. If split is of type CoalescedRDDPartition, + * then the preferred machine will be one which most parent splits prefer too. * @param split * @return the machine most preferred by split */ @@ -157,14 +159,14 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) private var noLocality = true // if true if no preferredLocations exists for parent RDD - this.setupGroups(math.min(prev.partitions.length, maxPartitions)) // setup the groups (bins) and preferred locations + this.setupGroups(math.min(prev.partitions.length, maxPartitions)) // setup the groups (bins) this.throwBalls() // assign partitions (balls) to each group (bins) def getPartitions : Array[PartitionGroup] = groupArr.filter( pg => pg.size > 0).toArray // this class just keeps iterating and rotating infinitely over the partitions of the RDD // next() returns the next preferred machine that a partition is replicated on - // the rotator first goes through the first replica copy of each partition, then second, then third + // the rotator first goes through the first replica copy of each partition, then second, third // the iterators return type is a tuple: (replicaString, partition) private class RotateLocations(prev: RDD[_]) extends Iterator[(String, Partition)] { @@ -174,13 +176,16 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) // initializes/resets to start iterating from the beginning private def resetIterator() = { val i1 = prev.partitions.view.map( (p: Partition) => - { if (prev.preferredLocations(p).length > 0) Some((prev.preferredLocations(p)(0),p)) else None } ) + { if (prev.preferredLocations(p).length > 0) + Some((prev.preferredLocations(p)(0),p)) else None } ) val i2 = prev.partitions.view.map( (p: Partition) => - { if (prev.preferredLocations(p).length > 1) Some((prev.preferredLocations(p)(1),p)) else None } ) + { if (prev.preferredLocations(p).length > 1) + Some((prev.preferredLocations(p)(1),p)) else None } ) val i3 = prev.partitions.view.map( (p: Partition) => - { if (prev.preferredLocations(p).length > 2) Some((prev.preferredLocations(p)(2),p)) else None } ) + { if (prev.preferredLocations(p).length > 2) + Some((prev.preferredLocations(p)(2),p)) else None } ) val res = List(i1,i2,i3) - res.view.flatMap(x => x).flatten.iterator // fuses the 3 iterators (1st replica, 2nd, 3rd) into one iterator + res.view.flatMap(x => x).flatten.iterator // fuses the 3 iterators (1st replica, 2nd, 3rd) } // hasNext() is false iff there are no preferredLocations for any of the partitions of the RDD @@ -215,22 +220,22 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) def addPartToPGroup(part : Partition, pgroup : PartitionGroup) : Boolean = { if (!initialHash.contains(part)) { - pgroup.list += part // already preassign this element, ensures every bucket will have 1 element - initialHash += (part -> true) // needed to avoid assigning partitions to multiple buckets/groups + pgroup.list += part // already assign this element + initialHash += (part -> true) // needed to avoid assigning partitions to multiple buckets true } else false } /** * Initializes targetLen partition groups and assigns a preferredLocation - * This uses coupon collector to estimate how many preferredLocations it must rotate through until it has seen - * most of the preferred locations (2 * n log(n)) + * This uses coupon collector to estimate how many preferredLocations it must rotate through + * until it has seen most of the preferred locations (2 * n log(n)) * @param targetLen */ private def setupGroups(targetLen: Int) { val rotIt = new RotateLocations(prev) - // deal with empty rotator case, just create targetLen partition groups with no preferred location + // deal with empty case, just create targetLen partition groups with no preferred location if (!rotIt.hasNext()) { (1 to targetLen).foreach(x => groupArr += PartitionGroup()) return @@ -243,9 +248,9 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) var numCreated = 0 var tries = 0 - // rotate through until either targetLen unique/distinct preferred locations have been created OR - // we've rotated expectedCoupons2, in which case we have likely seen all preferred locations, i.e. - // likely targetLen >> number of preferred locations (more buckets than there are machines) + // rotate through until either targetLen unique/distinct preferred locations have been created + // OR we've rotated expectedCoupons2, in which case we have likely seen all preferred locations, + // i.e. likely targetLen >> number of preferred locations (more buckets than there are machines) while (numCreated < targetLen && tries < expectedCoupons2) { tries += 1 val (nxt_replica, nxt_part) = rotIt.next() @@ -253,18 +258,18 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) val pgroup = PartitionGroup(nxt_replica) groupArr += pgroup addPartToPGroup(nxt_part, pgroup) - groupHash += (nxt_replica -> (mutable.ListBuffer(pgroup))) // list in case we have multiple groups per machine + groupHash += (nxt_replica -> (mutable.ListBuffer(pgroup))) // list in case we have multiple numCreated += 1 } } - while (numCreated < targetLen) { // if we don't have enough partition groups, just create duplicates + while (numCreated < targetLen) { // if we don't have enough partition groups, create duplicates var (nxt_replica, nxt_part) = rotIt.next() val pgroup = PartitionGroup(nxt_replica) groupArr += pgroup groupHash.get(nxt_replica).get += pgroup var tries = 0 - while (!addPartToPGroup(nxt_part, pgroup) && tries < targetLen) { // ensure each group has at least one partition + while (!addPartToPGroup(nxt_part, pgroup) && tries < targetLen) { // ensure at least one part nxt_part = rotIt.next()._2 tries += 1 } @@ -281,12 +286,12 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) * @return partition group (bin to be put in) */ private def pickBin(p: Partition): PartitionGroup = { - val pref = prev.preferredLocations(p).map(getLeastGroupHash(_)).sortWith(compare) // least loaded bin of replicas + val pref = prev.preferredLocations(p).map(getLeastGroupHash(_)).sortWith(compare) // least load val prefPart = if (pref == Nil) None else pref.head val r1 = rnd.nextInt(groupArr.size) val r2 = rnd.nextInt(groupArr.size) - val minPowerOfTwo = if (groupArr(r1).size < groupArr(r2).size) groupArr(r1) else groupArr(r2) // power of 2 + val minPowerOfTwo = if (groupArr(r1).size < groupArr(r2).size) groupArr(r1) else groupArr(r2) if (prefPart== None) // if no preferred locations, just use basic power of two return minPowerOfTwo @@ -305,7 +310,7 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) for ((p,i) <- prev.partitions.zipWithIndex) { groupArr(i).list += p } - } else { // old code, just splits them grouping partitions that are next to each other in the array + } else { // old code, just splits them grouping partitions that are next to each other (0 until maxPartitions).foreach { i => val rangeStart = ((i.toLong * prev.partitions.length) / maxPartitions).toInt val rangeEnd = (((i.toLong + 1) * prev.partitions.length) / maxPartitions).toInt @@ -313,7 +318,7 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) } } } else { - for (p <- prev.partitions if (!initialHash.contains(p))) { // throw every partition into a partition group + for (p <- prev.partitions if (!initialHash.contains(p))) { // throw every partition into group pickBin(p).list += p } } diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index c200bfe909b..a757aebd655 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -178,18 +178,20 @@ class RDDSuite extends FunSuite with SharedSparkContext { // RDD with locality preferences spread (non-randomly) over 6 machines, m0 through m5 val data = sc.makeRDD((1 to 9).map( i => (i, (i to (i+2)).map{ j => "m" + (j%6)} ))) val coalesced1 = data.coalesce(3) - assert(coalesced1.collect().toList.sorted === (1 to 9).toList) // no data lost (NB: order might reshuffle) + assert(coalesced1.collect().toList.sorted === (1 to 9).toList) // no data lost val splits = coalesced1.glom().collect().map(_.toList).toList assert(splits.length === 3) // ensure it indeed created 3 partitions - assert(splits.foldLeft(true)( (x,y) => if (!x) false else y.length >= 2) === true) // descent balance (2+ per bin) + assert(splits.foldLeft(true) + ( (x,y) => if (!x) false else y.length >= 2) === true) // (2+ balance) // If we try to coalesce into more partitions than the original RDD, it should just // keep the original number of partitions. val coalesced4 = data.coalesce(20) assert(coalesced4.glom().collect().map(_.toList).toList.sortWith( - (x, y) => if (x.isEmpty) false else if (y.isEmpty) true else x(0) < y(0)) === (1 to 9).map(x => List(x)).toList) + (x, y) => if (x.isEmpty) false else if (y.isEmpty) true else x(0) < y(0)) === (1 to 9). + map(x => List(x)).toList) // large scale experiment @@ -200,18 +202,21 @@ class RDDSuite extends FunSuite with SharedSparkContext { val machines = mutable.ListBuffer[String]() (1 to numMachines).foreach(machines += "m"+_) - val blocks = (1 to partitions).map( i => (i, (i to (i+2)).map{ j => machines(rnd.nextInt(machines.size)) } )) + val blocks = (1 to partitions).map( i => (i, (i to (i+2)) + .map{ j => machines(rnd.nextInt(machines.size)) } )) val data2 = sc.makeRDD(blocks) val coalesced2 = data2.coalesce(numMachines*2) // test that you get over 95% locality in each group - val minLocality = coalesced2.partitions.map( part => part.asInstanceOf[CoalescedRDDPartition].localFraction ) + val minLocality = coalesced2.partitions + .map( part => part.asInstanceOf[CoalescedRDDPartition].localFraction ) .foldLeft(100.)( (perc, loc) => math.min(perc,loc) ) assert(minLocality > 0.95) // test that the groups are load balanced with 100 +/- 15 elements in each - val maxImbalance = coalesced2.partitions.map( part => part.asInstanceOf[CoalescedRDDPartition].parents.size ) + val maxImbalance = coalesced2.partitions + .map( part => part.asInstanceOf[CoalescedRDDPartition].parents.size ) .foldLeft(0)((dev, curr) => math.max(math.abs(100-curr),dev)) assert(maxImbalance < 15) } From f6e47e8b51c55a664ce062dbe8a320591644ad62 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Thu, 15 Aug 2013 10:58:39 -0700 Subject: [PATCH 0797/2521] Renamed split to partition --- .../main/scala/spark/rdd/CoalescedRDD.scala | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index beed2d5b694..f46dd1ee6c9 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -31,7 +31,7 @@ private[spark] case class CoalescedRDDPartition( @throws(classOf[IOException]) private def writeObject(oos: ObjectOutputStream) { - // Update the reference to parent split at the time of task serialization + // Update the reference to parent partition at the time of task serialization parents = parentsIndices.map(rdd.partitions(_)) oos.defaultWriteObject() } @@ -68,7 +68,7 @@ private[spark] case class CoalescedRDDPartition( * If there is locality information, it proceeds to pack them with the following three goals: * * (1) Balance the groups so they roughly have the same number of parent partitions - * (2) Achieve locality per partition, i.e. there exists one machine which most parent splits prefer + * (2) Achieve locality per partition, i.e. find one machine which most parent partitions prefer * (3) Be efficient, i.e. O(n) algorithm for n parent partitions (problem is likely NP-hard) * (4) Balance preferred machines, i.e. avoid as much as possible picking the same preferred machine * @@ -102,9 +102,9 @@ class CoalescedRDD[T: ClassManifest]( res.toArray } - override def compute(split: Partition, context: TaskContext): Iterator[T] = { - split.asInstanceOf[CoalescedRDDPartition].parents.iterator.flatMap { parentSplit => - firstParent[T].iterator(parentSplit, context) + override def compute(partition: Partition, context: TaskContext): Iterator[T] = { + partition.asInstanceOf[CoalescedRDDPartition].parents.iterator.flatMap { parentPartition => + firstParent[T].iterator(parentPartition, context) } } @@ -121,16 +121,16 @@ class CoalescedRDD[T: ClassManifest]( } /** - * Returns the preferred machine for the split. If split is of type CoalescedRDDPartition, + * Returns the preferred machine for the partition. If split is of type CoalescedRDDPartition, * then the preferred machine will be one which most parent splits prefer too. - * @param split + * @param partition * @return the machine most preferred by split */ - override def getPreferredLocations(split: Partition): Seq[String] = { - if (split.isInstanceOf[CoalescedRDDPartition]) - List(split.asInstanceOf[CoalescedRDDPartition].getPreferredLocation) + override def getPreferredLocations(partition: Partition): Seq[String] = { + if (partition.isInstanceOf[CoalescedRDDPartition]) + List(partition.asInstanceOf[CoalescedRDDPartition].getPreferredLocation) else - super.getPreferredLocations(split) + super.getPreferredLocations(partition) } } From f24861b60a9ddef1369a0a6816f6922575940656 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Thu, 15 Aug 2013 11:00:54 -0700 Subject: [PATCH 0798/2521] Fix bug in tests --- core/src/main/scala/spark/rdd/CoalescedRDD.scala | 2 +- core/src/test/scala/spark/RDDSuite.scala | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index f46dd1ee6c9..205175edfc9 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -310,7 +310,7 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) for ((p,i) <- prev.partitions.zipWithIndex) { groupArr(i).list += p } - } else { // old code, just splits them grouping partitions that are next to each other + } else { // no locality available, then simply split partitions based on positions in array (0 until maxPartitions).foreach { i => val rangeStart = ((i.toLong * prev.partitions.length) / maxPartitions).toInt val rangeEnd = (((i.toLong + 1) * prev.partitions.length) / maxPartitions).toInt diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index a757aebd655..8b6fa9c81e9 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -184,7 +184,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(splits.length === 3) // ensure it indeed created 3 partitions assert(splits.foldLeft(true) - ( (x,y) => if (!x) false else y.length >= 2) === true) // (2+ balance) + ( (x,y) => if (!x) false else y.length >= 1) === true) // (1+ balance) // If we try to coalesce into more partitions than the original RDD, it should just // keep the original number of partitions. @@ -208,17 +208,17 @@ class RDDSuite extends FunSuite with SharedSparkContext { val data2 = sc.makeRDD(blocks) val coalesced2 = data2.coalesce(numMachines*2) - // test that you get over 95% locality in each group + // test that you get over 90% locality in each group val minLocality = coalesced2.partitions .map( part => part.asInstanceOf[CoalescedRDDPartition].localFraction ) .foldLeft(100.)( (perc, loc) => math.min(perc,loc) ) - assert(minLocality > 0.95) + assert(minLocality > 0.90) - // test that the groups are load balanced with 100 +/- 15 elements in each + // test that the groups are load balanced with 100 +/- 20 elements in each val maxImbalance = coalesced2.partitions .map( part => part.asInstanceOf[CoalescedRDDPartition].parents.size ) .foldLeft(0)((dev, curr) => math.max(math.abs(100-curr),dev)) - assert(maxImbalance < 15) + assert(maxImbalance < 20) } test("zipped RDDs") { From f67753cdfcb08b3786320c45af98e097a8ed5d39 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Thu, 15 Aug 2013 11:03:42 -0700 Subject: [PATCH 0799/2521] made preferredLocation a val of the surrounding case class --- core/src/main/scala/spark/rdd/CoalescedRDD.scala | 13 +++---------- 1 file changed, 3 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 205175edfc9..bfd0cf257e0 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -25,7 +25,7 @@ private[spark] case class CoalescedRDDPartition( index: Int, @transient rdd: RDD[_], parentsIndices: Array[Int], - prefLoc: String = "" + @transient preferredLocation: String = "" ) extends Partition { var parents: Seq[Partition] = parentsIndices.map(rdd.partitions(_)) @@ -36,13 +36,6 @@ private[spark] case class CoalescedRDDPartition( oos.defaultWriteObject() } - /** - * Gets the preferred location for this coalesced RDD partition. - * Most parent indices should prefer this machine. - * @return preferred location - */ - def getPreferredLocation = prefLoc - /** * Computes how many of the parents partitions have getPreferredLocation * as one of their preferredLocations @@ -50,7 +43,7 @@ private[spark] case class CoalescedRDDPartition( */ def localFraction :Double = { var loc: Int = 0 - parents.foreach(p => if (rdd.preferredLocations(p).contains(getPreferredLocation)) loc += 1) + parents.foreach(p => if (rdd.preferredLocations(p).contains(preferredLocation)) loc += 1) if (parents.size == 0) 0.0 else (loc.toDouble / parents.size.toDouble) } } @@ -128,7 +121,7 @@ class CoalescedRDD[T: ClassManifest]( */ override def getPreferredLocations(partition: Partition): Seq[String] = { if (partition.isInstanceOf[CoalescedRDDPartition]) - List(partition.asInstanceOf[CoalescedRDDPartition].getPreferredLocation) + List(partition.asInstanceOf[CoalescedRDDPartition].preferredLocation) else super.getPreferredLocations(partition) } From 4f99be1ffd2f5237566b143b317e3677b50d06e8 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Thu, 15 Aug 2013 11:04:38 -0700 Subject: [PATCH 0800/2521] use count rather than foreach --- core/src/main/scala/spark/rdd/CoalescedRDD.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index bfd0cf257e0..450d8b33a93 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -42,8 +42,7 @@ private[spark] case class CoalescedRDDPartition( * @return locality of this coalesced partition between 0 and 1 */ def localFraction :Double = { - var loc: Int = 0 - parents.foreach(p => if (rdd.preferredLocations(p).contains(preferredLocation)) loc += 1) + val loc = parents.count(p => rdd.preferredLocations(p).contains(preferredLocation)) if (parents.size == 0) 0.0 else (loc.toDouble / parents.size.toDouble) } } From 02d6464f2f51217ab7435ef594f003943a742bbf Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Thu, 15 Aug 2013 11:12:08 -0700 Subject: [PATCH 0801/2521] space removed --- core/src/main/scala/spark/rdd/CoalescedRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 450d8b33a93..56f350b1f5a 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -79,7 +79,7 @@ private[spark] case class CoalescedRDDPartition( class CoalescedRDD[T: ClassManifest]( @transient var prev: RDD[T], maxPartitions: Int, - balanceSlack: Double = 0.10 ) + balanceSlack: Double = 0.10) extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies override def getPartitions: Array[Partition] = { From 339598c0806d01e3d43cd49dd02e9d510b5f586b Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Thu, 15 Aug 2013 12:05:20 -0700 Subject: [PATCH 0802/2521] several of Reynold's suggestions implemented --- .../main/scala/spark/rdd/CoalescedRDD.scala | 29 +++++++++---------- 1 file changed, 14 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 56f350b1f5a..8d06b4ceb87 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -83,15 +83,13 @@ class CoalescedRDD[T: ClassManifest]( extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies override def getPartitions: Array[Partition] = { - val res = mutable.ArrayBuffer[CoalescedRDDPartition]() val packer = new PartitionCoalescer(maxPartitions, prev, balanceSlack) - for ((pg, i) <- packer.getPartitions.zipWithIndex) { - val ids = pg.list.map(_.index).toArray - res += new CoalescedRDDPartition(i, prev, ids, pg.prefLoc) + packer.getPartitions.zipWithIndex.map { + case (pg, i) => + val ids = pg.list.map(_.index).toArray + new CoalescedRDDPartition(i, prev, ids, pg.prefLoc) } - - res.toArray } override def compute(partition: Partition, context: TaskContext): Iterator[T] = { @@ -119,10 +117,11 @@ class CoalescedRDD[T: ClassManifest]( * @return the machine most preferred by split */ override def getPreferredLocations(partition: Partition): Seq[String] = { - if (partition.isInstanceOf[CoalescedRDDPartition]) + if (partition.isInstanceOf[CoalescedRDDPartition]) { List(partition.asInstanceOf[CoalescedRDDPartition].preferredLocation) - else + } else { super.getPreferredLocations(partition) + } } } @@ -167,15 +166,15 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) // initializes/resets to start iterating from the beginning private def resetIterator() = { - val i1 = prev.partitions.view.map( (p: Partition) => + val i1 = prev.partitions.view.map{ p: Partition => { if (prev.preferredLocations(p).length > 0) - Some((prev.preferredLocations(p)(0),p)) else None } ) - val i2 = prev.partitions.view.map( (p: Partition) => + Some((prev.preferredLocations(p)(0),p)) else None } } + val i2 = prev.partitions.view.map{ p: Partition => { if (prev.preferredLocations(p).length > 1) - Some((prev.preferredLocations(p)(1),p)) else None } ) - val i3 = prev.partitions.view.map( (p: Partition) => + Some((prev.preferredLocations(p)(1),p)) else None } } + val i3 = prev.partitions.view.map{ p: Partition => { if (prev.preferredLocations(p).length > 2) - Some((prev.preferredLocations(p)(2),p)) else None } ) + Some((prev.preferredLocations(p)(2),p)) else None } } val res = List(i1,i2,i3) res.view.flatMap(x => x).flatten.iterator // fuses the 3 iterators (1st replica, 2nd, 3rd) } @@ -215,7 +214,7 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) pgroup.list += part // already assign this element initialHash += (part -> true) // needed to avoid assigning partitions to multiple buckets true - } else false + } else { false } } /** From 35537e6341dad72366a7ba6d92d6de9c710542ac Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Thu, 15 Aug 2013 12:52:13 -0700 Subject: [PATCH 0803/2521] Made a function object that returns the coalesced groups --- .../main/scala/spark/rdd/CoalescedRDD.scala | 65 ++++++++++--------- 1 file changed, 35 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 8d06b4ceb87..1cfa404fd80 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -83,9 +83,9 @@ class CoalescedRDD[T: ClassManifest]( extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies override def getPartitions: Array[Partition] = { - val packer = new PartitionCoalescer(maxPartitions, prev, balanceSlack) + val groupList = coalescePartitions(maxPartitions, prev, balanceSlack) - packer.getPartitions.zipWithIndex.map { + groupList.zipWithIndex.map { case (pg, i) => val ids = pg.list.map(_.index).toArray new CoalescedRDDPartition(i, prev, ids, pg.prefLoc) @@ -126,46 +126,40 @@ class CoalescedRDD[T: ClassManifest]( } +class coalescePartitions protected (maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { -class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { - - private def compare(o1: PartitionGroup, o2: PartitionGroup): Boolean = o1.size < o2.size - private def compare(o1: Option[PartitionGroup], o2: Option[PartitionGroup]): Boolean = + protected def compare(o1: PartitionGroup, o2: PartitionGroup): Boolean = o1.size < o2.size + protected def compare(o1: Option[PartitionGroup], o2: Option[PartitionGroup]): Boolean = if (o1 == None) false else if (o2 == None) true else compare(o1.get, o2.get) - private val rnd = new scala.util.Random(7919) // keep this class deterministic + protected val rnd = new scala.util.Random(7919) // keep this class deterministic // each element of groupArr represents one coalesced partition - private val groupArr = mutable.ArrayBuffer[PartitionGroup]() + protected val groupArr = mutable.ArrayBuffer[PartitionGroup]() // hash used to check whether some machine is already in groupArr - private val groupHash = mutable.Map[String, mutable.ListBuffer[PartitionGroup]]() + protected val groupHash = mutable.Map[String, mutable.ListBuffer[PartitionGroup]]() // hash used for the first maxPartitions (to avoid duplicates) - private val initialHash = mutable.Map[Partition, Boolean]() + protected val initialHash = mutable.Map[Partition, Boolean]() // determines the tradeoff between load-balancing the partitions sizes and their locality // e.g. balanceSlack=0.10 means that it allows up to 10% imbalance in favor of locality - private val slack = (balanceSlack * prev.partitions.size).toInt - - private var noLocality = true // if true if no preferredLocations exists for parent RDD + protected val slack = (balanceSlack * prev.partitions.size).toInt - this.setupGroups(math.min(prev.partitions.length, maxPartitions)) // setup the groups (bins) - this.throwBalls() // assign partitions (balls) to each group (bins) - - def getPartitions : Array[PartitionGroup] = groupArr.filter( pg => pg.size > 0).toArray + protected var noLocality = true // if true if no preferredLocations exists for parent RDD // this class just keeps iterating and rotating infinitely over the partitions of the RDD // next() returns the next preferred machine that a partition is replicated on // the rotator first goes through the first replica copy of each partition, then second, third // the iterators return type is a tuple: (replicaString, partition) - private class RotateLocations(prev: RDD[_]) extends Iterator[(String, Partition)] { + protected class RotateLocations(prev: RDD[_]) extends Iterator[(String, Partition)] { - private var it: Iterator[(String, Partition)] = resetIterator() + protected var it: Iterator[(String, Partition)] = resetIterator() override val isEmpty = !it.hasNext // initializes/resets to start iterating from the beginning - private def resetIterator() = { + protected def resetIterator() = { val i1 = prev.partitions.view.map{ p: Partition => { if (prev.preferredLocations(p).length > 0) Some((prev.preferredLocations(p)(0),p)) else None } } @@ -193,19 +187,13 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) } } - case class PartitionGroup(prefLoc: String = "") { - var list = mutable.ListBuffer[Partition]() - - def size = list.size - } - /** * Sorts and gets the least element of the list associated with key in groupHash * The returned PartitionGroup is the least loaded of all groups that represent the machine "key" * @param key string representing a partitioned group on preferred machine key * @return Option of PartitionGroup that has least elements for key */ - private def getLeastGroupHash(key: String): Option[PartitionGroup] = { + protected def getLeastGroupHash(key: String): Option[PartitionGroup] = { groupHash.get(key).map(_.sortWith(compare).head) } @@ -223,7 +211,7 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) * until it has seen most of the preferred locations (2 * n log(n)) * @param targetLen */ - private def setupGroups(targetLen: Int) { + protected def setupGroups(targetLen: Int) { val rotIt = new RotateLocations(prev) // deal with empty case, just create targetLen partition groups with no preferred location @@ -276,7 +264,7 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) * @param p partition (ball to be thrown) * @return partition group (bin to be put in) */ - private def pickBin(p: Partition): PartitionGroup = { + protected def pickBin(p: Partition): PartitionGroup = { val pref = prev.preferredLocations(p).map(getLeastGroupHash(_)).sortWith(compare) // least load val prefPart = if (pref == Nil) None else pref.head @@ -295,7 +283,7 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) } } - private def throwBalls() { + protected def throwBalls() { if (noLocality) { // no preferredLocations in parent RDD, no randomization needed if (maxPartitions > groupArr.size) { // just return prev.partitions for ((p,i) <- prev.partitions.zipWithIndex) { @@ -315,4 +303,21 @@ class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) } } + def getPartitions : Array[PartitionGroup] = groupArr.filter( pg => pg.size > 0).toArray + +} + +case class PartitionGroup(prefLoc: String = "") { + var list = mutable.ListBuffer[Partition]() + + def size = list.size +} + +object coalescePartitions { + def apply(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) : Array[PartitionGroup] = { + val pc = new coalescePartitions(maxPartitions, prev, balanceSlack) + pc.setupGroups(math.min(prev.partitions.length, maxPartitions)) // setup the groups (bins) + pc.throwBalls() // assign partitions (balls) to each group (bins) + pc.getPartitions + } } From abcefb3858aac373bd8898c3e998375d5f26b803 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Fri, 16 Aug 2013 11:56:44 -0700 Subject: [PATCH 0804/2521] fixed matei's comments --- .../main/scala/spark/rdd/CoalescedRDD.scala | 31 ++++++++++--------- 1 file changed, 16 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 1cfa404fd80..e6fe8ec8ee5 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -41,7 +41,7 @@ private[spark] case class CoalescedRDDPartition( * as one of their preferredLocations * @return locality of this coalesced partition between 0 and 1 */ - def localFraction :Double = { + def localFraction: Double = { val loc = parents.count(p => rdd.preferredLocations(p).contains(preferredLocation)) if (parents.size == 0) 0.0 else (loc.toDouble / parents.size.toDouble) } @@ -83,9 +83,9 @@ class CoalescedRDD[T: ClassManifest]( extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies override def getPartitions: Array[Partition] = { - val groupList = coalescePartitions(maxPartitions, prev, balanceSlack) + val pc = new PartitionCoalescer(maxPartitions, prev, balanceSlack) - groupList.zipWithIndex.map { + pc.run().zipWithIndex.map { case (pg, i) => val ids = pg.list.map(_.index).toArray new CoalescedRDDPartition(i, prev, ids, pg.prefLoc) @@ -126,7 +126,7 @@ class CoalescedRDD[T: ClassManifest]( } -class coalescePartitions protected (maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { +private[spark] class PartitionCoalescer (maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { protected def compare(o1: PartitionGroup, o2: PartitionGroup): Boolean = o1.size < o2.size protected def compare(o1: Option[PartitionGroup], o2: Option[PartitionGroup]): Boolean = @@ -303,21 +303,22 @@ class coalescePartitions protected (maxPartitions: Int, prev: RDD[_], balanceSla } } - def getPartitions : Array[PartitionGroup] = groupArr.filter( pg => pg.size > 0).toArray + protected def getPartitions : Array[PartitionGroup] = groupArr.filter( pg => pg.size > 0).toArray + /** + * Runs the packing algorithm and returns an array of PartitionGroups that if possible are + * load balanced and grouped by locality + * @return array of partition groups + */ + def run() : Array[PartitionGroup] = { + setupGroups(math.min(prev.partitions.length, maxPartitions)) // setup the groups (bins) + throwBalls() // assign partitions (balls) to each group (bins) + getPartitions + } } -case class PartitionGroup(prefLoc: String = "") { +private[spark] case class PartitionGroup(prefLoc: String = "") { var list = mutable.ListBuffer[Partition]() def size = list.size } - -object coalescePartitions { - def apply(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) : Array[PartitionGroup] = { - val pc = new coalescePartitions(maxPartitions, prev, balanceSlack) - pc.setupGroups(math.min(prev.partitions.length, maxPartitions)) // setup the groups (bins) - pc.throwBalls() // assign partitions (balls) to each group (bins) - pc.getPartitions - } -} From 33a0f59354197d667a97c600e2bb8fefe50c181b Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Fri, 16 Aug 2013 12:26:03 -0700 Subject: [PATCH 0805/2521] Added error messages to the tests to make failed tests less cryptic --- core/src/test/scala/spark/RDDSuite.scala | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 8b6fa9c81e9..ac406b94477 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -178,20 +178,20 @@ class RDDSuite extends FunSuite with SharedSparkContext { // RDD with locality preferences spread (non-randomly) over 6 machines, m0 through m5 val data = sc.makeRDD((1 to 9).map( i => (i, (i to (i+2)).map{ j => "m" + (j%6)} ))) val coalesced1 = data.coalesce(3) - assert(coalesced1.collect().toList.sorted === (1 to 9).toList) // no data lost + assert(coalesced1.collect().toList.sorted === (1 to 9).toList, "Data got *lost* in coalescing") val splits = coalesced1.glom().collect().map(_.toList).toList - assert(splits.length === 3) // ensure it indeed created 3 partitions + assert(splits.length == 3, "Supposed to coalesce to 3 but got " + splits.length) assert(splits.foldLeft(true) - ( (x,y) => if (!x) false else y.length >= 1) === true) // (1+ balance) + ( (x,y) => if (!x) false else y.length >= 1) === true, "Some partitions were empty") // If we try to coalesce into more partitions than the original RDD, it should just // keep the original number of partitions. val coalesced4 = data.coalesce(20) assert(coalesced4.glom().collect().map(_.toList).toList.sortWith( (x, y) => if (x.isEmpty) false else if (y.isEmpty) true else x(0) < y(0)) === (1 to 9). - map(x => List(x)).toList) + map(x => List(x)).toList, "Tried coalescing 9 partitions to 20 but didn't get 9 back") // large scale experiment @@ -205,20 +205,20 @@ class RDDSuite extends FunSuite with SharedSparkContext { val blocks = (1 to partitions).map( i => (i, (i to (i+2)) .map{ j => machines(rnd.nextInt(machines.size)) } )) - val data2 = sc.makeRDD(blocks) + val data2 = sc.makeRDD(blocks) // .map( i => i*2 ) val coalesced2 = data2.coalesce(numMachines*2) // test that you get over 90% locality in each group val minLocality = coalesced2.partitions .map( part => part.asInstanceOf[CoalescedRDDPartition].localFraction ) .foldLeft(100.)( (perc, loc) => math.min(perc,loc) ) - assert(minLocality > 0.90) + assert(minLocality >= 0.90, "Expected 90% locality but got " + (minLocality*100.).toInt + "%") // test that the groups are load balanced with 100 +/- 20 elements in each val maxImbalance = coalesced2.partitions .map( part => part.asInstanceOf[CoalescedRDDPartition].parents.size ) .foldLeft(0)((dev, curr) => math.max(math.abs(100-curr),dev)) - assert(maxImbalance < 20) + assert(maxImbalance <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance) } test("zipped RDDs") { From 3b5bb8a4ae1ebc0bbfa34c908a99274c343fe883 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Fri, 16 Aug 2013 13:12:11 -0700 Subject: [PATCH 0806/2521] added one test that will test a future functionality --- core/src/test/scala/spark/RDDSuite.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index ac406b94477..ad912633227 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -211,7 +211,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { // test that you get over 90% locality in each group val minLocality = coalesced2.partitions .map( part => part.asInstanceOf[CoalescedRDDPartition].localFraction ) - .foldLeft(100.)( (perc, loc) => math.min(perc,loc) ) + .foldLeft(1.)( (perc, loc) => math.min(perc,loc) ) assert(minLocality >= 0.90, "Expected 90% locality but got " + (minLocality*100.).toInt + "%") // test that the groups are load balanced with 100 +/- 20 elements in each @@ -219,6 +219,15 @@ class RDDSuite extends FunSuite with SharedSparkContext { .map( part => part.asInstanceOf[CoalescedRDDPartition].parents.size ) .foldLeft(0)((dev, curr) => math.max(math.abs(100-curr),dev)) assert(maxImbalance <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance) + + // TDD: Test for later when we have implemented functionality to get locality from DAGScheduler +// val data3 = sc.makeRDD(blocks).map( i => i*2 ) +// val coalesced3 = data3.coalesce(numMachines*2) +// val minLocality2 = coalesced3.partitions +// .map( part => part.asInstanceOf[CoalescedRDDPartition].localFraction ) +// .foldLeft(1.)( (perc, loc) => math.min(perc,loc) ) +// assert(minLocality2 >= 0.90, "Expected 90% locality for derived RDD but got " + +// (minLocality2*100.).toInt + "%") } test("zipped RDDs") { From b69e7166ba76d35d75b98015b0d39a8a004a7436 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Fri, 16 Aug 2013 14:03:45 -0700 Subject: [PATCH 0807/2521] Coalescer now uses current preferred locations for derived RDDs. Made run() in DAGScheduler thread safe and added a method to be able to ask it for preferred locations. Added a similar method that wraps the former inside SparkContext. --- core/src/main/scala/spark/SparkContext.scala | 9 +++++ .../main/scala/spark/rdd/CoalescedRDD.scala | 30 +++++++++----- .../scala/spark/scheduler/DAGScheduler.scala | 40 +++++++++++-------- core/src/test/scala/spark/RDDSuite.scala | 14 +++---- 4 files changed, 59 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index fdd2dfa810a..544c971efc5 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -614,6 +614,15 @@ class SparkContext( addedFiles.clear() } + /** + * Gets the locality information associated with the partition in a particular rdd + * @param rdd of interest + * @param partition to be looked up for locality + * @return list of preferred locations for the partition + */ + def getPreferredLocs(rdd: RDD[_], partition: Int): List[String] = + dagScheduler.getPreferredLocs(rdd, partition) + /** * Adds a JAR dependency for all tasks to be executed on this SparkContext in the future. * The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index e6fe8ec8ee5..01d4bcadc2f 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -17,9 +17,11 @@ package spark.rdd -import spark.{Dependency, OneToOneDependency, NarrowDependency, RDD, Partition, TaskContext} +import spark._ import java.io.{ObjectOutputStream, IOException} import scala.collection.mutable +import scala.Some +import spark.rdd.CoalescedRDDPartition private[spark] case class CoalescedRDDPartition( index: Int, @@ -42,7 +44,8 @@ private[spark] case class CoalescedRDDPartition( * @return locality of this coalesced partition between 0 and 1 */ def localFraction: Double = { - val loc = parents.count(p => rdd.preferredLocations(p).contains(preferredLocation)) + val loc = parents.count(p => rdd.context.getPreferredLocs(rdd, p.index) + .contains(preferredLocation)) if (parents.size == 0) 0.0 else (loc.toDouble / parents.size.toDouble) } } @@ -126,8 +129,8 @@ class CoalescedRDD[T: ClassManifest]( } -private[spark] class PartitionCoalescer (maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { - +private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], + balanceSlack: Double) { protected def compare(o1: PartitionGroup, o2: PartitionGroup): Boolean = o1.size < o2.size protected def compare(o1: Option[PartitionGroup], o2: Option[PartitionGroup]): Boolean = if (o1 == None) false else if (o2 == None) true else compare(o1.get, o2.get) @@ -149,6 +152,10 @@ private[spark] class PartitionCoalescer (maxPartitions: Int, prev: RDD[_], balan protected var noLocality = true // if true if no preferredLocations exists for parent RDD + // gets the *current* preferred locations from the DAGScheduler (as opposed to the static ones) + protected def currentPreferredLocations(rdd: RDD[_], part: Partition) : Seq[String] = + rdd.context.getPreferredLocs(rdd, part.index) + // this class just keeps iterating and rotating infinitely over the partitions of the RDD // next() returns the next preferred machine that a partition is replicated on // the rotator first goes through the first replica copy of each partition, then second, third @@ -161,14 +168,14 @@ private[spark] class PartitionCoalescer (maxPartitions: Int, prev: RDD[_], balan // initializes/resets to start iterating from the beginning protected def resetIterator() = { val i1 = prev.partitions.view.map{ p: Partition => - { if (prev.preferredLocations(p).length > 0) - Some((prev.preferredLocations(p)(0),p)) else None } } + { if (currentPreferredLocations(prev, p).length > 0) + Some((currentPreferredLocations(prev, p)(0),p)) else None } } val i2 = prev.partitions.view.map{ p: Partition => - { if (prev.preferredLocations(p).length > 1) - Some((prev.preferredLocations(p)(1),p)) else None } } + { if (currentPreferredLocations(prev, p).length > 1) + Some((currentPreferredLocations(prev, p)(1),p)) else None } } val i3 = prev.partitions.view.map{ p: Partition => - { if (prev.preferredLocations(p).length > 2) - Some((prev.preferredLocations(p)(2),p)) else None } } + { if (currentPreferredLocations(prev, p).length > 2) + Some((currentPreferredLocations(prev, p)(2),p)) else None } } val res = List(i1,i2,i3) res.view.flatMap(x => x).flatten.iterator // fuses the 3 iterators (1st replica, 2nd, 3rd) } @@ -265,7 +272,8 @@ private[spark] class PartitionCoalescer (maxPartitions: Int, prev: RDD[_], balan * @return partition group (bin to be put in) */ protected def pickBin(p: Partition): PartitionGroup = { - val pref = prev.preferredLocations(p).map(getLeastGroupHash(_)).sortWith(compare) // least load + val pref = prev.context.getPreferredLocs(prev, p.index). + map(getLeastGroupHash(_)).sortWith(compare) // least loaded of the pref locations val prefPart = if (pref == Nil) None else pref.head val r1 = rnd.nextInt(groupArr.size) diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala index 9402f18a0f6..7275bd346ad 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala @@ -435,23 +435,24 @@ class DAGScheduler( if (event != null) { logDebug("Got event of type " + event.getClass.getName) } - - if (event != null) { - if (processEvent(event)) { - return + this.synchronized { // needed in case other threads makes calls into methods of this class + if (event != null) { + if (processEvent(event)) { + return + } } - } - val time = System.currentTimeMillis() // TODO: use a pluggable clock for testability - // Periodically resubmit failed stages if some map output fetches have failed and we have - // waited at least RESUBMIT_TIMEOUT. We wait for this short time because when a node fails, - // tasks on many other nodes are bound to get a fetch failure, and they won't all get it at - // the same time, so we want to make sure we've identified all the reduce tasks that depend - // on the failed node. - if (failed.size > 0 && time > lastFetchFailureTime + RESUBMIT_TIMEOUT) { - resubmitFailedStages() - } else { - submitWaitingStages() + val time = System.currentTimeMillis() // TODO: use a pluggable clock for testability + // Periodically resubmit failed stages if some map output fetches have failed and we have + // waited at least RESUBMIT_TIMEOUT. We wait for this short time because when a node fails, + // tasks on many other nodes are bound to get a fetch failure, and they won't all get it at + // the same time, so we want to make sure we've identified all the reduce tasks that depend + // on the failed node. + if (failed.size > 0 && time > lastFetchFailureTime + RESUBMIT_TIMEOUT) { + resubmitFailedStages() + } else { + submitWaitingStages() + } } } } @@ -789,7 +790,14 @@ class DAGScheduler( visitedRdds.contains(target.rdd) } - private def getPreferredLocs(rdd: RDD[_], partition: Int): Seq[TaskLocation] = { + /** + * Synchronized method that might be called from other threads. + * @param rdd whose partitions are to be looked at + * @param partition to lookup locality information for + * @return list of machines that are preferred by the partition + */ + private[spark] + def getPreferredLocs(rdd: RDD[_], partition: Int): Seq[TaskLocation] = synchronized { // If the partition is cached, return the cache locations val cached = getCacheLocs(rdd)(partition) if (!cached.isEmpty) { diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index ad912633227..0532435288d 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -221,13 +221,13 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(maxImbalance <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance) // TDD: Test for later when we have implemented functionality to get locality from DAGScheduler -// val data3 = sc.makeRDD(blocks).map( i => i*2 ) -// val coalesced3 = data3.coalesce(numMachines*2) -// val minLocality2 = coalesced3.partitions -// .map( part => part.asInstanceOf[CoalescedRDDPartition].localFraction ) -// .foldLeft(1.)( (perc, loc) => math.min(perc,loc) ) -// assert(minLocality2 >= 0.90, "Expected 90% locality for derived RDD but got " + -// (minLocality2*100.).toInt + "%") + val data3 = sc.makeRDD(blocks).map( i => i*2 ) + val coalesced3 = data3.coalesce(numMachines*2) + val minLocality2 = coalesced3.partitions + .map( part => part.asInstanceOf[CoalescedRDDPartition].localFraction ) + .foldLeft(1.)( (perc, loc) => math.min(perc,loc) ) + assert(minLocality2 >= 0.90, "Expected 90% locality for derived RDD but got " + + (minLocality2*100.).toInt + "%") } test("zipped RDDs") { From d6b6c680be7d8dcd186b42dbd6899de67852a8d9 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Fri, 16 Aug 2013 14:14:50 -0700 Subject: [PATCH 0808/2521] comment in the test to make it more understandable --- core/src/test/scala/spark/RDDSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 0532435288d..da5f78aef5c 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -221,7 +221,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(maxImbalance <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance) // TDD: Test for later when we have implemented functionality to get locality from DAGScheduler - val data3 = sc.makeRDD(blocks).map( i => i*2 ) + val data3 = sc.makeRDD(blocks).map( i => i*2 ) // derived RDD to test *current* pref locs val coalesced3 = data3.coalesce(numMachines*2) val minLocality2 = coalesced3.partitions .map( part => part.asInstanceOf[CoalescedRDDPartition].localFraction ) From 890ea6ba792f9d3d07916a0833c3a83f0150e8cc Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Fri, 16 Aug 2013 17:52:53 -0700 Subject: [PATCH 0809/2521] making CoalescedRDDPartition public --- core/src/main/scala/spark/rdd/CoalescedRDD.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 01d4bcadc2f..a5e3d744474 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -21,9 +21,8 @@ import spark._ import java.io.{ObjectOutputStream, IOException} import scala.collection.mutable import scala.Some -import spark.rdd.CoalescedRDDPartition -private[spark] case class CoalescedRDDPartition( +case class CoalescedRDDPartition( index: Int, @transient rdd: RDD[_], parentsIndices: Array[Int], From f1c853d76dce4fbc34f580be0a3ae15cc5be9c80 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Sun, 18 Aug 2013 20:42:35 -0700 Subject: [PATCH 0810/2521] fixed Matei's comments --- core/src/main/scala/spark/SparkContext.scala | 2 +- .../main/scala/spark/rdd/CoalescedRDD.scala | 145 ++++++++++-------- core/src/test/scala/spark/RDDSuite.scala | 25 +-- 3 files changed, 99 insertions(+), 73 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 544c971efc5..7639749ecbd 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -620,7 +620,7 @@ class SparkContext( * @param partition to be looked up for locality * @return list of preferred locations for the partition */ - def getPreferredLocs(rdd: RDD[_], partition: Int): List[String] = + private [spark] def getPreferredLocs(rdd: RDD[_], partition: Int): List[String] = dagScheduler.getPreferredLocs(rdd, partition) /** diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index a5e3d744474..c475b7a8aa9 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -21,13 +21,21 @@ import spark._ import java.io.{ObjectOutputStream, IOException} import scala.collection.mutable import scala.Some +import scala.collection.mutable.ArrayBuffer +/** + * Class that captures a coalesced RDD by essentially keeping track of parent partitions + * @param index of this coalesced partition + * @param rdd which it belongs to + * @param parentsIndices list of indices in the parent that have been coalesced into this partition + * @param preferredLocation the preferred location for this partition + */ case class CoalescedRDDPartition( - index: Int, - @transient rdd: RDD[_], - parentsIndices: Array[Int], - @transient preferredLocation: String = "" - ) extends Partition { + index: Int, + @transient rdd: RDD[_], + parentsIndices: Array[Int], + @transient preferredLocation: String = "" + ) extends Partition { var parents: Seq[Partition] = parentsIndices.map(rdd.partitions(_)) @throws(classOf[IOException]) @@ -43,45 +51,27 @@ case class CoalescedRDDPartition( * @return locality of this coalesced partition between 0 and 1 */ def localFraction: Double = { - val loc = parents.count(p => rdd.context.getPreferredLocs(rdd, p.index) - .contains(preferredLocation)) + val loc = parents.count(p => + rdd.context.getPreferredLocs(rdd, p.index).contains(preferredLocation)) + if (parents.size == 0) 0.0 else (loc.toDouble / parents.size.toDouble) } } /** - * Coalesce the partitions of a parent RDD (`prev`) into fewer partitions, so that each partition of - * this RDD computes one or more of the parent ones. Will produce exactly `maxPartitions` if the - * parent had more than this many partitions, or fewer if the parent had fewer. - * - * This transformation is useful when an RDD with many partitions gets filtered into a smaller one, - * or to avoid having a large number of small tasks when processing a directory with many files. - * - * If there is no locality information (no preferredLocations) in the parent, then the coalescing - * is very simple: chunk parents that are close in the Array in chunks. - * If there is locality information, it proceeds to pack them with the following three goals: - * - * (1) Balance the groups so they roughly have the same number of parent partitions - * (2) Achieve locality per partition, i.e. find one machine which most parent partitions prefer - * (3) Be efficient, i.e. O(n) algorithm for n parent partitions (problem is likely NP-hard) - * (4) Balance preferred machines, i.e. avoid as much as possible picking the same preferred machine - * - * Furthermore, it is assumed that the parent RDD may have many partitions, e.g. 100 000. - * We assume the final number of desired partitions is small, e.g. less than 1000. - * - * The algorithm tries to assign unique preferred machines to each partition. If the number of - * desired partitions is greater than the number of preferred machines (can happen), it needs to - * start picking duplicate preferred machines. This is determined using coupon collector estimation - * (2n log(n)). The load balancing is done using power-of-two randomized bins-balls with one twist: - * it tries to also achieve locality. This is done by allowing a slack (balanceSlack) between two - * bins. If two bins are within the slack in terms of balance, the algorithm will assign partitions - * according to locality. (contact alig for questions) - * + * Represents a coalesced RDD that has fewer partitions than its parent RDD + * This class uses the PartitionCoalescer class to find a good partitioning of the parent RDD + * so that each new partition has roughly the same number of parent partitions and that + * the preferred location of each new partition overlaps with as many preferred locations of its + * parent partitions + * @param prev RDD to be coalesced + * @param maxPartitions number of desired partitions in the coalesced RDD + * @param balanceSlack used to trade-off balance and locality. 1.0 is all locality, 0 is all balance */ class CoalescedRDD[T: ClassManifest]( - @transient var prev: RDD[T], - maxPartitions: Int, - balanceSlack: Double = 0.10) + @transient var prev: RDD[T], + maxPartitions: Int, + balanceSlack: Double = 0.10) extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies override def getPartitions: Array[Partition] = { @@ -128,44 +118,75 @@ class CoalescedRDD[T: ClassManifest]( } -private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], - balanceSlack: Double) { - protected def compare(o1: PartitionGroup, o2: PartitionGroup): Boolean = o1.size < o2.size - protected def compare(o1: Option[PartitionGroup], o2: Option[PartitionGroup]): Boolean = +/** + * Coalesce the partitions of a parent RDD (`prev`) into fewer partitions, so that each partition of + * this RDD computes one or more of the parent ones. Will produce exactly `maxPartitions` if the + * parent had more than this many partitions, or fewer if the parent had fewer. + * + * This transformation is useful when an RDD with many partitions gets filtered into a smaller one, + * or to avoid having a large number of small tasks when processing a directory with many files. + * + * If there is no locality information (no preferredLocations) in the parent, then the coalescing + * is very simple: chunk parents that are close in the Array in chunks. + * If there is locality information, it proceeds to pack them with the following three goals: + * + * (1) Balance the groups so they roughly have the same number of parent partitions + * (2) Achieve locality per partition, i.e. find one machine which most parent partitions prefer + * (3) Be efficient, i.e. O(n) algorithm for n parent partitions (problem is likely NP-hard) + * (4) Balance preferred machines, i.e. avoid as much as possible picking the same preferred machine + * + * Furthermore, it is assumed that the parent RDD may have many partitions, e.g. 100 000. + * We assume the final number of desired partitions is small, e.g. less than 1000. + * + * The algorithm tries to assign unique preferred machines to each partition. If the number of + * desired partitions is greater than the number of preferred machines (can happen), it needs to + * start picking duplicate preferred machines. This is determined using coupon collector estimation + * (2n log(n)). The load balancing is done using power-of-two randomized bins-balls with one twist: + * it tries to also achieve locality. This is done by allowing a slack (balanceSlack) between two + * bins. If two bins are within the slack in terms of balance, the algorithm will assign partitions + * according to locality. (contact alig for questions) + * + */ + +private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { + + def compare(o1: PartitionGroup, o2: PartitionGroup): Boolean = o1.size < o2.size + def compare(o1: Option[PartitionGroup], o2: Option[PartitionGroup]): Boolean = if (o1 == None) false else if (o2 == None) true else compare(o1.get, o2.get) - protected val rnd = new scala.util.Random(7919) // keep this class deterministic + val rnd = new scala.util.Random(7919) // keep this class deterministic // each element of groupArr represents one coalesced partition - protected val groupArr = mutable.ArrayBuffer[PartitionGroup]() + val groupArr = ArrayBuffer[PartitionGroup]() // hash used to check whether some machine is already in groupArr - protected val groupHash = mutable.Map[String, mutable.ListBuffer[PartitionGroup]]() + val groupHash = mutable.Map[String, ArrayBuffer[PartitionGroup]]() // hash used for the first maxPartitions (to avoid duplicates) - protected val initialHash = mutable.Map[Partition, Boolean]() + val initialHash = mutable.Set[Partition]() // determines the tradeoff between load-balancing the partitions sizes and their locality // e.g. balanceSlack=0.10 means that it allows up to 10% imbalance in favor of locality - protected val slack = (balanceSlack * prev.partitions.size).toInt + val slack = (balanceSlack * prev.partitions.size).toInt - protected var noLocality = true // if true if no preferredLocations exists for parent RDD + var noLocality = true // if true if no preferredLocations exists for parent RDD // gets the *current* preferred locations from the DAGScheduler (as opposed to the static ones) - protected def currentPreferredLocations(rdd: RDD[_], part: Partition) : Seq[String] = + def currentPreferredLocations(rdd: RDD[_], part: Partition) : Seq[String] = rdd.context.getPreferredLocs(rdd, part.index) // this class just keeps iterating and rotating infinitely over the partitions of the RDD // next() returns the next preferred machine that a partition is replicated on // the rotator first goes through the first replica copy of each partition, then second, third // the iterators return type is a tuple: (replicaString, partition) - protected class RotateLocations(prev: RDD[_]) extends Iterator[(String, Partition)] { + class LocationIterator(prev: RDD[_]) extends Iterator[(String, Partition)] { + + var it: Iterator[(String, Partition)] = resetIterator() - protected var it: Iterator[(String, Partition)] = resetIterator() override val isEmpty = !it.hasNext // initializes/resets to start iterating from the beginning - protected def resetIterator() = { + def resetIterator() = { val i1 = prev.partitions.view.map{ p: Partition => { if (currentPreferredLocations(prev, p).length > 0) Some((currentPreferredLocations(prev, p)(0),p)) else None } } @@ -177,10 +198,14 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], Some((currentPreferredLocations(prev, p)(2),p)) else None } } val res = List(i1,i2,i3) res.view.flatMap(x => x).flatten.iterator // fuses the 3 iterators (1st replica, 2nd, 3rd) + + // prev.partitions.iterator.map{p: Partition => { (currentPreferredLocations(prev, p)(0),p) }} ++ + // prev.partitions.iterator.map{p: Partition => { (currentPreferredLocations(prev, p)(1),p) }} ++ + // prev.partitions.iterator.map{p: Partition => { (currentPreferredLocations(prev, p)(2),p) }} } // hasNext() is false iff there are no preferredLocations for any of the partitions of the RDD - def hasNext(): Boolean = !isEmpty + def hasNext(): Boolean = { !isEmpty } // return the next preferredLocation of some partition of the RDD def next(): (String, Partition) = { @@ -199,14 +224,14 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], * @param key string representing a partitioned group on preferred machine key * @return Option of PartitionGroup that has least elements for key */ - protected def getLeastGroupHash(key: String): Option[PartitionGroup] = { + def getLeastGroupHash(key: String): Option[PartitionGroup] = { groupHash.get(key).map(_.sortWith(compare).head) } def addPartToPGroup(part : Partition, pgroup : PartitionGroup) : Boolean = { if (!initialHash.contains(part)) { pgroup.list += part // already assign this element - initialHash += (part -> true) // needed to avoid assigning partitions to multiple buckets + initialHash += part // needed to avoid assigning partitions to multiple buckets true } else { false } } @@ -217,8 +242,8 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], * until it has seen most of the preferred locations (2 * n log(n)) * @param targetLen */ - protected def setupGroups(targetLen: Int) { - val rotIt = new RotateLocations(prev) + def setupGroups(targetLen: Int) { + val rotIt = new LocationIterator(prev) // deal with empty case, just create targetLen partition groups with no preferred location if (!rotIt.hasNext()) { @@ -243,7 +268,7 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], val pgroup = PartitionGroup(nxt_replica) groupArr += pgroup addPartToPGroup(nxt_part, pgroup) - groupHash += (nxt_replica -> (mutable.ListBuffer(pgroup))) // list in case we have multiple + groupHash += (nxt_replica -> (ArrayBuffer(pgroup))) // list in case we have multiple numCreated += 1 } } @@ -270,7 +295,7 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], * @param p partition (ball to be thrown) * @return partition group (bin to be put in) */ - protected def pickBin(p: Partition): PartitionGroup = { + def pickBin(p: Partition): PartitionGroup = { val pref = prev.context.getPreferredLocs(prev, p.index). map(getLeastGroupHash(_)).sortWith(compare) // least loaded of the pref locations val prefPart = if (pref == Nil) None else pref.head @@ -290,7 +315,7 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], } } - protected def throwBalls() { + def throwBalls() { if (noLocality) { // no preferredLocations in parent RDD, no randomization needed if (maxPartitions > groupArr.size) { // just return prev.partitions for ((p,i) <- prev.partitions.zipWithIndex) { @@ -310,7 +335,7 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], } } - protected def getPartitions : Array[PartitionGroup] = groupArr.filter( pg => pg.size > 0).toArray + def getPartitions : Array[PartitionGroup] = groupArr.filter( pg => pg.size > 0).toArray /** * Runs the packing algorithm and returns an array of PartitionGroups that if possible are diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index da5f78aef5c..14f7c627825 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -176,7 +176,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { } test("cogrouped RDDs with locality") { // RDD with locality preferences spread (non-randomly) over 6 machines, m0 through m5 - val data = sc.makeRDD((1 to 9).map( i => (i, (i to (i+2)).map{ j => "m" + (j%6)} ))) + val data = sc.makeRDD((1 to 9).map(i => (i, (i to (i+2)).map{ j => "m" + (j%6)}))) val coalesced1 = data.coalesce(3) assert(coalesced1.collect().toList.sorted === (1 to 9).toList, "Data got *lost* in coalescing") @@ -184,7 +184,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(splits.length == 3, "Supposed to coalesce to 3 but got " + splits.length) assert(splits.foldLeft(true) - ( (x,y) => if (!x) false else y.length >= 1) === true, "Some partitions were empty") + ((x,y) => if (!x) false else y.length >= 1) === true, "Some partitions were empty") // If we try to coalesce into more partitions than the original RDD, it should just // keep the original number of partitions. @@ -193,7 +193,9 @@ class RDDSuite extends FunSuite with SharedSparkContext { (x, y) => if (x.isEmpty) false else if (y.isEmpty) true else x(0) < y(0)) === (1 to 9). map(x => List(x)).toList, "Tried coalescing 9 partitions to 20 but didn't get 9 back") + } + test("cogrouped RDDs with locality, large scale (10K partitions)") { // large scale experiment import collection.mutable val rnd = scala.util.Random @@ -202,30 +204,29 @@ class RDDSuite extends FunSuite with SharedSparkContext { val machines = mutable.ListBuffer[String]() (1 to numMachines).foreach(machines += "m"+_) - val blocks = (1 to partitions).map( i => (i, (i to (i+2)) - .map{ j => machines(rnd.nextInt(machines.size)) } )) + val blocks = (1 to partitions).map(i => (i, (i to (i+2)) + .map{ j => machines(rnd.nextInt(machines.size))})) - val data2 = sc.makeRDD(blocks) // .map( i => i*2 ) + val data2 = sc.makeRDD(blocks) val coalesced2 = data2.coalesce(numMachines*2) // test that you get over 90% locality in each group val minLocality = coalesced2.partitions - .map( part => part.asInstanceOf[CoalescedRDDPartition].localFraction ) - .foldLeft(1.)( (perc, loc) => math.min(perc,loc) ) + .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) + .foldLeft(1.)((perc, loc) => math.min(perc,loc)) assert(minLocality >= 0.90, "Expected 90% locality but got " + (minLocality*100.).toInt + "%") // test that the groups are load balanced with 100 +/- 20 elements in each val maxImbalance = coalesced2.partitions - .map( part => part.asInstanceOf[CoalescedRDDPartition].parents.size ) + .map(part => part.asInstanceOf[CoalescedRDDPartition].parents.size) .foldLeft(0)((dev, curr) => math.max(math.abs(100-curr),dev)) assert(maxImbalance <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance) - // TDD: Test for later when we have implemented functionality to get locality from DAGScheduler - val data3 = sc.makeRDD(blocks).map( i => i*2 ) // derived RDD to test *current* pref locs + val data3 = sc.makeRDD(blocks).map(i => i*2) // derived RDD to test *current* pref locs val coalesced3 = data3.coalesce(numMachines*2) val minLocality2 = coalesced3.partitions - .map( part => part.asInstanceOf[CoalescedRDDPartition].localFraction ) - .foldLeft(1.)( (perc, loc) => math.min(perc,loc) ) + .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) + .foldLeft(1.)((perc, loc) => math.min(perc,loc)) assert(minLocality2 >= 0.90, "Expected 90% locality for derived RDD but got " + (minLocality2*100.).toInt + "%") } From a75a64eade3f540e72fc0bc646ba74a689f03f3e Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Mon, 19 Aug 2013 12:09:08 -0700 Subject: [PATCH 0811/2521] Fixed almost all of Matei's feedback --- .../main/scala/spark/rdd/CoalescedRDD.scala | 43 ++++++++----------- core/src/test/scala/spark/RDDSuite.scala | 14 +++--- 2 files changed, 26 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index c475b7a8aa9..04bb8089a49 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -79,7 +79,7 @@ class CoalescedRDD[T: ClassManifest]( pc.run().zipWithIndex.map { case (pg, i) => - val ids = pg.list.map(_.index).toArray + val ids = pg.arr.map(_.index).toArray new CoalescedRDDPartition(i, prev, ids, pg.prefLoc) } } @@ -109,26 +109,21 @@ class CoalescedRDD[T: ClassManifest]( * @return the machine most preferred by split */ override def getPreferredLocations(partition: Partition): Seq[String] = { - if (partition.isInstanceOf[CoalescedRDDPartition]) { - List(partition.asInstanceOf[CoalescedRDDPartition].preferredLocation) - } else { - super.getPreferredLocations(partition) - } + List(partition.asInstanceOf[CoalescedRDDPartition].preferredLocation) } - } /** * Coalesce the partitions of a parent RDD (`prev`) into fewer partitions, so that each partition of - * this RDD computes one or more of the parent ones. Will produce exactly `maxPartitions` if the - * parent had more than this many partitions, or fewer if the parent had fewer. + * this RDD computes one or more of the parent ones. It will produce exactly `maxPartitions` if the + * parent had more than maxPartitions, or fewer if the parent had fewer. * * This transformation is useful when an RDD with many partitions gets filtered into a smaller one, * or to avoid having a large number of small tasks when processing a directory with many files. * * If there is no locality information (no preferredLocations) in the parent, then the coalescing * is very simple: chunk parents that are close in the Array in chunks. - * If there is locality information, it proceeds to pack them with the following three goals: + * If there is locality information, it proceeds to pack them with the following four goals: * * (1) Balance the groups so they roughly have the same number of parent partitions * (2) Achieve locality per partition, i.e. find one machine which most parent partitions prefer @@ -172,7 +167,7 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc var noLocality = true // if true if no preferredLocations exists for parent RDD // gets the *current* preferred locations from the DAGScheduler (as opposed to the static ones) - def currentPreferredLocations(rdd: RDD[_], part: Partition) : Seq[String] = + def currentPreferredLocations(rdd: RDD[_], part: Partition): Seq[String] = rdd.context.getPreferredLocs(rdd, part.index) // this class just keeps iterating and rotating infinitely over the partitions of the RDD @@ -199,9 +194,9 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc val res = List(i1,i2,i3) res.view.flatMap(x => x).flatten.iterator // fuses the 3 iterators (1st replica, 2nd, 3rd) - // prev.partitions.iterator.map{p: Partition => { (currentPreferredLocations(prev, p)(0),p) }} ++ - // prev.partitions.iterator.map{p: Partition => { (currentPreferredLocations(prev, p)(1),p) }} ++ - // prev.partitions.iterator.map{p: Partition => { (currentPreferredLocations(prev, p)(2),p) }} + // prev.partitions.iterator.map(p: Partition => { (currentPreferredLocations(prev, p)(0),p) }) ++ + // prev.partitions.iterator.map(p: Partition => { (currentPreferredLocations(prev, p)(1),p) }) ++ + // prev.partitions.iterator.map(p: Partition => { (currentPreferredLocations(prev, p)(2),p) }) } // hasNext() is false iff there are no preferredLocations for any of the partitions of the RDD @@ -228,9 +223,9 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc groupHash.get(key).map(_.sortWith(compare).head) } - def addPartToPGroup(part : Partition, pgroup : PartitionGroup) : Boolean = { + def addPartToPGroup(part: Partition, pgroup: PartitionGroup): Boolean = { if (!initialHash.contains(part)) { - pgroup.list += part // already assign this element + pgroup.arr += part // already assign this element initialHash += part // needed to avoid assigning partitions to multiple buckets true } else { false } @@ -319,30 +314,30 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc if (noLocality) { // no preferredLocations in parent RDD, no randomization needed if (maxPartitions > groupArr.size) { // just return prev.partitions for ((p,i) <- prev.partitions.zipWithIndex) { - groupArr(i).list += p + groupArr(i).arr += p } } else { // no locality available, then simply split partitions based on positions in array - (0 until maxPartitions).foreach { i => + for(i <- 0 until maxPartitions) { val rangeStart = ((i.toLong * prev.partitions.length) / maxPartitions).toInt val rangeEnd = (((i.toLong + 1) * prev.partitions.length) / maxPartitions).toInt - (rangeStart until rangeEnd).foreach{ j => groupArr(i).list += prev.partitions(j) } + (rangeStart until rangeEnd).foreach{ j => groupArr(i).arr += prev.partitions(j) } } } } else { for (p <- prev.partitions if (!initialHash.contains(p))) { // throw every partition into group - pickBin(p).list += p + pickBin(p).arr += p } } } - def getPartitions : Array[PartitionGroup] = groupArr.filter( pg => pg.size > 0).toArray + def getPartitions: Array[PartitionGroup] = groupArr.filter( pg => pg.size > 0).toArray /** * Runs the packing algorithm and returns an array of PartitionGroups that if possible are * load balanced and grouped by locality * @return array of partition groups */ - def run() : Array[PartitionGroup] = { + def run(): Array[PartitionGroup] = { setupGroups(math.min(prev.partitions.length, maxPartitions)) // setup the groups (bins) throwBalls() // assign partitions (balls) to each group (bins) getPartitions @@ -350,7 +345,7 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc } private[spark] case class PartitionGroup(prefLoc: String = "") { - var list = mutable.ListBuffer[Partition]() + var arr = mutable.ArrayBuffer[Partition]() - def size = list.size + def size = arr.size } diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 14f7c627825..64e2c0605b2 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -181,7 +181,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(coalesced1.collect().toList.sorted === (1 to 9).toList, "Data got *lost* in coalescing") val splits = coalesced1.glom().collect().map(_.toList).toList - assert(splits.length == 3, "Supposed to coalesce to 3 but got " + splits.length) + assert(splits.length === 3, "Supposed to coalesce to 3 but got " + splits.length) assert(splits.foldLeft(true) ((x,y) => if (!x) false else y.length >= 1) === true, "Some partitions were empty") @@ -189,10 +189,10 @@ class RDDSuite extends FunSuite with SharedSparkContext { // If we try to coalesce into more partitions than the original RDD, it should just // keep the original number of partitions. val coalesced4 = data.coalesce(20) - assert(coalesced4.glom().collect().map(_.toList).toList.sortWith( - (x, y) => if (x.isEmpty) false else if (y.isEmpty) true else x(0) < y(0)) === (1 to 9). - map(x => List(x)).toList, "Tried coalescing 9 partitions to 20 but didn't get 9 back") - + val listOfLists = coalesced4.glom().collect().map(_.toList).toList + val sortedList = listOfLists.sortWith{ (x, y) => !x.isEmpty && (y.isEmpty || (x(0) < y(0))) } + assert( sortedList === (1 to 9). + map{x => List(x)}.toList, "Tried coalescing 9 partitions to 20 but didn't get 9 back") } test("cogrouped RDDs with locality, large scale (10K partitions)") { @@ -204,8 +204,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { val machines = mutable.ListBuffer[String]() (1 to numMachines).foreach(machines += "m"+_) - val blocks = (1 to partitions).map(i => (i, (i to (i+2)) - .map{ j => machines(rnd.nextInt(machines.size))})) + val blocks = (1 to partitions).map(i => + { (i, Array.fill(3)(machines(rnd.nextInt(machines.size))).toList) } ) val data2 = sc.makeRDD(blocks) val coalesced2 = data2.coalesce(numMachines*2) From 9192c358e40f2b3954d9939d7e153e3dd4d4ba75 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Mon, 19 Aug 2013 13:13:24 -0700 Subject: [PATCH 0812/2521] simpler code --- .../main/scala/spark/rdd/CoalescedRDD.scala | 23 ++++++------------- 1 file changed, 7 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 04bb8089a49..25c48caad38 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -167,7 +167,7 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc var noLocality = true // if true if no preferredLocations exists for parent RDD // gets the *current* preferred locations from the DAGScheduler (as opposed to the static ones) - def currentPreferredLocations(rdd: RDD[_], part: Partition): Seq[String] = + def currPrefLocs(rdd: RDD[_], part: Partition): Seq[String] = rdd.context.getPreferredLocs(rdd, part.index) // this class just keeps iterating and rotating infinitely over the partitions of the RDD @@ -182,21 +182,12 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc // initializes/resets to start iterating from the beginning def resetIterator() = { - val i1 = prev.partitions.view.map{ p: Partition => - { if (currentPreferredLocations(prev, p).length > 0) - Some((currentPreferredLocations(prev, p)(0),p)) else None } } - val i2 = prev.partitions.view.map{ p: Partition => - { if (currentPreferredLocations(prev, p).length > 1) - Some((currentPreferredLocations(prev, p)(1),p)) else None } } - val i3 = prev.partitions.view.map{ p: Partition => - { if (currentPreferredLocations(prev, p).length > 2) - Some((currentPreferredLocations(prev, p)(2),p)) else None } } - val res = List(i1,i2,i3) - res.view.flatMap(x => x).flatten.iterator // fuses the 3 iterators (1st replica, 2nd, 3rd) - - // prev.partitions.iterator.map(p: Partition => { (currentPreferredLocations(prev, p)(0),p) }) ++ - // prev.partitions.iterator.map(p: Partition => { (currentPreferredLocations(prev, p)(1),p) }) ++ - // prev.partitions.iterator.map(p: Partition => { (currentPreferredLocations(prev, p)(2),p) }) + val iterators = (0 to 2).map( x => + prev.partitions.iterator.flatMap(p => { + if (currPrefLocs(prev, p).size > x) Some((currPrefLocs(prev, p)(x), p)) else None + } ) + ) + iterators.reduceLeft((x, y) => x ++ y) } // hasNext() is false iff there are no preferredLocations for any of the partitions of the RDD From 7b123b3126d555237f31a0787411c4bbc1abd39a Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Mon, 19 Aug 2013 18:52:43 -0700 Subject: [PATCH 0813/2521] Simpler code --- core/src/main/scala/spark/rdd/CoalescedRDD.scala | 6 +++--- core/src/test/scala/spark/RDDSuite.scala | 3 +-- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 25c48caad38..b83d443de38 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -167,8 +167,8 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc var noLocality = true // if true if no preferredLocations exists for parent RDD // gets the *current* preferred locations from the DAGScheduler (as opposed to the static ones) - def currPrefLocs(rdd: RDD[_], part: Partition): Seq[String] = - rdd.context.getPreferredLocs(rdd, part.index) + def currPrefLocs(part: Partition): Seq[String] = + prev.context.getPreferredLocs(prev, part.index) // this class just keeps iterating and rotating infinitely over the partitions of the RDD // next() returns the next preferred machine that a partition is replicated on @@ -184,7 +184,7 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc def resetIterator() = { val iterators = (0 to 2).map( x => prev.partitions.iterator.flatMap(p => { - if (currPrefLocs(prev, p).size > x) Some((currPrefLocs(prev, p)(x), p)) else None + if (currPrefLocs(p).size > x) Some((currPrefLocs(p)(x), p)) else None } ) ) iterators.reduceLeft((x, y) => x ++ y) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 64e2c0605b2..9e737033719 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -183,8 +183,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { val splits = coalesced1.glom().collect().map(_.toList).toList assert(splits.length === 3, "Supposed to coalesce to 3 but got " + splits.length) - assert(splits.foldLeft(true) - ((x,y) => if (!x) false else y.length >= 1) === true, "Some partitions were empty") + assert(splits.forall(_.length >= 1) === true, "Some partitions were empty") // If we try to coalesce into more partitions than the original RDD, it should just // keep the original number of partitions. From 5db41919b5aafcad050726601c6f3be8e0bf879a Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Mon, 19 Aug 2013 19:11:02 -0700 Subject: [PATCH 0814/2521] Added a test to make sure no locality preferences are ignored --- core/src/test/scala/spark/RDDSuite.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 9e737033719..3c1b387d16f 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -175,6 +175,11 @@ class RDDSuite extends FunSuite with SharedSparkContext { null) } test("cogrouped RDDs with locality") { + val data3 = sc.makeRDD(List((1,List("a","c")), (2,List("a","b","c")), (3,List("b")))) + val coalesced3 = data3.coalesce(3) + val list3 = coalesced3.partitions.map(p => p.asInstanceOf[CoalescedRDDPartition]) + assert(list3.map(p => p.preferredLocation).length === 3, "Locality preferences are dropped") + // RDD with locality preferences spread (non-randomly) over 6 machines, m0 through m5 val data = sc.makeRDD((1 to 9).map(i => (i, (i to (i+2)).map{ j => "m" + (j%6)}))) val coalesced1 = data.coalesce(3) From c0942a710f25c9c690761b0814a07deacd4df595 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Mon, 19 Aug 2013 19:14:30 -0700 Subject: [PATCH 0815/2521] Bug in test fixed --- core/src/test/scala/spark/RDDSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 3c1b387d16f..e306952bbd6 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -176,9 +176,9 @@ class RDDSuite extends FunSuite with SharedSparkContext { } test("cogrouped RDDs with locality") { val data3 = sc.makeRDD(List((1,List("a","c")), (2,List("a","b","c")), (3,List("b")))) - val coalesced3 = data3.coalesce(3) - val list3 = coalesced3.partitions.map(p => p.asInstanceOf[CoalescedRDDPartition]) - assert(list3.map(p => p.preferredLocation).length === 3, "Locality preferences are dropped") + val coal3 = data3.coalesce(3) + val list3 = coal3.partitions.map(p => p.asInstanceOf[CoalescedRDDPartition].preferredLocation) + assert(list3.sorted === Array("a","b","c"), "Locality preferences are dropped") // RDD with locality preferences spread (non-randomly) over 6 machines, m0 through m5 val data = sc.makeRDD((1 to 9).map(i => (i, (i to (i+2)).map{ j => "m" + (j%6)}))) From db4bc55bef47aaac85c280d5c7d0d82217101b84 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Tue, 20 Aug 2013 12:17:55 -0700 Subject: [PATCH 0816/2521] indent --- core/src/main/scala/spark/rdd/CoalescedRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index b83d443de38..f880cea2ad3 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -260,7 +260,7 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc } while (numCreated < targetLen) { // if we don't have enough partition groups, create duplicates - var (nxt_replica, nxt_part) = rotIt.next() + var (nxt_replica, nxt_part) = rotIt.next() val pgroup = PartitionGroup(nxt_replica) groupArr += pgroup groupHash.get(nxt_replica).get += pgroup From 5cd21c41950e3004d3326bbc56286285531063f9 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Tue, 20 Aug 2013 13:59:48 -0700 Subject: [PATCH 0817/2521] added curly braces to make the code more consistent --- core/src/main/scala/spark/SparkContext.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 7639749ecbd..a8cc4f3eb8d 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -620,8 +620,9 @@ class SparkContext( * @param partition to be looked up for locality * @return list of preferred locations for the partition */ - private [spark] def getPreferredLocs(rdd: RDD[_], partition: Int): List[String] = + private [spark] def getPreferredLocs(rdd: RDD[_], partition: Int): List[String] = { dagScheduler.getPreferredLocs(rdd, partition) + } /** * Adds a JAR dependency for all tasks to be executed on this SparkContext in the future. From f20ed14e87aa1b9ff148f44b590ffd3c9d024f3c Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Tue, 20 Aug 2013 16:21:43 -0700 Subject: [PATCH 0818/2521] Merged in from upstream to use TaskLocation instead of strings --- core/src/main/scala/spark/SparkContext.scala | 9 ++++++--- core/src/main/scala/spark/rdd/CoalescedRDD.scala | 10 +++++----- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index a8cc4f3eb8d..23dfbcd6047 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -56,8 +56,7 @@ import spark.deploy.LocalSparkCluster import spark.partial.{ApproximateEvaluator, PartialResult} import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD, OrderedRDDFunctions} -import spark.scheduler.{DAGScheduler, DAGSchedulerSource, ResultTask, ShuffleMapTask, SparkListener, - SplitInfo, Stage, StageInfo, TaskScheduler} +import spark.scheduler._ import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler, Schedulable, SchedulingMode} import spark.scheduler.local.LocalScheduler @@ -65,6 +64,10 @@ import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend import spark.storage.{StorageStatus, StorageUtils, RDDInfo, BlockManagerSource} import spark.ui.SparkUI import spark.util.{MetadataCleaner, TimeStampedHashMap} +import scala.Some +import spark.scheduler.StageInfo +import spark.storage.RDDInfo +import spark.storage.StorageStatus /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -620,7 +623,7 @@ class SparkContext( * @param partition to be looked up for locality * @return list of preferred locations for the partition */ - private [spark] def getPreferredLocs(rdd: RDD[_], partition: Int): List[String] = { + private [spark] def getPreferredLocs(rdd: RDD[_], partition: Int): Seq[TaskLocation] = { dagScheduler.getPreferredLocs(rdd, partition) } diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index f880cea2ad3..e612d026b22 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -52,7 +52,7 @@ case class CoalescedRDDPartition( */ def localFraction: Double = { val loc = parents.count(p => - rdd.context.getPreferredLocs(rdd, p.index).contains(preferredLocation)) + rdd.context.getPreferredLocs(rdd, p.index).map(tl => tl.host).contains(preferredLocation)) if (parents.size == 0) 0.0 else (loc.toDouble / parents.size.toDouble) } @@ -167,8 +167,9 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc var noLocality = true // if true if no preferredLocations exists for parent RDD // gets the *current* preferred locations from the DAGScheduler (as opposed to the static ones) - def currPrefLocs(part: Partition): Seq[String] = - prev.context.getPreferredLocs(prev, part.index) + def currPrefLocs(part: Partition): Seq[String] = { + prev.context.getPreferredLocs(prev, part.index).map(tl => tl.host) + } // this class just keeps iterating and rotating infinitely over the partitions of the RDD // next() returns the next preferred machine that a partition is replicated on @@ -282,8 +283,7 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc * @return partition group (bin to be put in) */ def pickBin(p: Partition): PartitionGroup = { - val pref = prev.context.getPreferredLocs(prev, p.index). - map(getLeastGroupHash(_)).sortWith(compare) // least loaded of the pref locations + val pref = currPrefLocs(p).map(getLeastGroupHash(_)).sortWith(compare) // least loaded pref locs val prefPart = if (pref == Nil) None else pref.head val r1 = rnd.nextInt(groupArr.size) From 5eea613ec08c24ca4d4d5f0a8beebbe424334605 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 20 Aug 2013 16:49:18 -0700 Subject: [PATCH 0819/2521] Removed meaningless types --- core/src/main/scala/spark/util/MutablePair.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/util/MutablePair.scala b/core/src/main/scala/spark/util/MutablePair.scala index 3063806e83e..78d404e66b1 100644 --- a/core/src/main/scala/spark/util/MutablePair.scala +++ b/core/src/main/scala/spark/util/MutablePair.scala @@ -32,5 +32,5 @@ case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/ { override def toString = "(" + _1 + "," + _2 + ")" - override def canEqual(that: Any): Boolean = that.isInstanceOf[MutablePair[T1, T2]] + override def canEqual(that: Any): Boolean = that.isInstanceOf[MutablePair[_,_]] } From 51a1a0c602481620dd914f3024dee52c3312058d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 20 Aug 2013 22:14:52 -0700 Subject: [PATCH 0820/2521] Bump spark version --- ec2/spark_ec2.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 182cbbbb7c2..6c48d9765f6 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -68,7 +68,7 @@ def parse_args(): parser.add_option("-a", "--ami", help="Amazon Machine Image ID to use") - parser.add_option("-v", "--spark-version", default="0.7.2", + parser.add_option("-v", "--spark-version", default="0.7.3", help="Version of Spark to use: 'X.Y.Z' or a specific git hash") parser.add_option("--spark-git-repo", default="https://github.com/mesos/spark", @@ -158,7 +158,7 @@ def is_active(instance): # Return correct versions of Spark and Shark, given the supplied Spark version def get_spark_shark_version(opts): - spark_shark_map = {"0.7.2": "0.7.0"} + spark_shark_map = {"0.7.3": "0.7.0"} version = opts.spark_version.replace("v", "") if version not in spark_shark_map: print >> stderr, "Don't know about Spark version: %s" % version From 53b1c30607a9b19e795fd5b6107dfefb83820282 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 20 Aug 2013 22:57:11 -0700 Subject: [PATCH 0821/2521] Update docs for Spark UI port --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index dff08a06f56..b125eeb03c8 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -146,7 +146,7 @@ Apart from these, the following properties are also available, and may be useful - + From af602ba9d31ccf5a71b77904fce31c3c1376eeac Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 21 Aug 2013 11:38:24 -0700 Subject: [PATCH 0822/2521] Downgraded default build hadoop version to 1.0.4. --- pom.xml | 2 +- project/SparkBuild.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index fc0b3140701..587b42c55af 100644 --- a/pom.xml +++ b/pom.xml @@ -74,7 +74,7 @@ 2.0.3 1.7.2 1.2.17 - 1.2.1 + 1.0.4 64m diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 831bfbed78f..81080741ca9 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -28,7 +28,7 @@ object SparkBuild extends Build { // "1.0.4" for Apache releases, or "0.20.2-cdh3u5" for Cloudera Hadoop. // Note that these variables can be set through the environment variables // SPARK_HADOOP_VERSION and SPARK_WITH_YARN. - val DEFAULT_HADOOP_VERSION = "1.2.1" + val DEFAULT_HADOOP_VERSION = "1.0.4" val DEFAULT_WITH_YARN = false // HBase version; set as appropriate. From 111b2741fd4bacd5f0b31add22acd28d7d884299 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 21 Aug 2013 11:54:10 -0700 Subject: [PATCH 0823/2521] Change default SPARK_HADOOP_VERSION in make-distribution.sh too --- make-distribution.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/make-distribution.sh b/make-distribution.sh index 55dc22b9927..70aff418c70 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -46,7 +46,7 @@ export TERM=dumb # Prevents color codes in SBT output VERSION=$($FWDIR/sbt/sbt "show version" | tail -1 | cut -f 2 | sed 's/^\([a-zA-Z0-9.-]*\).*/\1/') # Initialize defaults -SPARK_HADOOP_VERSION=1.2.1 +SPARK_HADOOP_VERSION=1.0.4 SPARK_WITH_YARN=false MAKE_TGZ=false From 9c6f8df30f70ee8ee3a8c132dec251c355db2c28 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 21 Aug 2013 12:57:56 -0700 Subject: [PATCH 0824/2521] Update jekyll plugin to match docs/README.md --- 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 217254c59f8..c574ea7f5cd 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -18,7 +18,7 @@ require 'fileutils' include FileUtils -if ENV['SKIP_API'] != '1' +if not (ENV['SKIP_API'] == '1' or ENV['SKIP_SCALADOC'] == '1') # Build Scaladoc for Java/Scala projects = ["core", "examples", "repl", "bagel", "streaming", "mllib"] From 31644a011c1c61ca010b91dc83373e15960f4d23 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 21 Aug 2013 13:24:28 -0700 Subject: [PATCH 0825/2521] Use "hadoop.version" property when specifying Hadoop YARN version too --- pom.xml | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index 587b42c55af..b963782ee83 100644 --- a/pom.xml +++ b/pom.xml @@ -579,8 +579,8 @@ 2 - - 2.0.5-alpha + + 2.0.5-alpha @@ -607,7 +607,7 @@ org.apache.hadoop hadoop-client - ${yarn.version} + ${hadoop.version} asm @@ -638,7 +638,7 @@ org.apache.hadoop hadoop-yarn-api - ${yarn.version} + ${hadoop.version} asm @@ -669,7 +669,7 @@ org.apache.hadoop hadoop-yarn-common - ${yarn.version} + ${hadoop.version} asm @@ -700,7 +700,7 @@ org.apache.hadoop hadoop-yarn-client - ${yarn.version} + ${hadoop.version} asm From ff6f1b0500a9a75b61c225f3dc80d4026e98bcd5 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 21 Aug 2013 13:50:24 -0700 Subject: [PATCH 0826/2521] Synced sbt and maven builds --- core/pom.xml | 4 ++++ pom.xml | 16 +++++++++++----- project/SparkBuild.scala | 5 +---- repl/pom.xml | 6 ++++++ 4 files changed, 22 insertions(+), 9 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 6627a87de1d..3d70a195841 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -52,6 +52,10 @@ com.google.guava guava + + com.google.code.findbugs + jsr305 + org.slf4j slf4j-api diff --git a/pom.xml b/pom.xml index fc0b3140701..db696bad12a 100644 --- a/pom.xml +++ b/pom.xml @@ -71,10 +71,10 @@ 1.5 2.9.3 0.12.1 - 2.0.3 + 2.0.5 1.7.2 1.2.17 - 1.2.1 + 1.0.4 64m @@ -157,12 +157,17 @@ org.eclipse.jetty jetty-server - 7.5.3.v20111011 + 7.6.8.v20121106 com.google.guava guava - 11.0.1 + 14.0.1 + + + com.google.code.findbugs + jsr305 + 1.3.9 org.slf4j @@ -318,7 +323,7 @@ com.novocode junit-interface - 0.8 + 0.9 test @@ -437,6 +442,7 @@ -unchecked -optimise + -deprecation -Xms64m diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 831bfbed78f..78f9c4e981e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -134,7 +134,6 @@ object SparkBuild extends Build { */ libraryDependencies ++= Seq( - "io.netty" % "netty" % "3.5.3.Final", "org.eclipse.jetty" % "jetty-server" % "7.6.8.v20121106", "org.scalatest" %% "scalatest" % "1.9.1" % "test", "org.scalacheck" %% "scalacheck" % "1.10.0" % "test", @@ -165,17 +164,15 @@ object SparkBuild extends Build { name := "spark-core", 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/" ), libraryDependencies ++= Seq( "com.google.guava" % "guava" % "14.0.1", "com.google.code.findbugs" % "jsr305" % "1.3.9", - "log4j" % "log4j" % "1.2.16", + "log4j" % "log4j" % "1.2.17", "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, - "commons-daemon" % "commons-daemon" % "1.0.10", "com.ning" % "compress-lzf" % "0.8.4", "org.xerial.snappy" % "snappy-java" % "1.0.5", "org.ow2.asm" % "asm" % "4.0", diff --git a/repl/pom.xml b/repl/pom.xml index 5bc9a99c5c4..f800664cff6 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -48,6 +48,12 @@ ${project.version} runtime + + org.spark-project + spark-mllib + ${project.version} + runtime + org.eclipse.jetty jetty-server From 66e7a38a3229eeb6d980193048ebebcda1522acb Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 21 Aug 2013 14:25:53 -0700 Subject: [PATCH 0827/2521] Allow build configuration to be set in conf/spark-env.sh --- conf/spark-env.sh.template | 10 +++++++++- make-distribution.sh | 5 +++++ sbt/sbt | 15 +++++++++------ 3 files changed, 23 insertions(+), 7 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index c978db00d95..eefd1f86c53 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -19,6 +19,14 @@ # - SPARK_JAVA_OPTS, to set the jvm options for executor backend. Note: This is # only for node-specific options, whereas app-specific options should be set # in the application. -# Examples of node-speicic options : -Dspark.local.dir, GC related options. +# Examples of node-specific options : -Dspark.local.dir, GC related options. # Examples of app-specific options : -Dspark.serializer +# Hadoop version to build against +# export SPARK_HADOOP_VERSION=1.0.4 + +# Uncomment this when using a Hadoop version with YARN +# export SPARK_WITH_YARN=true + +# Extra arguments to pass to `java` when building with SBT +# export EXTRA_BUILD_ARGS="$EXTRA_BUILD_ARGS -Xmx8g" diff --git a/make-distribution.sh b/make-distribution.sh index 70aff418c70..846548617a0 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -50,6 +50,11 @@ SPARK_HADOOP_VERSION=1.0.4 SPARK_WITH_YARN=false MAKE_TGZ=false +# Load configuration +if [ -f "$FWDIR/conf/spark-env.sh" ]; then + source "$FWDIR/conf/spark-env.sh" +fi + # Parse arguments while (( "$#" )); do case $1 in diff --git a/sbt/sbt b/sbt/sbt index 397895276c1..a38a2985d14 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -17,12 +17,15 @@ # 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)/.." 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 "$@" +if [ -f "$SPARK_HOME/conf/spark-env.sh" ]; then + source "$SPARK_HOME/conf/spark-env.sh" +fi + +if [ "$MESOS_HOME" != "" ]; then + EXTRA_BUILD_ARGS="$EXTRA_BUILD_ARGS -Djava.library.path=$MESOS_HOME/lib/java" +fi + +java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=128m $EXTRA_BUILD_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" From 6585f49841ada637b0811e0aadcf93132fff7001 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 21 Aug 2013 14:51:56 -0700 Subject: [PATCH 0828/2521] Update build docs --- README.md | 46 +++++++++++++++++++++++++++++++++---- docs/building-with-maven.md | 35 +++++++++++++++++++--------- docs/running-on-yarn.md | 20 ++++++---------- 3 files changed, 73 insertions(+), 28 deletions(-) diff --git a/README.md b/README.md index 1dd96a0a4a9..1e388ff380f 100644 --- a/README.md +++ b/README.md @@ -16,7 +16,7 @@ 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: - sbt/sbt package + sbt/sbt package assembly Spark also supports building using Maven. If you would like to build using Maven, see the [instructions for building Spark with Maven](http://spark-project.org/docs/latest/building-with-maven.html) @@ -43,10 +43,48 @@ locally with one thread, or "local[N]" to run locally with N threads. ## A Note About Hadoop Versions Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported -storage systems. Because the HDFS API has changed in different versions of +storage systems. Because the protocols have changed in different versions of Hadoop, you must build Spark against the same version that your cluster runs. -You can change the version by setting the `HADOOP_VERSION` variable at the top -of `project/SparkBuild.scala`, then rebuilding Spark. +You can change the version by setting the `SPARK_HADOOP_VERSION` environment +when building Spark. + +For Apache Hadoop versions 1.x, 0.20.x, Cloudera CDH MRv1, and other Hadoop +versions without YARN, use: + + # Apache Hadoop 1.2.1 + $ SPARK_HADOOP_VERSION=1.2.1 sbt/sbt package assembly + + # Cloudera CDH 4.2.0 with MapReduce v1 + $ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt/sbt package assembly + +For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions +with YARN, also set `SPARK_WITH_YARN=true`: + + # Apache Hadoop 2.0.5-alpha + $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_WITH_YARN=true sbt/sbt package assembly + + # Cloudera CDH 4.2.0 with MapReduce v2 + $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_WITH_YARN=true sbt/sbt package assembly + +For convenience, these variables may also be set through the `conf/spark-env.sh` file +described below. + +When developing a Spark application, specify the Hadoop version by adding the +"hadoop-client" artifact to your project's dependencies. For example, if you're +using Hadoop 0.23.9 and build your application using SBT, add this to +`libraryDependencies`: + + // "force()" is required because "0.23.9" is less than Spark's default of "1.0.4" + "org.apache.hadoop" % "hadoop-client" % "0.23.9" force() + +If your project is built with Maven, add this to your POM file's `` section: + + + org.apache.hadoop + hadoop-client + + [0.23.9] + ## Configuration diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index 04cd79d039e..d71d94fa63c 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -8,22 +8,26 @@ title: Building Spark with Maven Building Spark using Maven Requires Maven 3 (the build process is tested with Maven 3.0.4) and Java 1.6 or newer. -Building with Maven requires that a Hadoop profile be specified explicitly at the command line, there is no default. There are two profiles to choose from, one for building for Hadoop 1 or Hadoop 2. +## Specifying the Hadoop version ## -for Hadoop 1 (using 0.20.205.0) use: +To enable support for HDFS and other Hadoop-supported storage systems, specify the exact Hadoop version by setting the "hadoop.version" property. If unset, Spark will build against Hadoop 1.0.4 by default. - $ mvn -Phadoop1 clean install +For Apache Hadoop versions 1.x, 0.20.x, Cloudera CDH MRv1, and other Hadoop versions without YARN, use: + # Apache Hadoop 1.2.1 + $ mvn -Dhadoop.version=1.2.1 clean install -for Hadoop 2 (using 2.0.0-mr1-cdh4.1.1) use: + # Cloudera CDH 4.2.0 with MapReduce v1 + $ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 clean install - $ mvn -Phadoop2 clean install +For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, enable the "hadoop2-yarn" profile: -It uses the scala-maven-plugin which supports incremental and continuous compilation. E.g. + # Apache Hadoop 2.0.5-alpha + $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.5-alpha clean install - $ mvn -Phadoop2 scala:cc + # Cloudera CDH 4.2.0 with MapReduce v2 + $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.0-cdh4.2.0 clean install -…should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively. ## Spark Tests in Maven ## @@ -31,11 +35,11 @@ Tests are run by default via the scalatest-maven-plugin. With this you can do th Skip test execution (but not compilation): - $ mvn -DskipTests -Phadoop2 clean install + $ mvn -Dhadoop.version=... -DskipTests clean install To run a specific test suite: - $ mvn -Phadoop2 -Dsuites=spark.repl.ReplSuite test + $ mvn -Dhadoop.version=... -Dsuites=spark.repl.ReplSuite test ## Setting up JVM Memory Usage Via Maven ## @@ -53,6 +57,15 @@ To fix these, you can do the following: export MAVEN_OPTS="-Xmx1024m -XX:MaxPermSize=128M" +## Continuous Compilation ## + +We use the scala-maven-plugin which supports incremental and continuous compilation. E.g. + + $ mvn scala:cc + +…should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively. + + ## Using With IntelliJ IDEA ## This setup works fine in IntelliJ IDEA 11.1.4. After opening the project via the pom.xml file in the project root folder, you only need to activate either the hadoop1 or hadoop2 profile in the "Maven Properties" popout. We have not tried Eclipse/Scala IDE with this. @@ -61,6 +74,6 @@ This setup works fine in IntelliJ IDEA 11.1.4. After opening the project via the It includes support for building a Debian package containing a 'fat-jar' which includes the repl, the examples and bagel. This can be created by specifying the deb profile: - $ mvn -Phadoop2,deb clean install + $ mvn -Pdeb clean install The debian package can then be found under repl/target. We added the short commit hash to the file name so that we can distinguish individual packages build for SNAPSHOT versions. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 9c2cedfd886..6bada9bdd75 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -6,7 +6,7 @@ title: Launching Spark on YARN Experimental support for running over a [YARN (Hadoop NextGen)](http://hadoop.apache.org/docs/r2.0.2-alpha/hadoop-yarn/hadoop-yarn-site/YARN.html) cluster was added to Spark in version 0.6.0. This was merged into master as part of 0.7 effort. -To build spark core with YARN support, please use the hadoop2-yarn profile. +To build spark with YARN support, please use the hadoop2-yarn profile. Ex: mvn -Phadoop2-yarn clean install # Building spark core consolidated jar. @@ -15,18 +15,12 @@ We need a consolidated spark core jar (which bundles all the required dependenci 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. +Enable YARN support by setting `SPARK_WITH_YARN=true` when invoking sbt: -Assembly of the jar Ex: - - ./sbt/sbt clean assembly + SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_WITH_YARN=true ./sbt/sbt clean assembly The assembled jar would typically be something like : -`./core/target/spark-core-assembly-0.8.0-SNAPSHOT.jar` +`./yarn/target/spark-yarn-assembly-0.8.0-SNAPSHOT.jar` - Building spark assembled jar via Maven. @@ -34,16 +28,16 @@ The assembled jar would typically be something like : Something like this. Ex: - mvn -Phadoop2-yarn clean package -DskipTests=true + mvn -Phadoop2-yarn -Dhadoop.version=2.0.5-alpha clean package -DskipTests=true This will build the shaded (consolidated) jar. Typically something like : -`./repl-bin/target/spark-repl-bin--shaded-hadoop2-yarn.jar` +`./yarn/target/spark-yarn-bin--shaded.jar` # Preparations -- Building spark core assembled jar (see above). +- Building spark-yarn assembly (see above). - Your application code must be packaged into a separate JAR file. If you want to test out the YARN deployment mode, you can use the current Spark examples. A `spark-examples_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}` file can be generated by running `sbt/sbt package`. NOTE: since the documentation you're reading is for Spark version {{site.SPARK_VERSION}}, we are assuming here that you have downloaded Spark {{site.SPARK_VERSION}} or checked it out of source control. If you are using a different version of Spark, the version numbers in the jar generated by the sbt package command will obviously be different. From 4d737b6d320993432f286e05b773135fc3334ce3 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 21 Aug 2013 14:56:25 -0700 Subject: [PATCH 0829/2521] Example should make sense --- README.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index 1e388ff380f..9b2c1a9c4dc 100644 --- a/README.md +++ b/README.md @@ -71,19 +71,19 @@ described below. When developing a Spark application, specify the Hadoop version by adding the "hadoop-client" artifact to your project's dependencies. For example, if you're -using Hadoop 0.23.9 and build your application using SBT, add this to +using Hadoop 1.0.1 and build your application using SBT, add this to `libraryDependencies`: - // "force()" is required because "0.23.9" is less than Spark's default of "1.0.4" - "org.apache.hadoop" % "hadoop-client" % "0.23.9" force() + // "force()" is required because "1.0.1" is less than Spark's default of "1.0.4" + "org.apache.hadoop" % "hadoop-client" % "1.0.1" force() If your project is built with Maven, add this to your POM file's `` section: org.apache.hadoop hadoop-client - - [0.23.9] + + [1.0.1] From c02585ea130045ef27e579172ac2acc71bc8da63 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 21 Aug 2013 15:45:45 -0700 Subject: [PATCH 0830/2521] Make initial connection failure message less daunting. Right now it seems like something has gone wrong when this message is printed out. Instead, this is a normal condition. So I changed the message a bit. --- 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 30253a94b84..932e70db96e 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -562,7 +562,7 @@ def ssh(host, opts, command): except subprocess.CalledProcessError as e: if (tries > 2): raise e - print "Error connecting to host {0}, sleeping 30".format(e) + print "Couldn't connect to host {0}, waiting 30 seconds".format(e) time.sleep(30) tries = tries + 1 From 76077bf9f4b726699ba9e59cdfa9c4361df4ea92 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Tue, 20 Aug 2013 13:22:06 -0700 Subject: [PATCH 0831/2521] Implementing SPARK-838: Add DoubleRDDFunctions methods to PySpark --- python/pyspark/rdd.py | 60 ++++++++++++++++++- python/pyspark/statcounter.py | 109 ++++++++++++++++++++++++++++++++++ 2 files changed, 168 insertions(+), 1 deletion(-) create mode 100644 python/pyspark/statcounter.py diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 99f5967a8ed..1e9b3bb5c03 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -31,6 +31,7 @@ read_from_pickle_file from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_cogroup +from pyspark.statcounter import StatCounter from py4j.java_collections import ListConverter, MapConverter @@ -357,6 +358,63 @@ def count(self): 3 """ return self.mapPartitions(lambda i: [sum(1 for _ in i)]).sum() + + def stats(self): + """ + Return a L{StatCounter} object that captures the mean, variance + and count of the RDD's elements in one operation. + """ + def redFunc(left_counter, right_counter): + return left_counter.mergeStats(right_counter) + + return self.mapPartitions(lambda i: [StatCounter(i)]).reduce(redFunc) + + def mean(self): + """ + Compute the mean of this RDD's elements. + + >>> sc.parallelize([1, 2, 3]).mean() + 2.0 + """ + return self.stats().mean() + + def variance(self): + """ + Compute the variance of this RDD's elements. + + >>> sc.parallelize([1, 2, 3]).variance() + 0.666... + """ + return self.stats().variance() + + def stdev(self): + """ + Compute the standard deviation of this RDD's elements. + + >>> sc.parallelize([1, 2, 3]).stdev() + 0.816... + """ + return self.stats().stdev() + + def sampleStdev(self): + """ + 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). + + >>> sc.parallelize([1, 2, 3]).sampleStdev() + 1.0 + """ + return self.stats().sampleStdev() + + def sampleVariance(self): + """ + 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). + + >>> sc.parallelize([1, 2, 3]).sampleVariance() + 1.0 + """ + return self.stats().sampleVariance() def countByValue(self): """ @@ -777,7 +835,7 @@ def _test(): # The small batch size here ensures that we see multiple batches, # even in these small test examples: globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) - (failure_count, test_count) = doctest.testmod(globs=globs) + (failure_count, test_count) = doctest.testmod(globs=globs,optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: exit(-1) diff --git a/python/pyspark/statcounter.py b/python/pyspark/statcounter.py new file mode 100644 index 00000000000..8e1cbd4ad98 --- /dev/null +++ b/python/pyspark/statcounter.py @@ -0,0 +1,109 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 file is ported from spark/util/StatCounter.scala + +import copy +import math + +class StatCounter(object): + + def __init__(self, values=[]): + self.n = 0L # Running count of our values + self.mu = 0.0 # Running mean of our values + self.m2 = 0.0 # Running variance numerator (sum of (x - mean)^2) + + for v in values: + self.merge(v) + + # Add a value into this StatCounter, updating the internal statistics. + def merge(self, value): + delta = value - self.mu + self.n += 1 + self.mu += delta / self.n + self.m2 += delta * (value - self.mu) + return self + + # Merge another StatCounter into this one, adding up the internal statistics. + def mergeStats(self, other): + if not isinstance(other, StatCounter): + raise Exception("Can only merge Statcounters!") + + if other is self: # reference equality holds + self.merge(copy.deepcopy(other)) # Avoid overwriting fields in a weird order + else: + if self.n == 0: + self.mu = other.mu + self.m2 = other.m2 + self.n = other.n + elif other.n != 0: + delta = other.mu - self.mu + if other.n * 10 < self.n: + self.mu = self.mu + (delta * other.n) / (self.n + other.n) + elif self.n * 10 < other.n: + self.mu = other.mu - (delta * self.n) / (self.n + other.n) + else: + self.mu = (self.mu * self.n + other.mu * other.n) / (self.n + other.n) + + self.m2 += other.m2 + (delta * delta * self.n * other.n) / (self.n + other.n) + self.n += other.n + return self + + # Clone this StatCounter + def copy(self): + return copy.deepcopy(self) + + def count(self): + return self.n + + def mean(self): + return self.mu + + def sum(self): + return self.n * self.mu + + # Return the variance of the values. + def variance(self): + if self.n == 0: + return float('nan') + else: + return self.m2 / self.n + + # + # Return the sample variance, which corrects for bias in estimating the variance by dividing + # by N-1 instead of N. + # + def sampleVariance(self): + if self.n <= 1: + return float('nan') + else: + return self.m2 / (self.n - 1) + + # Return the standard deviation of the values. + def stdev(self): + return math.sqrt(self.variance()) + + # + # Return the sample standard deviation of the values, which corrects for bias in estimating the + # variance by dividing by N-1 instead of N. + # + def sampleStdev(self): + return math.sqrt(self.sampleVariance()) + + def __repr__(self): + return "(count: %s, mean: %s, stdev: %s)" % (self.count(), self.mean(), self.stdev()) + From f9cc1fbf272606ce0b679936ef38429b20916cc1 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 21 Aug 2013 17:12:03 -0700 Subject: [PATCH 0832/2521] Remove references to unsupported Hadoop versions --- README.md | 2 +- docs/building-with-maven.md | 2 +- project/SparkBuild.scala | 7 +++---- 3 files changed, 5 insertions(+), 6 deletions(-) diff --git a/README.md b/README.md index 9b2c1a9c4dc..8502410c52c 100644 --- a/README.md +++ b/README.md @@ -48,7 +48,7 @@ Hadoop, you must build Spark against the same version that your cluster runs. You can change the version by setting the `SPARK_HADOOP_VERSION` environment when building Spark. -For Apache Hadoop versions 1.x, 0.20.x, Cloudera CDH MRv1, and other Hadoop +For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions without YARN, use: # Apache Hadoop 1.2.1 diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index d71d94fa63c..a9f2cb8a7ae 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -12,7 +12,7 @@ Building Spark using Maven Requires Maven 3 (the build process is tested with Ma To enable support for HDFS and other Hadoop-supported storage systems, specify the exact Hadoop version by setting the "hadoop.version" property. If unset, Spark will build against Hadoop 1.0.4 by default. -For Apache Hadoop versions 1.x, 0.20.x, Cloudera CDH MRv1, and other Hadoop versions without YARN, use: +For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions without YARN, use: # Apache Hadoop 1.2.1 $ mvn -Dhadoop.version=1.2.1 clean install diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 81080741ca9..cc09bf7dd2a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -24,10 +24,9 @@ import AssemblyKeys._ //import com.jsuereth.pgp.sbtplugin.PgpKeys._ object SparkBuild extends Build { - // Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or - // "1.0.4" for Apache releases, or "0.20.2-cdh3u5" for Cloudera Hadoop. - // Note that these variables can be set through the environment variables - // SPARK_HADOOP_VERSION and SPARK_WITH_YARN. + // Hadoop version to build against. For example, "1.0.4" for Apache releases, or + // "2.0.0-mr1-cdh4.2.0" for Cloudera Hadoop. Note that these variables can be set + // through the environment variables SPARK_HADOOP_VERSION and SPARK_WITH_YARN. val DEFAULT_HADOOP_VERSION = "1.0.4" val DEFAULT_WITH_YARN = false From 54e9379de291c3d5e5a9749cfc022a17ff34852c Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 21 Aug 2013 21:13:34 -0700 Subject: [PATCH 0833/2521] Revert "Allow build configuration to be set in conf/spark-env.sh" This reverts commit 66e7a38a3229eeb6d980193048ebebcda1522acb. --- conf/spark-env.sh.template | 10 +--------- make-distribution.sh | 5 ----- sbt/sbt | 15 ++++++--------- 3 files changed, 7 insertions(+), 23 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index eefd1f86c53..c978db00d95 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -19,14 +19,6 @@ # - SPARK_JAVA_OPTS, to set the jvm options for executor backend. Note: This is # only for node-specific options, whereas app-specific options should be set # in the application. -# Examples of node-specific options : -Dspark.local.dir, GC related options. +# Examples of node-speicic options : -Dspark.local.dir, GC related options. # Examples of app-specific options : -Dspark.serializer -# Hadoop version to build against -# export SPARK_HADOOP_VERSION=1.0.4 - -# Uncomment this when using a Hadoop version with YARN -# export SPARK_WITH_YARN=true - -# Extra arguments to pass to `java` when building with SBT -# export EXTRA_BUILD_ARGS="$EXTRA_BUILD_ARGS -Xmx8g" diff --git a/make-distribution.sh b/make-distribution.sh index 846548617a0..70aff418c70 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -50,11 +50,6 @@ SPARK_HADOOP_VERSION=1.0.4 SPARK_WITH_YARN=false MAKE_TGZ=false -# Load configuration -if [ -f "$FWDIR/conf/spark-env.sh" ]; then - source "$FWDIR/conf/spark-env.sh" -fi - # Parse arguments while (( "$#" )); do case $1 in diff --git a/sbt/sbt b/sbt/sbt index a38a2985d14..397895276c1 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -17,15 +17,12 @@ # limitations under the License. # -export SPARK_HOME=$(cd "$(dirname $0)/.." 2>&1 >/dev/null ; pwd) -export SPARK_TESTING=1 # To put test classes on classpath - -if [ -f "$SPARK_HOME/conf/spark-env.sh" ]; then - source "$SPARK_HOME/conf/spark-env.sh" -fi - +EXTRA_ARGS="" if [ "$MESOS_HOME" != "" ]; then - EXTRA_BUILD_ARGS="$EXTRA_BUILD_ARGS -Djava.library.path=$MESOS_HOME/lib/java" + EXTRA_ARGS="-Djava.library.path=$MESOS_HOME/lib/java" fi -java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=128m $EXTRA_BUILD_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" +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 "$@" From 0087b43e9cddc726f661e1e047e63390d5d9b419 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 21 Aug 2013 21:15:00 -0700 Subject: [PATCH 0834/2521] Use Hadoop 1.2.1 in application example --- README.md | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index 8502410c52c..e5f527b84a9 100644 --- a/README.md +++ b/README.md @@ -71,19 +71,18 @@ described below. When developing a Spark application, specify the Hadoop version by adding the "hadoop-client" artifact to your project's dependencies. For example, if you're -using Hadoop 1.0.1 and build your application using SBT, add this to +using Hadoop 1.0.1 and build your application using SBT, add this entry to `libraryDependencies`: - // "force()" is required because "1.0.1" is less than Spark's default of "1.0.4" - "org.apache.hadoop" % "hadoop-client" % "1.0.1" force() + "org.apache.hadoop" % "hadoop-client" % "1.2.1" If your project is built with Maven, add this to your POM file's `` section: org.apache.hadoop hadoop-client - - [1.0.1] + + [1.2.1] From 9a90667d098487b205ab8fc63c824843e30c739e Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Wed, 21 Aug 2013 21:15:28 -0700 Subject: [PATCH 0835/2521] Increase ReservedCodeCacheSize to 256m --- sbt/sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbt/sbt b/sbt/sbt index 397895276c1..a79f60d087c 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -25,4 +25,4 @@ fi 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 "$@" +java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=256m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" From 281b6c5f28bd7377fb89097f557f180a7fdf5b50 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Thu, 22 Aug 2013 13:22:29 -0700 Subject: [PATCH 0836/2521] Re-add removed dependency on 'commons-daemon' Fixes SBT build under Hadoop 0.23.9 and 2.0.4 --- project/SparkBuild.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ae35de71cd4..2a80f9c9469 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -172,6 +172,7 @@ object SparkBuild extends Build { "log4j" % "log4j" % "1.2.17", "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, + "commons-daemon" % "commons-daemon" % "1.0.10", // workaround for bug HADOOP-9407 "com.ning" % "compress-lzf" % "0.8.4", "org.xerial.snappy" % "snappy-java" % "1.0.5", "org.ow2.asm" % "asm" % "4.0", From 215c13dd41d8500835ef00624a0b4ced2253554e Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 22 Aug 2013 16:13:46 -0700 Subject: [PATCH 0837/2521] Fix code style and a nondeterministic RDD issue in ALS --- .../spark/mllib/recommendation/ALS.scala | 31 ++++++++++++------- 1 file changed, 20 insertions(+), 11 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala index 9097f46db90..dbfbf599754 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala @@ -123,18 +123,27 @@ 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 randomly - var users = userOutLinks.mapPartitions {itr => - val rand = new Random() - itr.map({case (x, y) => - (x, y.elementIds.map(u => randomFactor(rank, rand))) - }) + // Initialize user and product factors randomly, but use a deterministic seed for each partition + // so that fault recovery works + val seedGen = new Random() + val seed1 = seedGen.nextInt() + val seed2 = seedGen.nextInt() + // Hash an integer to propagate random bits at all positions, similar to java.util.HashTable + def hash(x: Int): Int = { + val r = x ^ (x >>> 20) ^ (x >>> 12) + r ^ (r >>> 7) ^ (r >>> 4) } - var products = productOutLinks.mapPartitions {itr => - val rand = new Random() - itr.map({case (x, y) => - (x, y.elementIds.map(u => randomFactor(rank, rand))) - }) + var users = userOutLinks.mapPartitionsWithIndex { (index, itr) => + val rand = new Random(hash(seed1 ^ index)) + itr.map { case (x, y) => + (x, y.elementIds.map(_ => randomFactor(rank, rand))) + } + } + var products = productOutLinks.mapPartitionsWithIndex { (index, itr) => + val rand = new Random(hash(seed2 ^ index)) + itr.map { case (x, y) => + (x, y.elementIds.map(_ => randomFactor(rank, rand))) + } } for (iter <- 0 until iterations) { From 39a1d58da484165790c61a924550b58837997f0d Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 15 Jul 2013 17:25:51 +0530 Subject: [PATCH 0838/2521] Improved documentation for spark custom receiver --- docs/streaming-custom-receivers.md | 51 ++++++++++++++++++++++++++++-- 1 file changed, 48 insertions(+), 3 deletions(-) diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index 5476c00d020..dfa343bf94f 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -7,10 +7,45 @@ A "Spark Streaming" receiver can be a simple network stream, streams of messages This guide shows the programming model and features by walking through a simple sample receiver and corresponding Spark Streaming application. +### Write a simple receiver -## A quick and naive walk-through +This starts with implementing [NetworkReceiver](#References) -### Write a simple receiver +Following is a simple socket text-stream receiver. + +{% highlight scala %} + + class SocketTextStreamReceiver(host: String, + port: Int + ) extends NetworkReceiver[String] { + + protected lazy val blocksGenerator: BlockGenerator = + new BlockGenerator(StorageLevel.MEMORY_ONLY_SER_2) + + protected def onStart() = { + blocksGenerator.start() + val socket = new Socket(host, port) + val dataInputStream = new BufferedReader(new InputStreamReader(socket.getInputStream(), "UTF-8")) + var data: String = dataInputStream.readLine() + while (data != null) { + blocksGenerator += data + data = dataInputStream.readLine() + } + } + + protected def onStop() { + blocksGenerator.stop() + } + + } + +{% endhighlight %} + + +All we did here is extended NetworkReceiver and called blockGenerator's API method (i.e. +=) to push our blocks of data. Please refer to scala-docs of NetworkReceiver for more details. + + +### An Actor as Receiver. This starts with implementing [Actor](#References) @@ -46,7 +81,16 @@ All we did here is mixed in trait Receiver and called pushBlock api method to pu {% endhighlight %} -* Plug-in the actor configuration into the spark streaming context and create a DStream. +* Plug-in the custom receiver into the spark streaming context and create a DStream. + +{% highlight scala %} + + val lines = ssc.networkStream[String](new SocketTextStreamReceiver( + "localhost", 8445)) + +{% endhighlight %} + +* OR Plug-in the actor as receiver into the spark streaming context and create a DStream. {% highlight scala %} @@ -99,3 +143,4 @@ _A more comprehensive example is provided in the spark streaming examples_ ## References 1.[Akka Actor documentation](http://doc.akka.io/docs/akka/2.0.5/scala/actors.html) +2.[NetworkReceiver](http://spark-project.org/docs/latest/api/streaming/index.html#spark.streaming.dstream.NetworkReceiver) From 3049415e245eefac0593a3e33ee7c8a845eeb800 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 23 Aug 2013 09:40:28 +0530 Subject: [PATCH 0839/2521] Corrections in documentation comment --- .../scala/spark/streaming/dstream/NetworkInputDStream.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala index 122a529bb76..d98d73a5a8c 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala @@ -128,8 +128,8 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log } /** - * Stops the receiver and reports to exception to the tracker. - * This should be called whenever an exception has happened on any thread + * Stops the receiver and reports exception to the tracker. + * This should be called whenever an exception is to be handled on any thread * of the receiver. */ protected def stopOnError(e: Exception) { @@ -185,7 +185,7 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log } /** - * Batches objects created by a [[spark.streaming.NetworkReceiver]] and puts them into + * Batches objects created by a [[spark.streaming.dstream.NetworkReceiver]] and puts them into * appropriately named blocks at regular intervals. This class starts two threads, * one to periodically start a new batch and prepare the previous batch of as a block, * the other to push the blocks into the block manager. From 2bc348e92c458ea36872ac43a2583370d1f3eb41 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 23 Aug 2013 09:41:32 +0530 Subject: [PATCH 0840/2521] Linking custom receiver guide --- docs/streaming-programming-guide.md | 3 +++ .../src/main/scala/spark/streaming/StreamingContext.scala | 2 ++ .../main/scala/spark/streaming/receivers/ActorReceiver.scala | 2 ++ 3 files changed, 7 insertions(+) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 8cd1b0cd660..a74c17bdb7a 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -301,6 +301,9 @@ dstream.checkpoint(checkpointInterval) // checkpointInterval must be a multiple For DStreams that must be checkpointed (that is, DStreams created by `updateStateByKey` and `reduceByKeyAndWindow` with inverse function), the checkpoint interval of the DStream is by default set to a multiple of the DStream's sliding interval such that its at least 10 seconds. +## Customizing Receiver +Spark comes with a built in support for most common usage scenarios where input stream source can be either a network socket stream to support for a few message queues. Apart from that it is also possible to supply your own custom receiver via a convenient API. Find more details at [Custom Receiver Guide](streaming-custom-receivers.html) + # Performance Tuning Getting the best performance of a Spark Streaming application on a cluster requires a bit of tuning. This section explains a number of the parameters and configurations that can tuned to improve the performance of you application. At a high level, you need to consider two things:
      diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala index 36b841af8fa..9abe55ecce3 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala @@ -166,6 +166,7 @@ class StreamingContext private ( /** * Create an input stream with any arbitrary user implemented network receiver. + * Find more details at: http://spark-project.org/docs/latest/streaming-custom-receivers.html * @param receiver Custom implementation of NetworkReceiver */ def networkStream[T: ClassManifest]( @@ -178,6 +179,7 @@ class StreamingContext private ( /** * Create an input stream with any arbitrary user implemented actor receiver. + * Find more details at: http://spark-project.org/docs/latest/streaming-custom-receivers.html * @param props Props object defining creation of the actor * @param name Name of the actor * @param storageLevel RDD storage level. Defaults to memory-only. diff --git a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala index 036c95a860c..3eca397f1aa 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala @@ -28,6 +28,8 @@ object ReceiverSupervisorStrategy { * A receiver trait to be mixed in with your Actor to gain access to * pushBlock API. * + * Find more details at: http://spark-project.org/docs/latest/streaming-custom-receivers.html + * * @example {{{ * class MyActor extends Actor with Receiver{ * def receive { From d7f18e3d279bf7f5a56db434f3abf77a5ccd9971 Mon Sep 17 00:00:00 2001 From: Ian Buss Date: Fri, 23 Aug 2013 09:40:16 +0100 Subject: [PATCH 0841/2521] Pass SBT_OPTS environment through to sbt_launcher --- sbt/sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbt/sbt b/sbt/sbt index a79f60d087c..2227bc46969 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -25,4 +25,4 @@ fi 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=256m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" +java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=256m $EXTRA_ARGS $SBT_OPTS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" From b7f9e6374ae89568b5b7298d89825eaf0b33cc15 Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Fri, 23 Aug 2013 10:26:37 -0700 Subject: [PATCH 0842/2521] Fix SBT generation of IDE project files --- project/SparkBuild.scala | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 2a80f9c9469..fbeae277076 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -57,14 +57,14 @@ object SparkBuild extends Build { // Allows build configuration to be set through environment variables lazy val hadoopVersion = scala.util.Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION) - lazy val isYarnMode = scala.util.Properties.envOrNone("SPARK_WITH_YARN") match { + lazy val isYarnEnabled = scala.util.Properties.envOrNone("SPARK_WITH_YARN") match { case None => DEFAULT_WITH_YARN case Some(v) => v.toBoolean } // Conditionally include the yarn sub-project - lazy val maybeYarn = if(isYarnMode) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]() - lazy val maybeYarnRef = if(isYarnMode) Seq[ProjectReference](yarn) else Seq[ProjectReference]() + lazy val maybeYarn = if(isYarnEnabled) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]() + lazy val maybeYarnRef = if(isYarnEnabled) Seq[ProjectReference](yarn) else Seq[ProjectReference]() lazy val allProjects = Seq[ProjectReference](core, repl, examples, bagel, streaming, mllib, tools) ++ maybeYarnRef def sharedSettings = Defaults.defaultSettings ++ Seq( @@ -253,7 +253,14 @@ object SparkBuild extends Build { ) ++ assemblySettings ++ extraAssemblySettings def yarnSettings = sharedSettings ++ Seq( - name := "spark-yarn", + name := "spark-yarn" + ) ++ extraYarnSettings ++ assemblySettings ++ extraAssemblySettings + + // Conditionally include the YARN dependencies because some tools look at all sub-projects and will complain + // if we refer to nonexistent dependencies (e.g. hadoop-yarn-api from a Hadoop version without YARN). + def extraYarnSettings = if(isYarnEnabled) yarnEnabledSettings else Seq() + + def yarnEnabledSettings = Seq( libraryDependencies ++= Seq( // Exclude rule required for all ? "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm), @@ -261,7 +268,7 @@ object SparkBuild extends Build { "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm), "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm) ) - ) ++ assemblySettings ++ extraAssemblySettings + ) def extraAssemblySettings() = Seq(test in assembly := {}) ++ Seq( mergeStrategy in assembly := { From a9db1b7b6eb030feb7beee017d2eca402b73c67c Mon Sep 17 00:00:00 2001 From: Jey Kottalam Date: Fri, 23 Aug 2013 10:27:18 -0700 Subject: [PATCH 0843/2521] Upgrade SBT IDE project generators --- project/plugins.sbt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/plugins.sbt b/project/plugins.sbt index 1b0f879b948..783b40d4f5d 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -6,9 +6,9 @@ resolvers += "Spray Repository" at "http://repo.spray.cc/" addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.5") -addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.1") +addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.2.0") -addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.2.0") +addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.5.1") // For Sonatype publishing //resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns) From 2cfe52ef55d87172109db4ac81c1f97300c47278 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 24 Aug 2013 15:16:53 -0700 Subject: [PATCH 0844/2521] Version bump for ec2 docs --- docs/ec2-scripts.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md index bd787e0e461..da0c06e2a62 100644 --- a/docs/ec2-scripts.md +++ b/docs/ec2-scripts.md @@ -74,7 +74,7 @@ another. bidding for the given maximum price (in dollars). - `--spark-version=VERSION` will pre-load the cluster with the specified version of Spark. VERSION can be a version number - (e.g. "0.7.2") or a specific git hash. By default, a recent + (e.g. "0.7.3") or a specific git hash. By default, a recent version will be used. - If one of your launches fails due to e.g. not having the right permissions on your private key file, you can run `launch` with the From b8c50a0642cf74c25fd70cc1e7d1be95ddafc5d8 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 25 Aug 2013 22:24:27 -0700 Subject: [PATCH 0845/2521] Center & scale variables in Ridge, Lasso. Also add a unit test that checks if ridge regression lowers cross-validation error. --- .../scala/spark/mllib/regression/Lasso.scala | 45 +++++- .../mllib/regression/LinearRegression.scala | 44 +++--- .../mllib/regression/RidgeRegression.scala | 95 ++++++++---- .../mllib/util/LinearDataGenerator.scala | 24 ++- .../main/scala/spark/mllib/util/MLUtils.scala | 10 +- .../mllib/regression/JavaLassoSuite.java | 8 +- .../regression/JavaLinearRegressionSuite.java | 119 ++++++++------- .../regression/JavaRidgeRegressionSuite.java | 139 ++++++++++-------- .../regression/LinearRegressionSuite.scala | 27 +++- .../regression/RidgeRegressionSuite.scala | 64 +++++--- 10 files changed, 347 insertions(+), 228 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala index 6bbc990a5a0..929c36bd760 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala @@ -55,10 +55,17 @@ class LassoWithSGD private ( val gradient = new SquaredGradient() val updater = new L1Updater() - val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) - .setNumIterations(numIterations) - .setRegParam(regParam) - .setMiniBatchFraction(miniBatchFraction) + @transient val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) + .setNumIterations(numIterations) + .setRegParam(regParam) + .setMiniBatchFraction(miniBatchFraction) + + // We don't want to penalize the intercept, so set this to false. + setIntercept(false) + + var yMean = 0.0 + var xColMean: DoubleMatrix = _ + var xColSd: DoubleMatrix = _ /** * Construct a Lasso object with default parameters @@ -66,7 +73,35 @@ class LassoWithSGD private ( def this() = this(1.0, 100, 1.0, 1.0, true) def createModel(weights: Array[Double], intercept: Double) = { - new LassoModel(weights, intercept) + val weightsMat = new DoubleMatrix(weights.length + 1, 1, (Array(intercept) ++ weights):_*) + val weightsScaled = weightsMat.div(xColSd) + val interceptScaled = yMean - (weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0)) + + new LassoModel(weightsScaled.data, interceptScaled) + } + + override def run( + input: RDD[LabeledPoint], + initialWeights: Array[Double]) + : LassoModel = + { + val nfeatures: Int = input.first.features.length + val nexamples: Long = input.count() + + // To avoid penalizing the intercept, we center and scale the data. + val stats = MLUtils.computeStats(input, nfeatures, nexamples) + yMean = stats._1 + xColMean = stats._2 + xColSd = stats._3 + + val normalizedData = input.map { point => + val yNormalized = point.label - yMean + val featuresMat = new DoubleMatrix(nfeatures, 1, point.features:_*) + val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd) + LabeledPoint(yNormalized, featuresNormalized.toArray) + } + + super.run(normalizedData, initialWeights) } } diff --git a/mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala index 0ea5348a1fb..5b3743f2fa1 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala @@ -45,10 +45,10 @@ class LinearRegressionModel( * Train a regression model with no regularization using Stochastic Gradient Descent. */ class LinearRegressionWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var miniBatchFraction: Double, - var addIntercept: Boolean) + var stepSize: Double, + var numIterations: Int, + var miniBatchFraction: Double, + var addIntercept: Boolean) extends GeneralizedLinearAlgorithm[LinearRegressionModel] with Serializable { @@ -87,12 +87,12 @@ object LinearRegressionWithSGD { * the number of features in the data. */ def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - miniBatchFraction: Double, - initialWeights: Array[Double]) - : LinearRegressionModel = + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double, + initialWeights: Array[Double]) + : LinearRegressionModel = { new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction, true).run(input, initialWeights) @@ -109,11 +109,11 @@ object LinearRegressionWithSGD { * @param miniBatchFraction Fraction of data to be used per iteration. */ def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - miniBatchFraction: Double) - : LinearRegressionModel = + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double) + : LinearRegressionModel = { new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction, true).run(input) } @@ -129,10 +129,10 @@ object LinearRegressionWithSGD { * @return a LinearRegressionModel which has the weights and offset from training. */ def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double) - : LinearRegressionModel = + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double) + : LinearRegressionModel = { train(input, numIterations, stepSize, 1.0) } @@ -147,9 +147,9 @@ object LinearRegressionWithSGD { * @return a LinearRegressionModel which has the weights and offset from training. */ def train( - input: RDD[LabeledPoint], - numIterations: Int) - : LinearRegressionModel = + input: RDD[LabeledPoint], + numIterations: Int) + : LinearRegressionModel = { train(input, numIterations, 1.0, 1.0) } diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index addf8cd59ec..ccf73648061 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -55,18 +55,54 @@ class RidgeRegressionWithSGD private ( val gradient = new SquaredGradient() val updater = new SquaredL2Updater() - val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) + + @transient val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) .setNumIterations(numIterations) .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) + // We don't want to penalize the intercept in RidgeRegression, so set this to false. + setIntercept(false) + + var yMean = 0.0 + var xColMean: DoubleMatrix = _ + var xColSd: DoubleMatrix = _ + /** * Construct a RidgeRegression object with default parameters */ def this() = this(1.0, 100, 1.0, 1.0, true) def createModel(weights: Array[Double], intercept: Double) = { - new RidgeRegressionModel(weights, intercept) + val weightsMat = new DoubleMatrix(weights.length + 1, 1, (Array(intercept) ++ weights):_*) + val weightsScaled = weightsMat.div(xColSd) + val interceptScaled = yMean - (weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0)) + + new RidgeRegressionModel(weightsScaled.data, interceptScaled) + } + + override def run( + input: RDD[LabeledPoint], + initialWeights: Array[Double]) + : RidgeRegressionModel = + { + val nfeatures: Int = input.first.features.length + val nexamples: Long = input.count() + + // To avoid penalizing the intercept, we center and scale the data. + val stats = MLUtils.computeStats(input, nfeatures, nexamples) + yMean = stats._1 + xColMean = stats._2 + xColSd = stats._3 + + val normalizedData = input.map { point => + val yNormalized = point.label - yMean + val featuresMat = new DoubleMatrix(nfeatures, 1, point.features:_*) + val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd) + LabeledPoint(yNormalized, featuresNormalized.toArray) + } + + super.run(normalizedData, initialWeights) } } @@ -90,16 +126,16 @@ object RidgeRegressionWithSGD { * the number of features in the data. */ def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - regParam: Double, - miniBatchFraction: Double, - initialWeights: Array[Double]) - : RidgeRegressionModel = + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, + initialWeights: Array[Double]) + : RidgeRegressionModel = { - new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction, true).run(input, - initialWeights) + new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction, true).run( + input, initialWeights) } /** @@ -114,14 +150,15 @@ object RidgeRegressionWithSGD { * @param miniBatchFraction Fraction of data to be used per iteration. */ def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - regParam: Double, - miniBatchFraction: Double) - : RidgeRegressionModel = + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double) + : RidgeRegressionModel = { - new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction, true).run(input) + new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction, true).run( + input) } /** @@ -136,11 +173,11 @@ object RidgeRegressionWithSGD { * @return a RidgeRegressionModel which has the weights and offset from training. */ def train( - input: RDD[LabeledPoint], - numIterations: Int, - stepSize: Double, - regParam: Double) - : RidgeRegressionModel = + input: RDD[LabeledPoint], + numIterations: Int, + stepSize: Double, + regParam: Double) + : RidgeRegressionModel = { train(input, numIterations, stepSize, regParam, 1.0) } @@ -155,21 +192,23 @@ object RidgeRegressionWithSGD { * @return a RidgeRegressionModel which has the weights and offset from training. */ def train( - input: RDD[LabeledPoint], - numIterations: Int) - : RidgeRegressionModel = + input: RDD[LabeledPoint], + numIterations: Int) + : RidgeRegressionModel = { train(input, numIterations, 1.0, 1.0, 1.0) } def main(args: Array[String]) { if (args.length != 5) { - println("Usage: RidgeRegression ") + println("Usage: RidgeRegression " + + " ") System.exit(1) } val sc = new SparkContext(args(0), "RidgeRegression") val data = MLUtils.loadLabeledData(sc, args(1)) - val model = RidgeRegressionWithSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) + val model = RidgeRegressionWithSGD.train(data, args(4).toInt, args(2).toDouble, + args(3).toDouble) sc.stop() } diff --git a/mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala index 20e1656beb0..9f48477f84a 100644 --- a/mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala @@ -17,20 +17,19 @@ package spark.mllib.util +import scala.collection.JavaConversions._ import scala.util.Random import org.jblas.DoubleMatrix import spark.{RDD, SparkContext} import spark.mllib.regression.LabeledPoint -import scala.collection.JavaConversions._ import spark.mllib.regression.LabeledPoint /** * Generate sample data used for Linear Data. This class generates * uniformly random values for every feature and adds Gaussian noise with mean `eps` to the * response variable `Y`. - * */ object LinearDataGenerator { @@ -47,8 +46,9 @@ object LinearDataGenerator { intercept: Double, weights: Array[Double], nPoints: Int, - seed: Int): java.util.List[LabeledPoint] = { - seqAsJavaList(generateLinearInput(intercept, weights, nPoints, seed)) + seed: Int, + eps: Double): java.util.List[LabeledPoint] = { + seqAsJavaList(generateLinearInput(intercept, weights, nPoints, seed, eps)) } /** @@ -70,10 +70,10 @@ object LinearDataGenerator { val rnd = new Random(seed) val weightsMat = new DoubleMatrix(1, weights.length, weights:_*) val x = Array.fill[Array[Double]](nPoints)( - Array.fill[Double](weights.length)(rnd.nextGaussian())) - val y = x.map(xi => + Array.fill[Double](weights.length)(2 * rnd.nextDouble - 1.0)) + val y = x.map { xi => (new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) + intercept + eps * rnd.nextGaussian() - ) + } y.zip(x).map(p => LabeledPoint(p._1, p._2)) } @@ -95,19 +95,15 @@ object LinearDataGenerator { nexamples: Int, nfeatures: Int, eps: Double, - weights: Array[Double] = Array[Double](), nparts: Int = 2, intercept: Double = 0.0) : RDD[LabeledPoint] = { org.jblas.util.Random.seed(42) // Random values distributed uniformly in [-0.5, 0.5] val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) - (0 until weights.length.max(nfeatures)).map(i => w.put(i, 0, weights(i))) - val data: RDD[LabeledPoint] = sc.parallelize(0 until nparts, nparts).flatMap { p => - val seed = 42+p + val seed = 42 + p val examplesInPartition = nexamples / nparts - generateLinearInput(intercept, w.toArray, examplesInPartition, seed, eps) } data @@ -115,7 +111,7 @@ object LinearDataGenerator { def main(args: Array[String]) { if (args.length < 2) { - println("Usage: RidgeRegressionGenerator " + + println("Usage: LinearDataGenerator " + " [num_examples] [num_features] [num_partitions]") System.exit(1) } @@ -127,7 +123,7 @@ object LinearDataGenerator { val parts: Int = if (args.length > 4) args(4).toInt else 2 val eps = 10 - val sc = new SparkContext(sparkMaster, "RidgeRegressionDataGenerator") + val sc = new SparkContext(sparkMaster, "LinearDataGenerator") val data = generateLinearRDD(sc, nexamples, nfeatures, eps, nparts = parts) MLUtils.saveLabeledData(data, outputPath) diff --git a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala index 4e030a81b48..a8e6ae9953d 100644 --- a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/spark/mllib/util/MLUtils.scala @@ -72,16 +72,16 @@ object MLUtils { * 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): + def computeStats(data: RDD[LabeledPoint], nfeatures: Int, nexamples: Long): (Double, DoubleMatrix, DoubleMatrix) = { - val yMean: Double = data.map { case (y, features) => y }.reduce(_ + _) / nexamples + val yMean: Double = data.map { labeledPoint => labeledPoint.label }.reduce(_ + _) / nexamples // NOTE: We shuffle X by column here to compute column sum and sum of squares. - val xColSumSq: RDD[(Int, (Double, Double))] = data.flatMap { case(y, features) => - val nCols = features.length + val xColSumSq: RDD[(Int, (Double, Double))] = data.flatMap { labeledPoint => + val nCols = labeledPoint.features.length // Traverse over every column and emit (col, value, value^2) Iterator.tabulate(nCols) { i => - (i, (features(i), features(i)*features(i))) + (i, (labeledPoint.features(i), labeledPoint.features(i)*labeledPoint.features(i))) } }.reduceByKey { case(x1, x2) => (x1._1 + x2._1, x1._2 + x2._2) diff --git a/mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java b/mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java index 428902e85c9..5863140bafe 100644 --- a/mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java +++ b/mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java @@ -63,9 +63,9 @@ public void runLassoUsingConstructor() { double[] weights = {-1.5, 1.0e-2}; JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, - weights, nPoints, 42), 2).cache(); + weights, nPoints, 42, 0.1), 2).cache(); List validationData = - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17); + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); LassoWithSGD lassoSGDImpl = new LassoWithSGD(); lassoSGDImpl.optimizer().setStepSize(1.0) @@ -84,9 +84,9 @@ public void runLassoUsingStaticMethods() { double[] weights = {-1.5, 1.0e-2}; JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, - weights, nPoints, 42), 2).cache(); + weights, nPoints, 42, 0.1), 2).cache(); List validationData = - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17); + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); LassoModel model = LassoWithSGD.train(testRDD.rdd(), 100, 1.0, 0.01, 1.0); diff --git a/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java index 9642e898449..50716c78617 100644 --- a/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java @@ -30,68 +30,65 @@ import spark.mllib.util.LinearDataGenerator; public class JavaLinearRegressionSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaLinearRegressionSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - System.clearProperty("spark.driver.port"); - } - - int validatePrediction(List validationData, LinearRegressionModel model) { - int numAccurate = 0; - for (LabeledPoint point: validationData) { - Double prediction = model.predict(point.features()); - // A prediction is off if the prediction is more than 0.5 away from expected value. - if (Math.abs(prediction - point.label()) <= 0.5) { - numAccurate++; - } + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaLinearRegressionSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + System.clearProperty("spark.driver.port"); + } + + int validatePrediction(List validationData, LinearRegressionModel model) { + int numAccurate = 0; + for (LabeledPoint point: validationData) { + Double prediction = model.predict(point.features()); + // A prediction is off if the prediction is more than 0.5 away from expected value. + if (Math.abs(prediction - point.label()) <= 0.5) { + numAccurate++; } - return numAccurate; - } - - @Test - public void runLinearRegressionUsingConstructor() { - int nPoints = 10000; - double A = 2.0; - double[] weights = {-1.5, 1.0e-2}; - - JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, - weights, nPoints, 42), 2).cache(); - List validationData = - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17); - - LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(); - linSGDImpl.optimizer().setStepSize(1.0) - .setRegParam(0.01) - .setNumIterations(20); - LinearRegressionModel model = linSGDImpl.run(testRDD.rdd()); - - int numAccurate = validatePrediction(validationData, model); - Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); - } - - @Test - public void runLinearRegressionUsingStaticMethods() { - int nPoints = 10000; - double A = 2.0; - double[] weights = {-1.5, 1.0e-2}; - - JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, - weights, nPoints, 42), 2).cache(); - List validationData = - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17); - - LinearRegressionModel model = LinearRegressionWithSGD.train(testRDD.rdd(), 100, 1.0, 1.0); - - int numAccurate = validatePrediction(validationData, model); - Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); } + return numAccurate; + } + + @Test + public void runLinearRegressionUsingConstructor() { + int nPoints = 100; + double A = 3.0; + double[] weights = {10, 10}; + + JavaRDD testRDD = sc.parallelize( + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); + List validationData = + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); + + LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(); + LinearRegressionModel model = linSGDImpl.run(testRDD.rdd()); + + int numAccurate = validatePrediction(validationData, model); + Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); + } + + @Test + public void runLinearRegressionUsingStaticMethods() { + int nPoints = 100; + double A = 3.0; + double[] weights = {10, 10}; + + JavaRDD testRDD = sc.parallelize( + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); + List validationData = + LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); + + LinearRegressionModel model = LinearRegressionWithSGD.train(testRDD.rdd(), 100); + + int numAccurate = validatePrediction(validationData, model); + Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); + } } diff --git a/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java b/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java index 5df6d8076d4..2c0aabad309 100644 --- a/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java +++ b/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java @@ -25,73 +25,86 @@ import org.junit.Before; import org.junit.Test; +import org.jblas.DoubleMatrix; + import spark.api.java.JavaRDD; import spark.api.java.JavaSparkContext; import spark.mllib.util.LinearDataGenerator; public class JavaRidgeRegressionSuite implements Serializable { - private transient JavaSparkContext sc; - - @Before - public void setUp() { - sc = new JavaSparkContext("local", "JavaRidgeRegressionSuite"); - } - - @After - public void tearDown() { - sc.stop(); - sc = null; - System.clearProperty("spark.driver.port"); - } - - int validatePrediction(List validationData, RidgeRegressionModel model) { - int numAccurate = 0; - for (LabeledPoint point: validationData) { - Double prediction = model.predict(point.features()); - // A prediction is off if the prediction is more than 0.5 away from expected value. - if (Math.abs(prediction - point.label()) <= 0.5) { - numAccurate++; - } - } - return numAccurate; - } - - @Test - public void runRidgeRegressionUsingConstructor() { - int nPoints = 10000; - double A = 2.0; - double[] weights = {-1.5, 1.0e-2}; - - JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, - weights, nPoints, 42), 2).cache(); - List validationData = - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17); - - RidgeRegressionWithSGD ridgeSGDImpl = new RidgeRegressionWithSGD(); - ridgeSGDImpl.optimizer().setStepSize(1.0) - .setRegParam(0.01) - .setNumIterations(20); - RidgeRegressionModel model = ridgeSGDImpl.run(testRDD.rdd()); - - int numAccurate = validatePrediction(validationData, model); - Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaRidgeRegressionSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + System.clearProperty("spark.driver.port"); + } + + double predictionError(List validationData, RidgeRegressionModel model) { + double errorSum = 0; + for (LabeledPoint point: validationData) { + Double prediction = model.predict(point.features()); + errorSum += (prediction - point.label()) * (prediction - point.label()); } - - @Test - public void runRidgeRegressionUsingStaticMethods() { - int nPoints = 10000; - double A = 2.0; - double[] weights = {-1.5, 1.0e-2}; - - JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, - weights, nPoints, 42), 2).cache(); - List validationData = - LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17); - - RidgeRegressionModel model = RidgeRegressionWithSGD.train(testRDD.rdd(), 100, 1.0, 0.01, 1.0); - - int numAccurate = validatePrediction(validationData, model); - Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); - } - + return errorSum / validationData.size(); + } + + List generateRidgeData(int numPoints, int nfeatures, double eps) { + org.jblas.util.Random.seed(42); + // Pick weights as random values distributed uniformly in [-0.5, 0.5] + DoubleMatrix w = DoubleMatrix.rand(nfeatures, 1).subi(0.5); + // Set first two weights to eps + w.put(0, 0, eps); + w.put(1, 0, eps); + return LinearDataGenerator.generateLinearInputAsList(0.0, w.data, numPoints, 42, eps); + } + + @Test + public void runRidgeRegressionUsingConstructor() { + int nexamples = 200; + int nfeatures = 20; + double eps = 10.0; + List data = generateRidgeData(2*nexamples, nfeatures, eps); + + JavaRDD testRDD = sc.parallelize(data.subList(0, nexamples)); + List validationData = data.subList(nexamples, 2*nexamples); + + RidgeRegressionWithSGD ridgeSGDImpl = new RidgeRegressionWithSGD(); + ridgeSGDImpl.optimizer().setStepSize(1.0) + .setRegParam(0.0) + .setNumIterations(200); + RidgeRegressionModel model = ridgeSGDImpl.run(testRDD.rdd()); + double unRegularizedErr = predictionError(validationData, model); + + ridgeSGDImpl.optimizer().setRegParam(0.1); + model = ridgeSGDImpl.run(testRDD.rdd()); + double regularizedErr = predictionError(validationData, model); + + Assert.assertTrue(regularizedErr < unRegularizedErr); + } + + @Test + public void runRidgeRegressionUsingStaticMethods() { + int nexamples = 200; + int nfeatures = 20; + double eps = 10.0; + List data = generateRidgeData(2*nexamples, nfeatures, eps); + + JavaRDD testRDD = sc.parallelize(data.subList(0, nexamples)); + List validationData = data.subList(nexamples, 2*nexamples); + + RidgeRegressionModel model = RidgeRegressionWithSGD.train(testRDD.rdd(), 200, 1.0, 0.0); + double unRegularizedErr = predictionError(validationData, model); + + model = RidgeRegressionWithSGD.train(testRDD.rdd(), 200, 1.0, 0.1); + double regularizedErr = predictionError(validationData, model); + + Assert.assertTrue(regularizedErr < unRegularizedErr); + } } diff --git a/mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala index 3d22b7d3856..acc48a3283c 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala @@ -36,10 +36,19 @@ class LinearRegressionSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } - // Test if we can correctly learn Y = 3 + 10*X1 + 10*X2 when - // X1 and X2 are collinear. - test("multi-collinear variables") { - val testRDD = LinearDataGenerator.generateLinearRDD(sc, 100, 2, 0.0, Array(10.0, 10.0), intercept=3.0).cache() + def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) { + val numOffPredictions = predictions.zip(input).filter { case (prediction, expected) => + // A prediction is off if the prediction is more than 0.5 away from expected value. + math.abs(prediction - expected.label) > 0.5 + }.size + // At least 80% of the predictions should be on. + assert(numOffPredictions < input.length / 5) + } + + // Test if we can correctly learn Y = 3 + 10*X1 + 10*X2 + test("linear regression") { + val testRDD = sc.parallelize(LinearDataGenerator.generateLinearInput( + 3.0, Array(10.0, 10.0), 100, 42), 2).cache() val linReg = new LinearRegressionWithSGD() linReg.optimizer.setNumIterations(1000).setStepSize(1.0) @@ -49,5 +58,15 @@ class LinearRegressionSuite extends FunSuite with BeforeAndAfterAll { assert(model.weights.length === 2) assert(model.weights(0) >= 9.0 && model.weights(0) <= 11.0) assert(model.weights(1) >= 9.0 && model.weights(1) <= 11.0) + + val validationData = LinearDataGenerator.generateLinearInput( + 3.0, Array(10.0, 10.0), 100, 17) + val validationRDD = sc.parallelize(validationData, 2).cache() + + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_.features)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } } diff --git a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala index 0237ccdf87e..c4820357062 100644 --- a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala @@ -20,6 +20,7 @@ package spark.mllib.regression import scala.collection.JavaConversions._ import scala.util.Random +import org.jblas.DoubleMatrix import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite @@ -27,7 +28,6 @@ import spark.SparkContext import spark.SparkContext._ import spark.mllib.util.LinearDataGenerator - class RidgeRegressionSuite extends FunSuite with BeforeAndAfterAll { @transient private var sc: SparkContext = _ @@ -40,31 +40,51 @@ class RidgeRegressionSuite extends FunSuite with BeforeAndAfterAll { System.clearProperty("spark.driver.port") } - // Test if we can correctly learn Y = 3 + 10*X1 + 10*X2 when - // X1 and X2 are collinear. - test("multi-collinear variables") { - val testRDD = LinearDataGenerator.generateLinearRDD(sc, 100, 2, 0.0, Array(10.0, 10.0), intercept=3.0).cache() - val ridgeReg = new RidgeRegressionWithSGD() - ridgeReg.optimizer.setNumIterations(1000).setRegParam(0.0).setStepSize(1.0) + def predictionError(predictions: Seq[Double], input: Seq[LabeledPoint]) = { + predictions.zip(input).map { case (prediction, expected) => + (prediction - expected.label) * (prediction - expected.label) + }.reduceLeft(_ + _) / predictions.size + } - val model = ridgeReg.run(testRDD) + test("regularization with skewed weights") { + val nexamples = 200 + val nfeatures = 20 + val eps = 10 - assert(model.intercept >= 2.5 && model.intercept <= 3.5) - assert(model.weights.length === 2) - assert(model.weights(0) >= 9.0 && model.weights(0) <= 11.0) - assert(model.weights(1) >= 9.0 && model.weights(1) <= 11.0) - } + org.jblas.util.Random.seed(42) + // Pick weights as random values distributed uniformly in [-0.5, 0.5] + val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) + // Set first two weights to eps + w.put(0, 0, eps) + w.put(1, 0, eps) - test("multi-collinear variables with regularization") { - val testRDD = LinearDataGenerator.generateLinearRDD(sc, 100, 2, 0.0, Array(10.0, 10.0), intercept=3.0).cache() - val ridgeReg = new RidgeRegressionWithSGD() - ridgeReg.optimizer.setNumIterations(1000).setRegParam(1.0).setStepSize(1.0) + // Use half of data for training and other half for validation + val data = LinearDataGenerator.generateLinearInput(3.0, w.toArray, 2*nexamples, 42, eps) + val testData = data.take(nexamples) + val validationData = data.takeRight(nexamples) - val model = ridgeReg.run(testRDD) + val testRDD = sc.parallelize(testData, 2).cache() + val validationRDD = sc.parallelize(validationData, 2).cache() + + // First run without regularization. + val linearReg = new LinearRegressionWithSGD() + linearReg.optimizer.setNumIterations(200) + .setStepSize(1.0) + + val linearModel = linearReg.run(testRDD) + val linearErr = predictionError( + linearModel.predict(validationRDD.map(_.features)).collect(), validationData) + + val ridgeReg = new RidgeRegressionWithSGD() + ridgeReg.optimizer.setNumIterations(200) + .setRegParam(0.1) + .setStepSize(1.0) + val ridgeModel = ridgeReg.run(testRDD) + val ridgeErr = predictionError( + ridgeModel.predict(validationRDD.map(_.features)).collect(), validationData) - assert(model.intercept <= 5.0) - assert(model.weights.length === 2) - assert(model.weights(0) <= 4.0) - assert(model.weights(1) <= 3.0) + // Ridge CV-error should be lower than linear regression + assert(ridgeErr < linearErr, + "ridgeError (" + ridgeErr + ") was not less than linearError(" + linearErr + ")") } } From dc06b528790c69b2e6de85cba84266fea81dd4f4 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 25 Aug 2013 23:14:35 -0700 Subject: [PATCH 0846/2521] Add an option to turn off data validation, test it. Also moves addIntercept to have default true to make it similar to validateData option --- .../classification/LogisticRegression.scala | 9 ++++----- .../scala/spark/mllib/classification/SVM.scala | 9 ++++----- .../regression/GeneralizedLinearAlgorithm.scala | 16 +++++++++++++--- .../scala/spark/mllib/regression/Lasso.scala | 9 ++++----- .../spark/mllib/classification/SVMSuite.scala | 3 +++ 5 files changed, 28 insertions(+), 18 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala index 474ca6e97c9..482e4a67457 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala @@ -54,8 +54,7 @@ class LogisticRegressionWithSGD private ( var stepSize: Double, var numIterations: Int, var regParam: Double, - var miniBatchFraction: Double, - var addIntercept: Boolean) + var miniBatchFraction: Double) extends GeneralizedLinearAlgorithm[LogisticRegressionModel] with Serializable { @@ -71,7 +70,7 @@ class LogisticRegressionWithSGD private ( /** * Construct a LogisticRegression object with default parameters */ - def this() = this(1.0, 100, 0.0, 1.0, true) + def this() = this(1.0, 100, 0.0, 1.0) def createModel(weights: Array[Double], intercept: Double) = { new LogisticRegressionModel(weights, intercept) @@ -108,7 +107,7 @@ object LogisticRegressionWithSGD { initialWeights: Array[Double]) : LogisticRegressionModel = { - new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction, true).run( + new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction).run( input, initialWeights) } @@ -131,7 +130,7 @@ object LogisticRegressionWithSGD { miniBatchFraction: Double) : LogisticRegressionModel = { - new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction, true).run( + new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction).run( input) } diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/spark/mllib/classification/SVM.scala index b680d81e86c..69393cd7b07 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/spark/mllib/classification/SVM.scala @@ -54,8 +54,7 @@ class SVMWithSGD private ( var stepSize: Double, var numIterations: Int, var regParam: Double, - var miniBatchFraction: Double, - var addIntercept: Boolean) + var miniBatchFraction: Double) extends GeneralizedLinearAlgorithm[SVMModel] with Serializable { val gradient = new HingeGradient() @@ -71,7 +70,7 @@ class SVMWithSGD private ( /** * Construct a SVM object with default parameters */ - def this() = this(1.0, 100, 1.0, 1.0, true) + def this() = this(1.0, 100, 1.0, 1.0) def createModel(weights: Array[Double], intercept: Double) = { new SVMModel(weights, intercept) @@ -107,7 +106,7 @@ object SVMWithSGD { initialWeights: Array[Double]) : SVMModel = { - new SVMWithSGD(stepSize, numIterations, regParam, miniBatchFraction, true).run(input, + new SVMWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input, initialWeights) } @@ -131,7 +130,7 @@ object SVMWithSGD { miniBatchFraction: Double) : SVMModel = { - new SVMWithSGD(stepSize, numIterations, regParam, miniBatchFraction, true).run(input) + new SVMWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input) } /** diff --git a/mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 03f991df399..d164d415d6a 100644 --- a/mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -87,13 +87,15 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] val optimizer: Optimizer + protected var addIntercept: Boolean = true + + protected var validateData: Boolean = true + /** * Create a model given the weights and intercept */ protected def createModel(weights: Array[Double], intercept: Double): M - protected var addIntercept: Boolean - /** * Set if the algorithm should add an intercept. Default true. */ @@ -102,6 +104,14 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] this } + /** + * Set if the algorithm should validate data before training. Default true. + */ + def setValidateData(validateData: Boolean): this.type = { + this.validateData = validateData + this + } + /** * Run the algorithm with the configured parameters on an input * RDD of LabeledPoint entries. @@ -119,7 +129,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] def run(input: RDD[LabeledPoint], initialWeights: Array[Double]) : M = { // Check the data properties before running the optimizer - if (!validators.forall(func => func(input))) { + if (validateData && !validators.forall(func => func(input))) { throw new SparkException("Input validation failed.") } diff --git a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala index 6bbc990a5a0..89f791e85a8 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/spark/mllib/regression/Lasso.scala @@ -48,8 +48,7 @@ class LassoWithSGD private ( var stepSize: Double, var numIterations: Int, var regParam: Double, - var miniBatchFraction: Double, - var addIntercept: Boolean) + var miniBatchFraction: Double) extends GeneralizedLinearAlgorithm[LassoModel] with Serializable { @@ -63,7 +62,7 @@ class LassoWithSGD private ( /** * Construct a Lasso object with default parameters */ - def this() = this(1.0, 100, 1.0, 1.0, true) + def this() = this(1.0, 100, 1.0, 1.0) def createModel(weights: Array[Double], intercept: Double) = { new LassoModel(weights, intercept) @@ -98,7 +97,7 @@ object LassoWithSGD { initialWeights: Array[Double]) : LassoModel = { - new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction, true).run(input, + new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input, initialWeights) } @@ -121,7 +120,7 @@ object LassoWithSGD { miniBatchFraction: Double) : LassoModel = { - new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction, true).run(input) + new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input) } /** diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala index 8fa9e4639b4..894ae458ad0 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala @@ -162,5 +162,8 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { intercept[spark.SparkException] { val model = SVMWithSGD.train(testRDDInvalid, 100) } + + // Turning off data validation should not throw an exception + val noValidationModel = new SVMWithSGD().setValidateData(false).run(testRDDInvalid) } } From c0b4095ee8b5363d157094ff11bab2c47e77dbf4 Mon Sep 17 00:00:00 2001 From: "Y.CORP.YAHOO.COM\\tgraves" Date: Mon, 26 Aug 2013 12:48:37 -0500 Subject: [PATCH 0847/2521] Change to use Yarn appropriate directories rather then /tmp or the user specified spark.local.dir --- .../spark/deploy/yarn/ApplicationMaster.scala | 17 +++++++++++++++++ .../main/scala/spark/deploy/yarn/Client.scala | 4 ++++ .../spark/deploy/yarn/WorkerRunnable.scala | 4 ++++ 3 files changed, 25 insertions(+) diff --git a/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala index 15dbd1c0fb2..cda3c21c586 100644 --- a/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -47,6 +47,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e private var isFinished:Boolean = false def run() { + // setup the directories so things go to yarn approved directories rather + // then user specified and /tmp + System.setProperty("spark.local.dir", getLocalDirs()) appAttemptId = getApplicationAttemptId() resourceManager = registerWithResourceManager() @@ -89,6 +92,20 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e System.exit(0) } + + /** Get the Yarn approved local directories. */ + private def getLocalDirs(): String = { + // Hadoop 0.23 and 2.x have different Environment variable names for the + // local dirs, so lets check both. We assume one of the 2 is set. + // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X + var localDirs = System.getenv("LOCAL_DIRS") + val yarnLocalSysDirs = Option(System.getenv("YARN_LOCAL_DIRS")) + yarnLocalSysDirs match { + case Some(s) => localDirs = s + case None => if (localDirs == null) localDirs = "" + } + return localDirs + } private def getApplicationAttemptId(): ApplicationAttemptId = { val envs = System.getenv() diff --git a/yarn/src/main/scala/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/spark/deploy/yarn/Client.scala index 9d3860b863b..aa70a3edb81 100644 --- a/yarn/src/main/scala/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/spark/deploy/yarn/Client.scala @@ -221,6 +221,10 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl // Add Xmx for am memory JAVA_OPTS += "-Xmx" + amMemory + "m " + JAVA_OPTS += " -Djava.io.tmpdir=" + new Path(Environment.PWD.$(), + YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + + // Commenting it out for now - so that people can refer to the properties if required. Remove it once cpuset version is pushed out. // The context is, default gc for server class machines end up using all cores to do gc - hence if there are multiple containers in same // node, spark gc effects all other containers performance (which can also be other spark containers) diff --git a/yarn/src/main/scala/spark/deploy/yarn/WorkerRunnable.scala b/yarn/src/main/scala/spark/deploy/yarn/WorkerRunnable.scala index f458f2f6a1e..fe713b36296 100644 --- a/yarn/src/main/scala/spark/deploy/yarn/WorkerRunnable.scala +++ b/yarn/src/main/scala/spark/deploy/yarn/WorkerRunnable.scala @@ -75,6 +75,10 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S if (env.isDefinedAt("SPARK_JAVA_OPTS")) { JAVA_OPTS += env("SPARK_JAVA_OPTS") + " " } + + JAVA_OPTS += " -Djava.io.tmpdir=" + new Path(Environment.PWD.$(), + YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + // Commenting it out for now - so that people can refer to the properties if required. Remove it once cpuset version is pushed out. // The context is, default gc for server class machines end up using all cores to do gc - hence if there are multiple containers in same // node, spark gc effects all other containers performance (which can also be other spark containers) From 9db1e50344cc49810c8549b5b3022038c64e4dfa Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 26 Aug 2013 11:05:14 -0700 Subject: [PATCH 0848/2521] Revert "Merge pull request #841 from rxin/json" This reverts commit 1fb1b0992838c8cdd57eec45793e67a0490f1a52, reversing changes made to c69c48947d5102c81a9425cb380d861c3903685c. --- core/pom.xml | 4 + .../scala/spark/deploy/JsonProtocol.scala | 108 +++++++++--------- .../deploy/master/ui/ApplicationPage.scala | 5 +- .../spark/deploy/master/ui/IndexPage.scala | 5 +- .../spark/deploy/worker/ui/IndexPage.scala | 5 +- core/src/main/scala/spark/ui/JettyUtils.scala | 7 +- pom.xml | 5 + project/SparkBuild.scala | 1 + 8 files changed, 76 insertions(+), 64 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 3d70a195841..53696367e98 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -106,6 +106,10 @@ org.scala-lang scala-library + + net.liftweb + lift-json_2.9.2 + it.unimi.dsi fastutil diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index 6b71b953dd7..bd1db7c2940 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -17,7 +17,7 @@ package spark.deploy -import scala.util.parsing.json.{JSONArray, JSONObject, JSONType} +import net.liftweb.json.JsonDSL._ import spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} import spark.deploy.master.{ApplicationInfo, WorkerInfo} @@ -25,63 +25,61 @@ import spark.deploy.worker.ExecutorRunner private[spark] 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) + } - def writeWorkerInfo(obj: WorkerInfo): JSONType = JSONObject(Map( - "id" -> obj.id, - "host" -> obj.host, - "port" -> obj.port, - "webuiaddress" -> obj.webUiAddress, - "cores" -> obj.cores, - "coresused" -> obj.coresUsed, - "memory" -> obj.memory, - "memoryused" -> obj.memoryUsed, - "state" -> obj.state.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) + } - def writeApplicationInfo(obj: ApplicationInfo): JSONType = JSONObject(Map( - "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 - )) + def writeApplicationDescription(obj: ApplicationDescription) = { + ("name" -> obj.name) ~ + ("cores" -> obj.maxCores) ~ + ("memoryperslave" -> obj.memoryPerSlave) ~ + ("user" -> obj.user) + } - def writeApplicationDescription(obj: ApplicationDescription): JSONType = JSONObject(Map( - "name" -> obj.name, - "cores" -> obj.maxCores, - "memoryperslave" -> obj.memoryPerSlave, - "user" -> obj.user - )) + def writeExecutorRunner(obj: ExecutorRunner) = { + ("id" -> obj.execId) ~ + ("memory" -> obj.memory) ~ + ("appid" -> obj.appId) ~ + ("appdesc" -> writeApplicationDescription(obj.appDesc)) + } - def writeExecutorRunner(obj: ExecutorRunner): JSONType = JSONObject(Map( - "id" -> obj.execId, - "memory" -> obj.memory, - "appid" -> obj.appId, - "appdesc" -> writeApplicationDescription(obj.appDesc) - )) + def writeMasterState(obj: MasterStateResponse) = { + ("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)) + } - def writeMasterState(obj: MasterStateResponse): JSONType = JSONObject(Map( - "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" -> JSONArray(obj.activeApps.toList.map(writeApplicationInfo)), - "completedapps" -> JSONArray(obj.completedApps.toList.map(writeApplicationInfo)) - )) - - def writeWorkerState(obj: WorkerStateResponse): JSONType = JSONObject(Map( - "id" -> obj.workerId, - "masterurl" -> obj.masterUrl, - "masterwebuiurl" -> obj.masterWebUiUrl, - "cores" -> obj.cores, - "coresused" -> obj.coresUsed, - "memory" -> obj.memory, - "memoryused" -> obj.memoryUsed, - "executors" -> JSONArray(obj.executors.toList.map(writeExecutorRunner)), - "finishedexecutors" -> JSONArray(obj.finishedExecutors.toList.map(writeExecutorRunner)) - )) + def writeWorkerState(obj: WorkerStateResponse) = { + ("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)) + } } 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 494a9b914d7..405a1ec3a60 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala @@ -17,7 +17,6 @@ package spark.deploy.master.ui -import scala.util.parsing.json.JSONType import scala.xml.Node import akka.dispatch.Await @@ -26,6 +25,8 @@ import akka.util.duration._ import javax.servlet.http.HttpServletRequest +import net.liftweb.json.JsonAST.JValue + import spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import spark.deploy.JsonProtocol import spark.deploy.master.ExecutorInfo @@ -36,7 +37,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { implicit val timeout = parent.timeout /** Executor details for a particular application */ - def renderJson(request: HttpServletRequest): JSONType = { + def renderJson(request: HttpServletRequest): JValue = { val appId = request.getParameter("appId") val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, 30 seconds) 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 28e421e3bc1..2000211b983 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala @@ -19,13 +19,14 @@ package spark.deploy.master.ui import javax.servlet.http.HttpServletRequest -import scala.util.parsing.json.JSONType import scala.xml.Node import akka.dispatch.Await import akka.pattern.ask import akka.util.duration._ +import net.liftweb.json.JsonAST.JValue + import spark.Utils import spark.deploy.DeployWebUI import spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} @@ -37,7 +38,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { val master = parent.masterActorRef implicit val timeout = parent.timeout - def renderJson(request: HttpServletRequest): JSONType = { + def renderJson(request: HttpServletRequest): JValue = { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, 30 seconds) JsonProtocol.writeMasterState(state) 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 02993d58a0a..b67059068bf 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala @@ -19,13 +19,14 @@ package spark.deploy.worker.ui import javax.servlet.http.HttpServletRequest -import scala.util.parsing.json.JSONType import scala.xml.Node import akka.dispatch.Await import akka.pattern.ask import akka.util.duration._ +import net.liftweb.json.JsonAST.JValue + import spark.Utils import spark.deploy.JsonProtocol import spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} @@ -38,7 +39,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { val worker = parent.worker val timeout = parent.timeout - def renderJson(request: HttpServletRequest): JSONType = { + def renderJson(request: HttpServletRequest): JValue = { val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] val workerState = Await.result(stateFuture, 30 seconds) JsonProtocol.writeWorkerState(workerState) diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/spark/ui/JettyUtils.scala index ba58f357299..f66fe399052 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/spark/ui/JettyUtils.scala @@ -21,9 +21,10 @@ import javax.servlet.http.{HttpServletResponse, HttpServletRequest} import scala.annotation.tailrec import scala.util.{Try, Success, Failure} -import scala.util.parsing.json.JSONType import scala.xml.Node +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 @@ -38,8 +39,8 @@ private[spark] object JettyUtils extends Logging { type Responder[T] = HttpServletRequest => T // Conversions from various types of Responder's to jetty Handlers - implicit def jsonResponderToHandler(responder: Responder[JSONType]): Handler = - createHandler(responder, "text/json", (in: JSONType) => in.toString) + 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", (in: Seq[Node]) => "" + in.toString) diff --git a/pom.xml b/pom.xml index de883e2abce..85bcd8696c2 100644 --- a/pom.xml +++ b/pom.xml @@ -260,6 +260,11 @@ 10.4.2.0 test + + net.liftweb + lift-json_2.9.2 + 2.5 + com.codahale.metrics metrics-core diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index fbeae277076..5fdcf19b624 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -182,6 +182,7 @@ object SparkBuild extends Build { "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", "org.apache.mesos" % "mesos" % "0.12.1", "io.netty" % "netty-all" % "4.0.0.Beta2", "org.apache.derby" % "derby" % "10.4.2.0" % "test", From a77e0abb96c4d4cd35822656e57b99320a16aec2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 26 Aug 2013 11:21:03 -0700 Subject: [PATCH 0849/2521] Added worker state to the cluster master JSON ui. --- core/src/main/scala/spark/deploy/JsonProtocol.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index bd1db7c2940..f8dcf025b44 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -33,7 +33,8 @@ private[spark] object JsonProtocol { ("cores" -> obj.cores) ~ ("coresused" -> obj.coresUsed) ~ ("memory" -> obj.memory) ~ - ("memoryused" -> obj.memoryUsed) + ("memoryused" -> obj.memoryUsed) ~ + ("state" -> obj.state.toString) } def writeApplicationInfo(obj: ApplicationInfo) = { From dfb4c697bcfcbbe7e0959894244e71f38edd79f9 Mon Sep 17 00:00:00 2001 From: "Y.CORP.YAHOO.COM\\tgraves" Date: Mon, 26 Aug 2013 13:57:01 -0500 Subject: [PATCH 0850/2521] Throw exception if the yarn local dirs isn't set --- .../main/scala/spark/deploy/yarn/ApplicationMaster.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala index cda3c21c586..33e6293a6b6 100644 --- a/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -102,7 +102,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e val yarnLocalSysDirs = Option(System.getenv("YARN_LOCAL_DIRS")) yarnLocalSysDirs match { case Some(s) => localDirs = s - case None => if (localDirs == null) localDirs = "" + case None => { + if ((localDirs == null) || (localDirs.isEmpty())) { + throw new Exception("Yarn Local dirs can't be empty") + } + } } return localDirs } From 6dd64e8bb2256b56e0908c628ebdb3b533adf432 Mon Sep 17 00:00:00 2001 From: "Y.CORP.YAHOO.COM\\tgraves" Date: Mon, 26 Aug 2013 14:29:24 -0500 Subject: [PATCH 0851/2521] Update docs and remove old reference to --user option --- docs/running-on-yarn.md | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 6bada9bdd75..7a344b3ce2f 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -57,7 +57,6 @@ The command to launch the YARN Client is as follows: --master-memory \ --worker-memory \ --worker-cores \ - --user \ --queue For example: @@ -77,5 +76,4 @@ The above starts a YARN Client programs which periodically polls the Application - 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. +- The local directories used for spark will be the local directories configured for YARN (Hadoop Yarn config yarn.nodemanager.local-dirs). If the user specifies spark.local.dir, it will be ignored. From c9464c74a1fbff9856ea3850bb5e070c1322bab4 Mon Sep 17 00:00:00 2001 From: "Y.CORP.YAHOO.COM\\tgraves" Date: Mon, 26 Aug 2013 16:44:27 -0500 Subject: [PATCH 0852/2521] Add ability for user to specify environment variables --- docs/running-on-yarn.md | 6 ++++++ yarn/src/main/scala/spark/deploy/yarn/Client.scala | 4 +++- yarn/src/main/scala/spark/deploy/yarn/WorkerRunnable.scala | 5 ++++- 3 files changed, 13 insertions(+), 2 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 6bada9bdd75..f9525743c81 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -42,6 +42,12 @@ This will build the shaded (consolidated) jar. Typically something like : If you want to test out the YARN deployment mode, you can use the current Spark examples. A `spark-examples_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}` file can be generated by running `sbt/sbt package`. NOTE: since the documentation you're reading is for Spark version {{site.SPARK_VERSION}}, we are assuming here that you have downloaded Spark {{site.SPARK_VERSION}} or checked it out of source control. If you are using a different version of Spark, the version numbers in the jar generated by the sbt package command will obviously be different. +# Configuration + +Most of the configs are the same for Spark on YARN as other deploys. See the Configuration page for more information on those. These are configs that are specific to SPARK on YARN. + +* `SPARK_YARN_USER_ENV`, environment variables to add to the Spark processes launched on YARN. This can be a comma separated list of environment variables. ie SPARK_YARN_USER_ENV="JAVA_HOME=/jdk64,FOO=bar" + # 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. diff --git a/yarn/src/main/scala/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/spark/deploy/yarn/Client.scala index 9d3860b863b..854ff53b099 100644 --- a/yarn/src/main/scala/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/spark/deploy/yarn/Client.scala @@ -185,6 +185,8 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl env("SPARK_YARN_LOG4J_SIZE") = log4jConfLocalRes.getSize().toString() } + // allow users to specify some environment variables + Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV")) // Add each SPARK-* key to the environment System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } @@ -241,7 +243,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl // Command for the ApplicationMaster var javaCommand = "java"; val javaHome = System.getenv("JAVA_HOME") - if (javaHome != null && !javaHome.isEmpty()) { + if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")){ javaCommand = Environment.JAVA_HOME.$() + "/bin/java" } diff --git a/yarn/src/main/scala/spark/deploy/yarn/WorkerRunnable.scala b/yarn/src/main/scala/spark/deploy/yarn/WorkerRunnable.scala index f458f2f6a1e..3e007509e6e 100644 --- a/yarn/src/main/scala/spark/deploy/yarn/WorkerRunnable.scala +++ b/yarn/src/main/scala/spark/deploy/yarn/WorkerRunnable.scala @@ -104,7 +104,7 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S var javaCommand = "java"; val javaHome = System.getenv("JAVA_HOME") - if (javaHome != null && !javaHome.isEmpty()) { + if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) { javaCommand = Environment.JAVA_HOME.$() + "/bin/java" } @@ -187,6 +187,9 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S Apps.addToEnvironment(env, Environment.CLASSPATH.name, "$CLASSPATH") Client.populateHadoopClasspath(yarnConf, env) + // allow users to specify some environment variables + Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV")) + System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } return env } From 63dc635de6e7a31095b3d246899a657c665e4ed7 Mon Sep 17 00:00:00 2001 From: "Y.CORP.YAHOO.COM\\tgraves" Date: Mon, 26 Aug 2013 17:06:20 -0500 Subject: [PATCH 0853/2521] fix typos --- docs/running-on-yarn.md | 2 +- yarn/src/main/scala/spark/deploy/yarn/Client.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index f9525743c81..cac9c5e4b6f 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -46,7 +46,7 @@ If you want to test out the YARN deployment mode, you can use the current Spark Most of the configs are the same for Spark on YARN as other deploys. See the Configuration page for more information on those. These are configs that are specific to SPARK on YARN. -* `SPARK_YARN_USER_ENV`, environment variables to add to the Spark processes launched on YARN. This can be a comma separated list of environment variables. ie SPARK_YARN_USER_ENV="JAVA_HOME=/jdk64,FOO=bar" +* `SPARK_YARN_USER_ENV`, to add environment variables to the Spark processes launched on YARN. This can be a comma separated list of environment variables. ie SPARK_YARN_USER_ENV="JAVA_HOME=/jdk64,FOO=bar" # Launching Spark on YARN diff --git a/yarn/src/main/scala/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/spark/deploy/yarn/Client.scala index 854ff53b099..e84fb7c9858 100644 --- a/yarn/src/main/scala/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/spark/deploy/yarn/Client.scala @@ -243,7 +243,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl // Command for the ApplicationMaster var javaCommand = "java"; val javaHome = System.getenv("JAVA_HOME") - if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")){ + if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) { javaCommand = Environment.JAVA_HOME.$() + "/bin/java" } From cf52a3cba6cce34d5d4da3da285c9d49d4ed3ceb Mon Sep 17 00:00:00 2001 From: "Y.CORP.YAHOO.COM\\tgraves" Date: Tue, 27 Aug 2013 11:00:21 -0500 Subject: [PATCH 0854/2521] Allow for Executors to have different directories then the Spark Master for Yarn --- .../main/scala/spark/executor/Executor.scala | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 036c7191adc..9e0356a711b 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -60,6 +60,13 @@ private[spark] class Executor( System.setProperty(key, value) } + // If we are in yarn mode, systems can have different disk layouts so we must set it + // to what Yarn on this system said was available. This will be used later when SparkEnv + // created. + if (java.lang.Boolean.valueOf(System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))) { + System.setProperty("spark.local.dir", getYarnLocalDirs()) + } + // Create our ClassLoader and set it on this thread private val urlClassLoader = createClassLoader() private val replClassLoader = addReplClassLoaderIfNeeded(urlClassLoader) @@ -107,6 +114,24 @@ private[spark] class Executor( threadPool.execute(new TaskRunner(context, taskId, serializedTask)) } + /** Get the Yarn approved local directories. */ + private def getYarnLocalDirs(): String = { + // Hadoop 0.23 and 2.x have different Environment variable names for the + // local dirs, so lets check both. We assume one of the 2 is set. + // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X + var localDirs = System.getenv("LOCAL_DIRS") + val yarnLocalSysDirs = Option(System.getenv("YARN_LOCAL_DIRS")) + yarnLocalSysDirs match { + case Some(s) => localDirs = s + case None => { + if ((localDirs == null) || (localDirs.isEmpty())) { + throw new Exception("Yarn Local dirs can't be empty") + } + } + } + return localDirs + } + class TaskRunner(context: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer) extends Runnable { From 3f206bf0b56888c718983b7832677a894e36eee3 Mon Sep 17 00:00:00 2001 From: "Y.CORP.YAHOO.COM\\tgraves" Date: Tue, 27 Aug 2013 14:34:27 -0500 Subject: [PATCH 0855/2521] Updated based on review comments. --- core/src/main/scala/spark/executor/Executor.scala | 15 ++++++--------- .../spark/deploy/yarn/ApplicationMaster.scala | 15 ++++++--------- 2 files changed, 12 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 9e0356a711b..25f55b391b9 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -119,15 +119,12 @@ private[spark] class Executor( // Hadoop 0.23 and 2.x have different Environment variable names for the // local dirs, so lets check both. We assume one of the 2 is set. // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X - var localDirs = System.getenv("LOCAL_DIRS") - val yarnLocalSysDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - yarnLocalSysDirs match { - case Some(s) => localDirs = s - case None => { - if ((localDirs == null) || (localDirs.isEmpty())) { - throw new Exception("Yarn Local dirs can't be empty") - } - } + val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) + .getOrElse(Option(System.getenv("LOCAL_DIRS")) + .getOrElse("")) + + if (localDirs.isEmpty()) { + throw new Exception("Yarn Local dirs can't be empty") } return localDirs } diff --git a/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala index 33e6293a6b6..0f3b6bc1a65 100644 --- a/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala @@ -98,15 +98,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e // Hadoop 0.23 and 2.x have different Environment variable names for the // local dirs, so lets check both. We assume one of the 2 is set. // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X - var localDirs = System.getenv("LOCAL_DIRS") - val yarnLocalSysDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - yarnLocalSysDirs match { - case Some(s) => localDirs = s - case None => { - if ((localDirs == null) || (localDirs.isEmpty())) { - throw new Exception("Yarn Local dirs can't be empty") - } - } + val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) + .getOrElse(Option(System.getenv("LOCAL_DIRS")) + .getOrElse("")) + + if (localDirs.isEmpty()) { + throw new Exception("Yarn Local dirs can't be empty") } return localDirs } From 93503a7054e07cec71d8cac006e8585acb0eceb0 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 27 Aug 2013 18:16:19 -0700 Subject: [PATCH 0856/2521] Allowing RDD to select its implementation of PairRDDFunctions --- .../scala/spark/IndexedRDDFunctions.scala | 273 ++++++++++++++ .../main/scala/spark/PairRDDFunctions.scala | 44 +-- core/src/main/scala/spark/RDD.scala | 13 + core/src/main/scala/spark/SparkContext.scala | 2 +- .../scala/spark/api/java/JavaPairRDD.scala | 78 +++- .../src/main/scala/spark/rdd/IndexedRDD.scala | 340 +----------------- 6 files changed, 380 insertions(+), 370 deletions(-) create mode 100644 core/src/main/scala/spark/IndexedRDDFunctions.scala diff --git a/core/src/main/scala/spark/IndexedRDDFunctions.scala b/core/src/main/scala/spark/IndexedRDDFunctions.scala new file mode 100644 index 00000000000..8bfe9d75e1e --- /dev/null +++ b/core/src/main/scala/spark/IndexedRDDFunctions.scala @@ -0,0 +1,273 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR 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, BitSet => JBitSet, HashSet => JHashSet} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer + +import spark._ + +import spark.rdd.ShuffledRDD +import spark.rdd.IndexedRDD +import spark.rdd.BlockIndex +import spark.rdd.RDDIndex + + +class IndexedRDDFunctions[K: ClassManifest, V: ClassManifest](self: IndexedRDD[K,V]) + extends PairRDDFunctions[K,V](self) { + + /** + * Construct a new IndexedRDD that is indexed by only the keys in the RDD + */ + def reindex(): IndexedRDD[K,V] = IndexedRDD(self) + + + /** + * Pass each value in the key-value pair RDD through a map function without changing the keys; + * this also retains the original RDD's partitioning. + */ + override def mapValues[U: ClassManifest](f: V => U): RDD[(K, U)] = { + val cleanF = self.index.rdd.context.clean(f) + val newValues = self.valuesRDD.mapPartitions(_.map(values => values.map{ + case null => null + case row => row.map(x => f(x)) + }), true) + new IndexedRDD[K,U](self.index, newValues) + } + + + /** + * Pass each value in the key-value pair RDD through a flatMap function without changing the + * keys; this also retains the original RDD's partitioning. + */ + override def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): RDD[(K,U)] = { + val cleanF = self.index.rdd.context.clean(f) + val newValues = self.valuesRDD.mapPartitions(_.map(values => values.map{ + case null => null + case row => row.flatMap(x => f(x)) + }), true) + new IndexedRDD[K,U](self.index, newValues) + } + + + /** + * Generic function to combine the elements for each key using a custom set of aggregation + * functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C + * Note that V and C can be different -- for example, one might group an RDD of type + * (Int, Int) into an RDD of type (Int, Seq[Int]). Users provide three functions: + * + * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) + * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) + * - `mergeCombiners`, to combine two C's into a single one. + */ + override def combineByKey[C: ClassManifest](createCombiner: V => C, + mergeValue: (C, V) => C, + mergeCombiners: (C, C) => C, + partitioner: Partitioner, + mapSideCombine: Boolean = true, + serializerClass: String = null): RDD[(K, C)] = { + val newValues = self.valuesRDD.mapPartitions( + _.map{ groups: Seq[Seq[V]] => + groups.map{ group: Seq[V] => + if (group != null && !group.isEmpty) { + val c: C = createCombiner(group.head) + val sum: C = group.tail.foldLeft(c)(mergeValue) + Seq(sum) + } else { + null + } + } + }, true) + new IndexedRDD[K,C](self.index, newValues) + } + + + + /** + * Group the values for each key in the RDD into a single sequence. Hash-partitions the + * resulting RDD with the existing partitioner/parallelism level. + */ + override def groupByKey(partitioner: Partitioner): RDD[(K, Seq[V])] = { + val newValues = self.valuesRDD.mapPartitions(_.map{ar => ar.map{s => Seq(s)} }, true) + new IndexedRDD[K, Seq[V]](self.index, newValues) + } + + + /** + * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the + * list of values for that key in `this` as well as `other`. + */ + override def cogroup[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): + IndexedRDD[K, (Seq[V], Seq[W])] = { + //RDD[(K, (Seq[V], Seq[W]))] = { + other match { + case other: IndexedRDD[_, _] if self.index == other.index => { + // if both RDDs share exactly the same index and therefore the same super set of keys + // then we simply merge the value RDDs. + // However it is possible that both RDDs are missing a value for a given key in + // which case the returned RDD should have a null value + val newValues = + self.valuesRDD.zipPartitions(other.valuesRDD)( + (thisIter, otherIter) => { + val thisValues: Seq[Seq[V]] = thisIter.next() + assert(!thisIter.hasNext) + val otherValues: Seq[Seq[W]] = otherIter.next() + assert(!otherIter.hasNext) + // Zip the values and if both arrays are null then the key is not present and + // so the resulting value must be null (not a tuple of empty sequences) + val tmp: Seq[Seq[(Seq[V], Seq[W])]] = thisValues.view.zip(otherValues).map{ + case (null, null) => null // The key is not present in either RDD + case (a, null) => Seq((a, Seq.empty[W])) + case (null, b) => Seq((Seq.empty[V], b)) + case (a, b) => Seq((a,b)) + }.toSeq + List(tmp).iterator + }) + new IndexedRDD[K, (Seq[V], Seq[W])](self.index, newValues) + } + case other: IndexedRDD[_, _] + if self.index.rdd.partitioner == other.index.rdd.partitioner => { + // If both RDDs are indexed using different indices but with the same partitioners + // then we we need to first merge the indicies and then use the merged index to + // merge the values. + val newIndex = + self.index.rdd.zipPartitions(other.index.rdd)( + (thisIter, otherIter) => { + val thisIndex = thisIter.next() + assert(!thisIter.hasNext) + val otherIndex = otherIter.next() + assert(!otherIter.hasNext) + val newIndex = new BlockIndex[K]() + // @todo Merge only the keys that correspond to non-null values + // Merge the keys + newIndex.putAll(thisIndex) + newIndex.putAll(otherIndex) + // We need to rekey the index + var ctr = 0 + for (e <- newIndex.entrySet) { + e.setValue(ctr) + ctr += 1 + } + List(newIndex).iterator + }).cache() + // Use the new index along with the this and the other indices to merge the values + val newValues = + newIndex.zipPartitions(self.tuples, other.tuples)( + (newIndexIter, thisTuplesIter, otherTuplesIter) => { + // Get the new index for this partition + val newIndex = newIndexIter.next() + assert(!newIndexIter.hasNext) + // Get the corresponding indicies and values for this and the other IndexedRDD + val (thisIndex, thisValues) = thisTuplesIter.next() + assert(!thisTuplesIter.hasNext) + val (otherIndex, otherValues) = otherTuplesIter.next() + assert(!otherTuplesIter.hasNext) + // Preallocate the new Values array + val newValues = new Array[Seq[(Seq[V],Seq[W])]](newIndex.size) + // Lookup the sequences in both submaps + for ((k,ind) <- newIndex) { + val thisSeq = if (thisIndex.contains(k)) thisValues(thisIndex.get(k)) else null + val otherSeq = if (otherIndex.contains(k)) otherValues(otherIndex.get(k)) else null + // if either of the sequences is not null then the key was in one of the two tables + // and so the value should appear in the returned table + newValues(ind) = (thisSeq, otherSeq) match { + case (null, null) => null + case (a, null) => Seq( (a, Seq.empty[W]) ) + case (null, b) => Seq( (Seq.empty[V], b) ) + case (a, b) => Seq( (a,b) ) + } + } + List(newValues.toSeq).iterator + }) + new IndexedRDD(new RDDIndex(newIndex), newValues) + } + case _ => { + // Get the partitioner from the index + val partitioner = self.index.rdd.partitioner match { + case Some(p) => p + case None => throw new SparkException("An index must have a partitioner.") + } + // Shuffle the other RDD using the partitioner for this index + val otherShuffled = + if (other.partitioner == Some(partitioner)) { + other + } else { + new ShuffledRDD[K,W](other, partitioner) + } + // Join the other RDD with this RDD building a new valueset and new index on the fly + val groups = + self.tuples.zipPartitions(otherShuffled)( + (thisTuplesIter, otherTuplesIter) => { + // Get the corresponding indicies and values for this IndexedRDD + val (thisIndex, thisValues) = thisTuplesIter.next() + assert(!thisTuplesIter.hasNext()) + // Construct a new index + val newIndex = thisIndex.clone().asInstanceOf[BlockIndex[K]] + // Construct a new array Buffer to store the values + val newValues = ArrayBuffer.fill[(Seq[V], Seq[W])](thisValues.size)(null) + // populate the newValues with the values in this IndexedRDD + for ((k,i) <- thisIndex) { + if (thisValues(i) != null) { + newValues(i) = (thisValues(i), ArrayBuffer.empty[W]) + } + } + // Now iterate through the other tuples updating the map + for ((k,w) <- otherTuplesIter){ + if (!newIndex.contains(k)) { + // update the index + val ind = newIndex.size + newIndex.put(k, ind) + // Update the values + newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) + } else { + val ind = newIndex.get(k) + if(newValues(ind) == null) { + // If the other key was in the index but not in the values + // of this indexed RDD then create a new values entry for it + newValues(ind) = (Seq.empty[V], ArrayBuffer(w)) + } else { + newValues(ind)._2.asInstanceOf[ArrayBuffer[W]].append(w) + } + } + } + // Finalize the new values array + val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = + newValues.view.map{ + case null => null + case (s, ab) => Seq((s, ab.toSeq)) + }.toSeq + List( (newIndex, newValuesArray) ).iterator + }).cache() + + // Extract the index and values from the above RDD + val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) + val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) + + new IndexedRDD[K, (Seq[V], Seq[W])](new RDDIndex(newIndex), newValues) + } + } + } + + +} + +//(self: IndexedRDD[K, V]) extends PairRDDFunctions(self) { } + + diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 03f25040a11..ca42980b468 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -69,7 +69,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * In addition, users can control the partitioning of the output RDD, and whether to perform * map-side aggregation (if a mapper can produce multiple items with the same key). */ - def combineByKey[C](createCombiner: V => C, + def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, partitioner: Partitioner, @@ -103,7 +103,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( /** * Simplified version of combineByKey that hash-partitions the output RDD. */ - def combineByKey[C](createCombiner: V => C, + def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, numPartitions: Int): RDD[(K, C)] = { @@ -248,7 +248,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. */ - def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { + def join[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => for (v <- vs.iterator; w <- ws.iterator) yield (v, w) @@ -261,7 +261,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to * partition the output RDD. */ - def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => if (ws.isEmpty) { @@ -278,7 +278,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair (k, (None, w)) if no elements in `this` have key k. Uses the given Partitioner to * partition the output RDD. */ - def rightOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner) + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner) : RDD[(K, (Option[V], W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => @@ -294,7 +294,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * Simplified version of combineByKey that hash-partitions the resulting RDD using the * existing partitioner/parallelism level. */ - def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) + def combineByKey[C: ClassManifest](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) : RDD[(K, C)] = { combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(self)) } @@ -322,7 +322,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = { + def join[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (V, W))] = { join(other, defaultPartitioner(self, other)) } @@ -331,7 +331,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))] = { + def join[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))] = { join(other, new HashPartitioner(numPartitions)) } @@ -341,7 +341,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output * using the existing partitioner/parallelism level. */ - def leftOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = { + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = { leftOuterJoin(other, defaultPartitioner(self, other)) } @@ -351,7 +351,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output * into `numPartitions` partitions. */ - def leftOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, Option[W]))] = { + def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, Option[W]))] = { leftOuterJoin(other, new HashPartitioner(numPartitions)) } @@ -361,7 +361,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting * RDD using the existing partitioner/parallelism level. */ - def rightOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = { + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = { rightOuterJoin(other, defaultPartitioner(self, other)) } @@ -371,7 +371,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting * RDD into the given number of partitions. */ - def rightOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], W))] = { + def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], W))] = { rightOuterJoin(other, new HashPartitioner(numPartitions)) } @@ -384,7 +384,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * Pass each value in the key-value pair RDD through a map function without changing the keys; * this also retains the original RDD's partitioning. */ - def mapValues[U](f: V => U): RDD[(K, U)] = { + def mapValues[U: ClassManifest](f: V => U): RDD[(K, U)] = { val cleanF = self.context.clean(f) new MappedValuesRDD(self, cleanF) } @@ -393,7 +393,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * Pass each value in the key-value pair RDD through a flatMap function without changing the * keys; this also retains the original RDD's partitioning. */ - def flatMapValues[U](f: V => TraversableOnce[U]): RDD[(K, U)] = { + def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): RDD[(K, U)] = { val cleanF = self.context.clean(f) new FlatMappedValuesRDD(self, cleanF) } @@ -402,7 +402,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W: ClassManifest](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[W]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } @@ -420,7 +420,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ - def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) + def cogroup[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") @@ -441,7 +441,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { cogroup(other, defaultPartitioner(self, other)) } @@ -449,7 +449,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ - def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) + def cogroup[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } @@ -458,7 +458,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Seq[V], Seq[W]))] = { cogroup(other, new HashPartitioner(numPartitions)) } @@ -466,18 +466,18 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ - def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) + def cogroup[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { cogroup(other1, other2, new HashPartitioner(numPartitions)) } /** Alias for cogroup. */ - def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + def groupWith[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { cogroup(other, defaultPartitioner(self, other)) } /** Alias for cogroup. */ - def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) + def groupWith[W1: ClassManifest, W2: ClassManifest](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 16644715246..95adbda43f4 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -774,6 +774,19 @@ abstract class RDD[T: ClassManifest]( } + + // def pairRDDFunctions[K: ClassManifest, V,](implicit t: T <:< (K, V), k: ClassManifest[K], v: ClassManifest[V]): + // PairRDDFunctions[K, V] = { + // new PairRDDFunctions(this.asInstanceOf[RDD[(K,V)]]) + // } + + def pairRDDFunctions[K, V]( + implicit t: T <:< (K, V), k: ClassManifest[K], v: ClassManifest[V]): + PairRDDFunctions[K, V] = { + new PairRDDFunctions(this.asInstanceOf[RDD[(K,V)]]) + } + + def makeIndex(partitioner: Option[Partitioner] = None): RDDIndex[T] = IndexedRDD.makeIndex(this, partitioner) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 80c65dfebd2..73947307860 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -878,7 +878,7 @@ object SparkContext { // TODO: Add AccumulatorParams for other types, e.g. lists and strings implicit def rddToPairRDDFunctions[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]) = - new PairRDDFunctions(rdd) + rdd.pairRDDFunctions implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable: ClassManifest]( rdd: RDD[(K, V)]) = diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index c2995b836a1..0b444c5a7e7 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -263,8 +263,11 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * the merging locally on each mapper before sending results to a reducer, similarly to a * "combiner" in MapReduce. */ - def join[W](other: JavaPairRDD[K, W], partitioner: Partitioner): JavaPairRDD[K, (V, W)] = + def join[W](other: JavaPairRDD[K, W], partitioner: Partitioner): JavaPairRDD[K, (V, W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(rdd.join(other, partitioner)) + } /** * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the @@ -274,6 +277,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif */ def leftOuterJoin[W](other: JavaPairRDD[K, W], partitioner: Partitioner) : JavaPairRDD[K, (V, Optional[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.leftOuterJoin(other, partitioner) fromRDD(joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}) } @@ -286,6 +291,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif */ def rightOuterJoin[W](other: JavaPairRDD[K, W], partitioner: Partitioner) : JavaPairRDD[K, (Optional[V], W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.rightOuterJoin(other, partitioner) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } @@ -324,16 +331,22 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, W)] = + def join[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(rdd.join(other)) + } /** * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and * (k, v2) is in `other`. Performs a hash join across the cluster. */ - def join[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, W)] = + def join[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(rdd.join(other, numPartitions)) + } /** * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the @@ -342,6 +355,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * using the existing partitioner/parallelism level. */ def leftOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, Optional[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.leftOuterJoin(other) fromRDD(joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}) } @@ -353,6 +368,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * into `numPartitions` partitions. */ def leftOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, Optional[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.leftOuterJoin(other, numPartitions) fromRDD(joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}) } @@ -364,6 +381,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * RDD using the existing partitioner/parallelism level. */ def rightOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (Optional[V], W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.rightOuterJoin(other) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } @@ -375,6 +394,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * RDD into the given number of partitions. */ def rightOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (Optional[V], W)] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] val joinResult = rdd.rightOuterJoin(other, numPartitions) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } @@ -411,55 +432,86 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * list of values for that key in `this` as well as `other`. */ def cogroup[W](other: JavaPairRDD[K, W], partitioner: Partitioner) - : JavaPairRDD[K, (JList[V], JList[W])] = + : JavaPairRDD[K, (JList[V], JList[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(cogroupResultToJava(rdd.cogroup(other, partitioner))) + } /** * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], partitioner: Partitioner) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = { + implicit val w1m: ClassManifest[W1] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] + implicit val w2m: ClassManifest[W2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W2]] fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, partitioner))) + } /** * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = + def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(cogroupResultToJava(rdd.cogroup(other))) + } /** * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = { + implicit val w1m: ClassManifest[W1] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] + implicit val w2m: ClassManifest[W2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W2]] fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2))) + } /** * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (JList[V], JList[W])] - = fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) - + def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (JList[V], JList[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] + fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) + } /** * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], numPartitions: Int) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = { + implicit val w1m: ClassManifest[W1] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] + implicit val w2m: ClassManifest[W2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W2]] fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, numPartitions))) + } /** Alias for cogroup. */ - def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = + def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = { + implicit val wm: ClassManifest[W] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]] fromRDD(cogroupResultToJava(rdd.groupWith(other))) + } /** Alias for cogroup. */ def groupWith[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = { + implicit val w1m: ClassManifest[W1] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W1]] + implicit val w2m: ClassManifest[W2] = + implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W2]] fromRDD(cogroupResult2ToJava(rdd.groupWith(other1, other2))) + } /** * Return the list of values in the RDD for key `key`. This operation is done efficiently if the diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index ab1e460aeb6..ac80a06c8a6 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -25,7 +25,6 @@ import java.util.{HashMap => JHashMap, BitSet => JBitSet, HashSet => JHashSet} import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer - import spark._ import spark.rdd._ import spark.SparkContext._ @@ -77,7 +76,7 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( /** * An internal representation which joins the block indices with the values */ - protected val tuples = new ZippedRDD(index.rdd.context, index.rdd, valuesRDD) + protected[spark] val tuples = new ZippedRDD(index.rdd.context, index.rdd, valuesRDD) /** @@ -108,341 +107,14 @@ class IndexedRDD[K: ClassManifest, V: ClassManifest]( } - /** - * Construct a new IndexedRDD that is indexed by only the keys in the RDD - */ - def reindex(): IndexedRDD[K,V] = IndexedRDD(this) - - - /** - * Pass each value in the key-value pair RDD through a map function without changing the keys; - * this also retains the original RDD's partitioning. - */ - def mapValues[U: ClassManifest](f: V => U): IndexedRDD[K, U] = { - val cleanF = index.rdd.context.clean(f) - val newValues = valuesRDD.mapPartitions(_.map(values => values.map{ - case null => null - case row => row.map(x => f(x)) - }), true) - new IndexedRDD[K,U](index, newValues) - } - - - /** - * Pass each value in the key-value pair RDD through a flatMap function without changing the - * keys; this also retains the original RDD's partitioning. - */ - def flatMapValues[U: ClassManifest](f: V => TraversableOnce[U]): IndexedRDD[K,U] = { - val cleanF = index.rdd.context.clean(f) - val newValues = valuesRDD.mapPartitions(_.map(values => values.map{ - case null => null - case row => row.flatMap(x => f(x)) - }), true) - new IndexedRDD[K,U](index, newValues) - } - - - /** - * Generic function to combine the elements for each key using a custom set of aggregation - * functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C - * Note that V and C can be different -- for example, one might group an RDD of type - * (Int, Int) into an RDD of type (Int, Seq[Int]). Users provide three functions: - * - * - `createCombiner`, which turns a V into a C (e.g., creates a one-element list) - * - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list) - * - `mergeCombiners`, to combine two C's into a single one. - */ - def combineByKey[C: ClassManifest](createCombiner: V => C, - mergeValue: (C, V) => C, - mergeCombiners: (C, C) => C, - serializerClass: String = null): IndexedRDD[K, C] = { - val newValues = valuesRDD.mapPartitions( - _.map{ groups: Seq[Seq[V]] => - groups.map{ group: Seq[V] => - if (group != null && !group.isEmpty) { - val c: C = createCombiner(group.head) - val sum: C = group.tail.foldLeft(c)(mergeValue) - Seq(sum) - } else { - null - } - } - }, true) - new IndexedRDD[K,C](index, newValues) - } - - /** - * Merge the values for each key using an associative function and a neutral "zero value" which may - * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for - * list concatenation, 0 for addition, or 1 for multiplication.). - */ - def foldByKey(zeroValue: V)(func: (V, V) => V): IndexedRDD[K, V] = { - // 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) - } - - /** - * Merge the values for each key using an associative reduce function. This will also perform - * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/ - * parallelism level. - */ - def reduceByKey(func: (V, V) => V): IndexedRDD[K, V] = { - combineByKey[V]((v: V) => v, func, func) - } - - - /** - * Group the values for each key in the RDD into a single sequence. Hash-partitions the - * resulting RDD with the existing partitioner/parallelism level. - */ - def groupByKey(): IndexedRDD[K, Seq[V]] = { - val newValues = valuesRDD.mapPartitions(_.map{ar => ar.map{s => Seq(s)} }, true) - new IndexedRDD[K, Seq[V]](index, newValues) - } - - - /** - * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the - * list of values for that key in `this` as well as `other`. - */ - def cogroup[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { - //RDD[(K, (Seq[V], Seq[W]))] = { - other match { - case other: IndexedRDD[_, _] if other.index == index => { - // if both RDDs share exactly the same index and therefore the same super set of keys - // then we simply merge the value RDDs. - // However it is possible that both RDDs are missing a value for a given key in - // which case the returned RDD should have a null value - val newValues = - valuesRDD.zipPartitions(other.valuesRDD)( - (thisIter, otherIter) => { - val thisValues: Seq[Seq[V]] = thisIter.next() - assert(!thisIter.hasNext()) - val otherValues: Seq[Seq[W]] = otherIter.next() - assert(!otherIter.hasNext()) - // Zip the values and if both arrays are null then the key is not present and - // so the resulting value must be null (not a tuple of empty sequences) - val tmp: Seq[Seq[(Seq[V], Seq[W])]] = thisValues.view.zip(otherValues).map{ - case (null, null) => null // The key is not present in either RDD - case (a, null) => Seq((a, Seq.empty[W])) - case (null, b) => Seq((Seq.empty[V], b)) - case (a, b) => Seq((a,b)) - }.toSeq - List(tmp).iterator - }) - new IndexedRDD[K, (Seq[V], Seq[W])](index, newValues) - } - case other: IndexedRDD[_, _] if other.index.rdd.partitioner == index.rdd.partitioner => { - // If both RDDs are indexed using different indices but with the same partitioners - // then we we need to first merge the indicies and then use the merged index to - // merge the values. - val newIndex = - index.rdd.zipPartitions(other.index.rdd)( - (thisIter, otherIter) => { - val thisIndex = thisIter.next() - assert(!thisIter.hasNext()) - val otherIndex = otherIter.next() - assert(!otherIter.hasNext()) - val newIndex = new BlockIndex[K]() - // @todo Merge only the keys that correspond to non-null values - // Merge the keys - newIndex.putAll(thisIndex) - newIndex.putAll(otherIndex) - // We need to rekey the index - var ctr = 0 - for (e <- newIndex.entrySet) { - e.setValue(ctr) - ctr += 1 - } - List(newIndex).iterator - }).cache() - // Use the new index along with the this and the other indices to merge the values - val newValues = - newIndex.zipPartitions(tuples, other.tuples)( - (newIndexIter, thisTuplesIter, otherTuplesIter) => { - // Get the new index for this partition - val newIndex = newIndexIter.next() - assert(!newIndexIter.hasNext()) - // Get the corresponding indicies and values for this and the other IndexedRDD - val (thisIndex, thisValues) = thisTuplesIter.next() - assert(!thisTuplesIter.hasNext()) - val (otherIndex, otherValues) = otherTuplesIter.next() - assert(!otherTuplesIter.hasNext()) - // Preallocate the new Values array - val newValues = new Array[Seq[(Seq[V],Seq[W])]](newIndex.size) - // Lookup the sequences in both submaps - for ((k,ind) <- newIndex) { - val thisSeq = if (thisIndex.contains(k)) thisValues(thisIndex.get(k)) else null - val otherSeq = if (otherIndex.contains(k)) otherValues(otherIndex.get(k)) else null - // if either of the sequences is not null then the key was in one of the two tables - // and so the value should appear in the returned table - newValues(ind) = (thisSeq, otherSeq) match { - case (null, null) => null - case (a, null) => Seq( (a, Seq.empty[W]) ) - case (null, b) => Seq( (Seq.empty[V], b) ) - case (a, b) => Seq( (a,b) ) - } - } - List(newValues.toSeq).iterator - }) - new IndexedRDD(new RDDIndex(newIndex), newValues) - } - case _ => { - // Get the partitioner from the index - val partitioner = index.rdd.partitioner match { - case Some(p) => p - case None => throw new SparkException("An index must have a partitioner.") - } - // Shuffle the other RDD using the partitioner for this index - val otherShuffled = - if (other.partitioner == Some(partitioner)) { - other - } else { - new ShuffledRDD[K,W](other, partitioner) - } - // Join the other RDD with this RDD building a new valueset and new index on the fly - val groups = - tuples.zipPartitions(otherShuffled)( - (thisTuplesIter, otherTuplesIter) => { - // Get the corresponding indicies and values for this IndexedRDD - val (thisIndex, thisValues) = thisTuplesIter.next() - assert(!thisTuplesIter.hasNext()) - // Construct a new index - val newIndex = thisIndex.clone().asInstanceOf[BlockIndex[K]] - // Construct a new array Buffer to store the values - val newValues = ArrayBuffer.fill[(Seq[V], Seq[W])](thisValues.size)(null) - // populate the newValues with the values in this IndexedRDD - for ((k,i) <- thisIndex) { - if (thisValues(i) != null) { - newValues(i) = (thisValues(i), ArrayBuffer.empty[W]) - } - } - // Now iterate through the other tuples updating the map - for ((k,w) <- otherTuplesIter){ - if (!newIndex.contains(k)) { - // update the index - val ind = newIndex.size - newIndex.put(k, ind) - // Update the values - newValues.append( (Seq.empty[V], ArrayBuffer(w) ) ) - } else { - val ind = newIndex.get(k) - if(newValues(ind) == null) { - // If the other key was in the index but not in the values - // of this indexed RDD then create a new values entry for it - newValues(ind) = (Seq.empty[V], ArrayBuffer(w)) - } else { - newValues(ind)._2.asInstanceOf[ArrayBuffer[W]].append(w) - } - } - } - // Finalize the new values array - val newValuesArray: Seq[Seq[(Seq[V],Seq[W])]] = - newValues.view.map{ - case null => null - case (s, ab) => Seq((s, ab.toSeq)) - }.toSeq - List( (newIndex, newValuesArray) ).iterator - }).cache() - - // Extract the index and values from the above RDD - val newIndex = groups.mapPartitions(_.map{ case (kMap,vAr) => kMap }, true) - val newValues = groups.mapPartitions(_.map{ case (kMap,vAr) => vAr }, true) - - new IndexedRDD[K, (Seq[V], Seq[W])](new RDDIndex(newIndex), newValues) - } - } - } - - - - // /** - // * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a - // * tuple with the list of values for that key in `this`, `other1` and `other2`. - // */ - // def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - // : IndexedRDD[K, (Seq[V], Seq[W1], Seq[W2])] = { - // cogroup(other1, other2, defaultPartitioner(this, other1, other2)) - // } - - // /** - // * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a - // * tuple with the list of values for that key in `this`, `other1` and `other2`. - // */ - // def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) - // : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { - // cogroup(other1, other2, new HashPartitioner(numPartitions)) - // } - - /** Alias for cogroup. */ - def groupWith[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Seq[V], Seq[W])] = { - cogroup(other) + override def pairRDDFunctions[K1, V1]( + implicit t: (K, V) <:< (K1,V1), k: ClassManifest[K1], v: ClassManifest[V1]): + PairRDDFunctions[K1, V1] = { + new IndexedRDDFunctions[K1,V1](this.asInstanceOf[IndexedRDD[K1,V1]]) } - // /** Alias for cogroup. */ - // def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - // : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { - // cogroup(other1, other2, defaultPartitioner(self, other1, other2)) - // } - - /** - * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each - * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and - * (k, v2) is in `other`. Performs a hash join across the cluster. - */ - def join[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, W)] = { - cogroup(other).flatMapValues { - case (vs, ws) => - for (v <- vs.iterator; w <- ws.iterator) yield (v, w) - } - } - - - /** - * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the - * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the - * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output - * using the existing partitioner/parallelism level. - */ - def leftOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (V, Option[W])] = { - cogroup(other).flatMapValues { - case (vs, ws) => - if (ws.isEmpty) { - vs.iterator.map(v => (v, None)) - } else { - for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w)) - } - } - - } - - - /** - * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the - * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the - * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting - * RDD using the existing partitioner/parallelism level. - */ - def rightOuterJoin[W: ClassManifest](other: RDD[(K, W)]): IndexedRDD[K, (Option[V], W)] = { - cogroup(other).flatMapValues { - case (vs, ws) => - if (vs.isEmpty) { - ws.iterator.map(w => (None, w)) - } else { - for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w) - } - } - - } + /** From 766b6fd380341da06cb1c97de71c74e0eb6c36d0 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Tue, 27 Aug 2013 18:54:26 -0700 Subject: [PATCH 0857/2521] Fixing IndexedRDD unit tests. --- core/src/main/scala/spark/rdd/IndexedRDD.scala | 4 ++-- core/src/test/scala/spark/IndexedRDDSuite.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/rdd/IndexedRDD.scala b/core/src/main/scala/spark/rdd/IndexedRDD.scala index 5ddfe5db2c0..8536bb4a4bb 100644 --- a/core/src/main/scala/spark/rdd/IndexedRDD.scala +++ b/core/src/main/scala/spark/rdd/IndexedRDD.scala @@ -68,8 +68,8 @@ class RDDIndex[@specialized K: ClassManifest](private[spark] val rdd: RDD[BlockI * */ class IndexedRDD[K: ClassManifest, V: ClassManifest]( - val index: RDDIndex[K], - val valuesRDD: RDD[ Seq[Seq[V]] ]) + @transient val index: RDDIndex[K], + @transient val valuesRDD: RDD[ Seq[Seq[V]] ]) extends RDD[(K, V)](index.rdd.context, List(new OneToOneDependency(index.rdd), new OneToOneDependency(valuesRDD)) ) { diff --git a/core/src/test/scala/spark/IndexedRDDSuite.scala b/core/src/test/scala/spark/IndexedRDDSuite.scala index c1433c50950..1a17f7c0d36 100644 --- a/core/src/test/scala/spark/IndexedRDDSuite.scala +++ b/core/src/test/scala/spark/IndexedRDDSuite.scala @@ -123,7 +123,7 @@ class IndexedRDDSuite extends FunSuite with SharedSparkContext { // count the dependencies to make sure there is only 1 ShuffledRDD val deps = lineage(sums) - assert(deps.filter(_.isInstanceOf[ShuffledRDD[_,_]]).size === 1) // ShuffledRDD, ParallelCollection + assert(deps.filter(_.isInstanceOf[ShuffledRDD[_,_,_]]).size === 1) // ShuffledRDD, ParallelCollection } From aac1214ee48ef143b0164f740380cdb0a5a7383b Mon Sep 17 00:00:00 2001 From: "Y.CORP.YAHOO.COM\\tgraves" Date: Wed, 28 Aug 2013 13:26:26 -0500 Subject: [PATCH 0858/2521] Change Executor to only look at the env variable SPARK_YARN_MODE --- 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 25f55b391b9..fa82d2b3245 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -63,7 +63,7 @@ private[spark] class Executor( // If we are in yarn mode, systems can have different disk layouts so we must set it // to what Yarn on this system said was available. This will be used later when SparkEnv // created. - if (java.lang.Boolean.valueOf(System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))) { + if (java.lang.Boolean.valueOf(System.getenv("SPARK_YARN_MODE"))) { System.setProperty("spark.local.dir", getYarnLocalDirs()) } From 33250835527c1894d9cecdc35b25017bddc68670 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 28 Aug 2013 14:57:12 -0700 Subject: [PATCH 0859/2521] Removing commented out test code --- core/src/main/scala/spark/RDD.scala | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 6a2d3eb78b3..b308d2f62df 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -777,11 +777,9 @@ abstract class RDD[T: ClassManifest]( - // def pairRDDFunctions[K: ClassManifest, V,](implicit t: T <:< (K, V), k: ClassManifest[K], v: ClassManifest[V]): - // PairRDDFunctions[K, V] = { - // new PairRDDFunctions(this.asInstanceOf[RDD[(K,V)]]) - // } - + /** + * For RDD[(K,V)] this function returns a pair-functions object for this RDD + */ def pairRDDFunctions[K, V]( implicit t: T <:< (K, V), k: ClassManifest[K], v: ClassManifest[V]): PairRDDFunctions[K, V] = { From 457bcd33436c91a6ef07591837ea048bb4bbcede Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Wed, 28 Aug 2013 16:10:38 -0700 Subject: [PATCH 0860/2521] PySpark: implementing subtractByKey(), subtract() and keyBy() --- python/pyspark/rdd.py | 37 +++++++++++++++++++++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 1e9b3bb5c03..dfc518a7b01 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -754,6 +754,43 @@ def cogroup(self, other, numPartitions=None): """ return python_cogroup(self, other, numPartitions) + def subtractByKey(self, other, numPartitions=None): + """ + Return each (key, value) pair in C{self} that has no pair with matching key + in C{other}. + + >>> x = sc.parallelize([("a", 1), ("b", 4), ("b", 5), ("a", 2)]) + >>> y = sc.parallelize([("a", 3), ("c", None)]) + >>> sorted(x.subtractByKey(y).collect()) + [('b', 4), ('b', 5)] + """ + filter_func = lambda tpl: len(tpl[1][0]) > 0 and len(tpl[1][1]) == 0 + map_func = lambda tpl: [(tpl[0], val) for val in tpl[1][0]] + return self.cogroup(other, numPartitions).filter(filter_func).flatMap(map_func) + + def subtract(self, other, numPartitions=None): + """ + Return each value in C{self} that is not contained in C{other}. + + >>> x = sc.parallelize([("a", 1), ("b", 4), ("b", 5), ("a", 3)]) + >>> y = sc.parallelize([("a", 3), ("c", None)]) + >>> sorted(x.subtract(y).collect()) + [('a', 1), ('b', 4), ('b', 5)] + """ + rdd = other.map(lambda x: (x, True)) # note: here 'True' is just a placeholder + return self.map(lambda x: (x, True)).subtractByKey(rdd).map(lambda tpl: tpl[0]) # note: here 'True' is just a placeholder + + def keyBy(self, f): + """ + Creates tuples of the elements in this RDD by applying C{f}. + + >>> x = sc.parallelize(range(0,3)).keyBy(lambda x: x*x) + >>> y = sc.parallelize(zip(range(0,5), range(0,5))) + >>> sorted(x.cogroup(y).collect()) + [(0, ([0], [0])), (1, ([1], [1])), (2, ([], [2])), (3, ([], [3])), (4, ([2], [4]))] + """ + return self.map(lambda x: (f(x), x)) + # TODO: `lookup` is disabled because we can't make direct comparisons based # on the key; we need to compare the hash of the key to the hash of the # keys in the pairs. This could be an expensive operation, since those From 742c44eae693d2bde76259043c962e416691258c Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 28 Aug 2013 16:39:44 -0700 Subject: [PATCH 0861/2521] Don't send SIGINT to Py4J gateway subprocess. This addresses SPARK-885, a usability issue where PySpark's Java gateway process would be killed if the user hit ctrl-c. Note that SIGINT still won't cancel the running s This fix is based on http://stackoverflow.com/questions/5045771 --- python/pyspark/java_gateway.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index e503fb7621e..fdc98514799 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -17,6 +17,7 @@ import os import sys +import signal from subprocess import Popen, PIPE from threading import Thread from py4j.java_gateway import java_import, JavaGateway, GatewayClient @@ -30,7 +31,10 @@ def launch_gateway(): # proper classpath and SPARK_MEM settings from spark-env.sh command = [os.path.join(SPARK_HOME, "run"), "py4j.GatewayServer", "--die-on-broken-pipe", "0"] - proc = Popen(command, stdout=PIPE, stdin=PIPE) + # Don't send ctrl-c / SIGINT to the Java gateway: + def preexec_function(): + signal.signal(signal.SIGINT, signal.SIG_IGN) + proc = Popen(command, stdout=PIPE, stdin=PIPE, preexec_fn=preexec_function) # Determine which ephemeral port the server started on: port = int(proc.stdout.readline()) # Create a thread to echo output from the GatewayServer, which is required From a511c5379ee156f08624e380b240af7d961a60f7 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Fri, 23 Aug 2013 11:16:44 -0700 Subject: [PATCH 0862/2521] RDD sample() and takeSample() prototypes for PySpark --- python/pyspark/rdd.py | 62 ++++++++++++++++--- python/pyspark/rddsampler.py | 112 +++++++++++++++++++++++++++++++++++ 2 files changed, 167 insertions(+), 7 deletions(-) create mode 100644 python/pyspark/rddsampler.py diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 1e9b3bb5c03..8394fe6a319 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -21,6 +21,7 @@ from itertools import chain, ifilter, imap, product import operator import os +import sys import shlex from subprocess import Popen, PIPE from tempfile import NamedTemporaryFile @@ -32,6 +33,7 @@ from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_cogroup from pyspark.statcounter import StatCounter +from pyspark.rddsampler import RDDSampler from py4j.java_collections import ListConverter, MapConverter @@ -165,14 +167,60 @@ def distinct(self): .reduceByKey(lambda x, _: x) \ .map(lambda (x, _): x) - # TODO: sampling needs to be re-implemented due to Batch - #def sample(self, withReplacement, fraction, seed): - # jrdd = self._jrdd.sample(withReplacement, fraction, seed) - # return RDD(jrdd, self.ctx) + def sample(self, withReplacement, fraction, seed): + """ + Return a sampled subset of this RDD (relies on numpy and falls back + on default random generator if numpy is unavailable). + + >>> sc.parallelize(range(0, 100)).sample(False, 0.1, 2).collect() #doctest: +SKIP + [2, 3, 20, 21, 24, 41, 42, 66, 67, 89, 90, 98] + """ + return self.mapPartitionsWithSplit(RDDSampler(withReplacement, fraction, seed).func, True) + + # this is ported from scala/spark/RDD.scala + def takeSample(self, withReplacement, num, seed): + """ + Return a fixed-size sampled subset of this RDD (currently requires numpy). + + >>> sc.parallelize(range(0, 10)).takeSample(True, 10, 1) #doctest: +SKIP + [4, 2, 1, 8, 2, 7, 0, 4, 1, 4] + """ - #def takeSample(self, withReplacement, num, seed): - # vals = self._jrdd.takeSample(withReplacement, num, seed) - # return [load_pickle(bytes(x)) for x in vals] + fraction = 0.0 + total = 0 + multiplier = 3.0 + initialCount = self.count() + maxSelected = 0 + + if (num < 0): + raise ValueError + + if initialCount > sys.maxint - 1: + maxSelected = sys.maxint - 1 + else: + maxSelected = initialCount + + if num > initialCount and not withReplacement: + total = maxSelected + fraction = multiplier * (maxSelected + 1) / initialCount + else: + fraction = multiplier * (num + 1) / initialCount + total = num + + samples = self.sample(withReplacement, fraction, seed).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 their initial size. + # See: scala/spark/RDD.scala + while len(samples) < total: + if seed > sys.maxint - 2: + seed = -1 + seed += 1 + samples = self.sample(withReplacement, fraction, seed).collect() + + sampler = RDDSampler(withReplacement, fraction, seed+1) + sampler.shuffle(samples) + return samples[0:total] def union(self, other): """ diff --git a/python/pyspark/rddsampler.py b/python/pyspark/rddsampler.py new file mode 100644 index 00000000000..aca2ef3b51e --- /dev/null +++ b/python/pyspark/rddsampler.py @@ -0,0 +1,112 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 +import random + +class RDDSampler(object): + def __init__(self, withReplacement, fraction, seed): + try: + import numpy + self._use_numpy = True + except ImportError: + print >> sys.stderr, "NumPy does not appear to be installed. Falling back to default random generator for sampling." + self._use_numpy = False + + self._seed = seed + self._withReplacement = withReplacement + self._fraction = fraction + self._random = None + self._split = None + self._rand_initialized = False + + def initRandomGenerator(self, split): + if self._use_numpy: + import numpy + self._random = numpy.random.RandomState(self._seed) + for _ in range(0, split): + # discard the next few values in the sequence to have a + # different seed for the different splits + self._random.randint(sys.maxint) + else: + import random + random.seed(self._seed) + for _ in range(0, split): + # discard the next few values in the sequence to have a + # different seed for the different splits + random.randint(0, sys.maxint) + self._split = split + self._rand_initialized = True + + def getUniformSample(self, split): + if not self._rand_initialized or split != self._split: + self.initRandomGenerator(split) + + if self._use_numpy: + return self._random.random_sample() + else: + return random.uniform(0.0, 1.0) + + def getPoissonSample(self, split, mean): + if not self._rand_initialized or split != self._split: + self.initRandomGenerator(split) + + if self._use_numpy: + return self._random.poisson(mean) + else: + # here we simulate drawing numbers n_i ~ Poisson(lambda = 1/mean) by + # drawing a sequence of numbers delta_j ~ Exp(mean) + num_arrivals = 1 + cur_time = 0.0 + + cur_time += random.expovariate(mean) + + if cur_time > 1.0: + return 0 + + while(cur_time <= 1.0): + cur_time += random.expovariate(mean) + num_arrivals += 1 + + return (num_arrivals - 1) + + def shuffle(self, vals): + if self._random == None or split != self._split: + self.initRandomGenerator(0) # this should only ever called on the master so + # the split does not matter + + if self._use_numpy: + self._random.shuffle(vals) + else: + random.shuffle(vals, self._random) + + def func(self, split, iterator): + if self._withReplacement: + for obj in iterator: + # For large datasets, the expected number of occurrences of each element in a sample with + # replacement is Poisson(frac). We use that to get a count for each element. + count = self.getPoissonSample(split, mean = self._fraction) + for _ in range(0, count): + yield obj + else: + for obj in iterator: + if self.getUniformSample(split) <= self._fraction: + yield obj + + + + From 2fc9a028f2b3ebd0ab993672ae88d96b8868316c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 28 Aug 2013 19:03:06 -0700 Subject: [PATCH 0863/2521] Hot fix for command runner --- run | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/run b/run index 3868332c90c..6fba005c0f9 100755 --- a/run +++ b/run @@ -172,4 +172,4 @@ if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then echo fi -exec $command +exec "$RUNNER" -cp "$CLASSPATH" "$@" From 30d24211121bdfc0894795a157413c80406c57b2 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 28 Aug 2013 19:53:31 -0700 Subject: [PATCH 0864/2521] Make local variable public --- 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 23dfbcd6047..7ce9505b9c2 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -102,7 +102,7 @@ class SparkContext( System.setProperty("spark.driver.port", "0") } - private val isLocal = (master == "local" || master.startsWith("local[")) + val isLocal = (master == "local" || master.startsWith("local[")) // Create the Spark execution environment (cache, map output tracker, etc) private[spark] val env = SparkEnv.createFromSystemProperties( From 1798e69e71d0005e770f0d66b86b80be9c0018a8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 28 Aug 2013 19:56:46 -0700 Subject: [PATCH 0865/2521] Adding extra args --- run | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/run b/run index 6fba005c0f9..715bbf93d55 100755 --- a/run +++ b/run @@ -172,4 +172,4 @@ if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then echo fi -exec "$RUNNER" -cp "$CLASSPATH" "$@" +exec "$RUNNER" -cp "$CLASSPATH" $EXTRA_ARGS "$@" From f3dbe6b2158d6fdf1eef3841e9522b4236a3fa81 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 30 Aug 2013 09:39:01 +0800 Subject: [PATCH 0866/2521] Fix removed block zero size log reporting --- .../main/scala/spark/storage/BlockManagerMasterActor.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index 2a2e1785504..b7a981d1014 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -372,12 +372,12 @@ object BlockManagerMasterActor { if (blockStatus.storageLevel.useMemory) { _remainingMem += blockStatus.memSize logInfo("Removed %s on %s in memory (size: %s, free: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(memSize), + blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.memSize), Utils.bytesToString(_remainingMem))) } if (blockStatus.storageLevel.useDisk) { logInfo("Removed %s on %s on disk (size: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(diskSize))) + blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.diskSize))) } } } From 53cd50c0699efc8733518658100c62426b425de2 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 23 Aug 2013 23:30:17 -0700 Subject: [PATCH 0867/2521] Change build and run instructions to use assemblies This commit makes Spark invocation saner by using an assembly JAR to find all of Spark's dependencies instead of adding all the JARs in lib_managed. It also packages the examples into an assembly and uses that as SPARK_EXAMPLES_JAR. Finally, it replaces the old "run" script with two better-named scripts: "run-examples" for examples, and "spark-class" for Spark internal classes (e.g. REPL, master, etc). This is also designed to minimize the confusion people have in trying to use "run" to run their own classes; it's not meant to do that, but now at least if they look at it, they can modify run-examples to do a decent job for them. As part of this, Bagel's examples are also now properly moved to the examples package instead of bagel. --- README.md | 6 +- bin/compute-classpath.sh | 82 +++------------- bin/spark-daemon.sh | 8 +- .../mesos/CoarseMesosSchedulerBackend.scala | 2 +- core/src/test/scala/spark/DriverSuite.scala | 2 +- docs/bagel-programming-guide.md | 2 +- docs/index.md | 4 +- docs/java-programming-guide.md | 2 +- docs/running-on-yarn.md | 8 +- docs/scala-programming-guide.md | 2 +- docs/spark-standalone.md | 4 +- docs/streaming-programming-guide.md | 4 +- examples/pom.xml | 5 + .../spark/examples/bagel}/PageRankUtils.scala | 2 +- .../examples/bagel}/WikipediaPageRank.scala | 2 +- .../bagel}/WikipediaPageRankStandalone.scala | 2 +- make-distribution.sh | 17 ++-- project/SparkBuild.scala | 52 +++++++---- project/build.properties | 2 +- project/plugins.sbt | 2 +- python/pyspark/java_gateway.py | 2 +- run-example | 76 +++++++++++++++ run => spark-class | 93 +++++-------------- spark-executor | 2 +- spark-shell | 3 +- 25 files changed, 187 insertions(+), 199 deletions(-) rename {bagel/src/main/scala/spark/bagel/examples => examples/src/main/scala/spark/examples/bagel}/PageRankUtils.scala (99%) rename {bagel/src/main/scala/spark/bagel/examples => examples/src/main/scala/spark/examples/bagel}/WikipediaPageRank.scala (99%) rename {bagel/src/main/scala/spark/bagel/examples => examples/src/main/scala/spark/examples/bagel}/WikipediaPageRankStandalone.scala (99%) create mode 100755 run-example rename run => spark-class (52%) diff --git a/README.md b/README.md index e5f527b84a9..e7af34f5131 100644 --- a/README.md +++ b/README.md @@ -20,16 +20,16 @@ Spark and its example programs, run: Spark also supports building using Maven. If you would like to build using Maven, see the [instructions for building Spark with Maven](http://spark-project.org/docs/latest/building-with-maven.html) -in the spark documentation.. +in the Spark documentation.. To run Spark, you will need to have Scala's bin directory in your `PATH`, or you will need to set the `SCALA_HOME` environment variable to point to where you've installed Scala. Scala must be accessible through one of these methods on your cluster's worker nodes as well as its master. -To run one of the examples, use `./run `. For example: +To run one of the examples, use `./run-example `. For example: - ./run spark.examples.SparkLR local[2] + ./run-example spark.examples.SparkLR local[2] will run the Logistic Regression example locally on 2 CPUs. diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 7a21b3c4a14..5dc86c51a4a 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -30,79 +30,25 @@ 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" -MLLIB_DIR="$FWDIR/mllib" -TOOLS_DIR="$FWDIR/tools" -YARN_DIR="$FWDIR/yarn" -STREAMING_DIR="$FWDIR/streaming" -PYSPARK_DIR="$FWDIR/python" - # Build up classpath -CLASSPATH="$SPARK_CLASSPATH" - -function dev_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.jar'`; do - CLASSPATH="$CLASSPATH:$jar" - done - # The shaded JAR doesn't contain examples, so include those separately - for jar in `find "$EXAMPLES_DIR/target" -name 'spark-examples*[0-9T].jar'`; do - CLASSPATH="$CLASSPATH:$jar" - done - 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" - CLASSPATH="$CLASSPATH:$YARN_DIR/target/scala-$SCALA_VERSION/classes" - for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do - CLASSPATH="$CLASSPATH:$jar" - done - - # 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" -} - -function release_classpath { - CLASSPATH="$CLASSPATH:$FWDIR/jars/*" -} - +CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" if [ -f "$FWDIR/RELEASE" ]; then - release_classpath + ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark-assembly*.jar` else - dev_classpath + ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*.jar` +fi +CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" + +# Add test classes if we're running from SBT or Maven with SPARK_TESTING set to 1 +if [[ $SPARK_TESTING == 1 ]]; then + CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/test-classes" fi -# Add hadoop conf dir - else FileSystem.*, etc fail ! +# Add hadoop conf dir if given -- otherwise 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 diff --git a/bin/spark-daemon.sh b/bin/spark-daemon.sh index 96c71e66cae..eac0774669e 100755 --- a/bin/spark-daemon.sh +++ b/bin/spark-daemon.sh @@ -87,7 +87,7 @@ TEST_LOG_DIR=$? if [ "${TEST_LOG_DIR}" = "0" ]; then rm -f $SPARK_LOG_DIR/.spark_test else - chown $SPARK_IDENT_STRING $SPARK_LOG_DIR + chown $SPARK_IDENT_STRING $SPARK_LOG_DIR fi if [ "$SPARK_PID_DIR" = "" ]; then @@ -109,7 +109,7 @@ fi case $startStop in (start) - + mkdir -p "$SPARK_PID_DIR" if [ -f $pid ]; then @@ -128,11 +128,11 @@ case $startStop in echo starting $command, logging to $log echo "Spark Daemon: $command" > $log cd "$SPARK_PREFIX" - nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/run $command "$@" >> "$log" 2>&1 < /dev/null & + nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & echo $! > $pid sleep 1; head "$log" ;; - + (stop) if [ -f $pid ]; then diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index 6ebbb5ec9b2..ebfc21392dd 100644 --- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -125,7 +125,7 @@ private[spark] class CoarseMesosSchedulerBackend( StandaloneSchedulerBackend.ACTOR_NAME) val uri = System.getProperty("spark.executor.uri") if (uri == null) { - val runScript = new File(sparkHome, "run").getCanonicalPath + val runScript = new File(sparkHome, "spark-class").getCanonicalPath command.setValue("\"%s\" spark.executor.StandaloneExecutorBackend %s %s %s %d".format( runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) } else { diff --git a/core/src/test/scala/spark/DriverSuite.scala b/core/src/test/scala/spark/DriverSuite.scala index ed16b9d8efd..553c0309f66 100644 --- a/core/src/test/scala/spark/DriverSuite.scala +++ b/core/src/test/scala/spark/DriverSuite.scala @@ -34,7 +34,7 @@ class DriverSuite extends FunSuite with Timeouts { val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]")) forAll(masters) { (master: String) => failAfter(30 seconds) { - Utils.execute(Seq("./run", "spark.DriverWithoutCleanup", master), + Utils.execute(Seq("./spark-class", "spark.DriverWithoutCleanup", master), new File(System.getenv("SPARK_HOME"))) } } diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md index 8a0fa42d946..c526da3ca0b 100644 --- a/docs/bagel-programming-guide.md +++ b/docs/bagel-programming-guide.md @@ -158,4 +158,4 @@ trait Message[K] { ## Where to Go from Here -Two example jobs, PageRank and shortest path, are included in `bagel/src/main/scala/spark/bagel/examples`. You can run them by passing the class name to the `run` script included in Spark -- for example, `./run spark.bagel.examples.WikipediaPageRank`. Each example program prints usage help when run without any arguments. +Two example jobs, PageRank and shortest path, are included in `examples/src/main/scala/spark/examples/bagel`. You can run them by passing the class name to the `run-example` script included in Spark -- for example, `./run-example spark.examples.bagel.WikipediaPageRank`. Each example program prints usage help when run without any arguments. diff --git a/docs/index.md b/docs/index.md index 0c4add45dcd..e51a6998f6c 100644 --- a/docs/index.md +++ b/docs/index.md @@ -27,9 +27,9 @@ Spark also supports building using Maven. If you would like to build using Maven # Testing the Build Spark comes with a number of sample programs in the `examples` directory. -To run one of the samples, use `./run ` in the top-level Spark directory +To run one of the samples, use `./run-example ` in the top-level Spark directory (the `run` script sets up the appropriate paths and launches that program). -For example, `./run spark.examples.SparkPi` will run a sample program that estimates Pi. Each of the +For example, `./run-example spark.examples.SparkPi` will run a sample program that estimates Pi. Each of the examples prints usage help if no params are given. Note that all of the sample programs take a `` parameter specifying the cluster URL diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md index ae8257b5393..dd19a5f0c97 100644 --- a/docs/java-programming-guide.md +++ b/docs/java-programming-guide.md @@ -190,6 +190,6 @@ We hope to generate documentation with Java-style syntax in the future. Spark includes several sample programs using the Java API in [`examples/src/main/java`](https://github.com/mesos/spark/tree/master/examples/src/main/java/spark/examples). You can run them by passing the class name to the -`run` script included in Spark -- for example, `./run +`run-example` script included in Spark -- for example, `./run-example spark.examples.JavaWordCount`. Each example program prints usage help when run without any arguments. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 1a0afd19d43..678cd57abac 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -15,9 +15,9 @@ We need a consolidated spark core jar (which bundles all the required dependenci This can be built either through sbt or via maven. - Building spark assembled jar via sbt. -Enable YARN support by setting `SPARK_WITH_YARN=true` when invoking sbt: +Enable YARN support by setting `SPARK_YARN=true` when invoking sbt: - SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_WITH_YARN=true ./sbt/sbt clean assembly + SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true ./sbt/sbt clean assembly The assembled jar would typically be something like : `./yarn/target/spark-yarn-assembly-0.8.0-SNAPSHOT.jar` @@ -55,7 +55,7 @@ This would be used to connect to the cluster, write to the dfs and submit jobs t The command to launch the YARN Client is as follows: - SPARK_JAR= ./run spark.deploy.yarn.Client \ + SPARK_JAR= ./spark-class spark.deploy.yarn.Client \ --jar \ --class \ --args \ @@ -67,7 +67,7 @@ The command to launch the YARN Client is as follows: For example: - SPARK_JAR=./yarn/target/spark-yarn-assembly-{{site.SPARK_VERSION}}.jar ./run spark.deploy.yarn.Client \ + SPARK_JAR=./yarn/target/spark-yarn-assembly-{{site.SPARK_VERSION}}.jar ./spark-class 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 yarn-standalone \ diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index e9cf9ef36fc..db584d2096e 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -356,7 +356,7 @@ res2: Int = 10 # Where to Go from Here You can see some [example Spark programs](http://www.spark-project.org/examples.html) on the Spark website. -In addition, Spark includes several sample programs in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `run` script included in Spark -- for example, `./run spark.examples.SparkPi`. Each example program prints usage help when run without any arguments. +In addition, Spark includes several sample programs in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `run-example` script included in Spark -- for example, `./run-example spark.examples.SparkPi`. Each example program prints usage help when run without any arguments. For help on optimizing your program, the [configuration](configuration.html) and [tuning](tuning.html) guides provide information on best practices. They are especially important for diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 7463844a4e3..bb8be276c53 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -20,7 +20,7 @@ Compile Spark with `sbt package` as described in the [Getting Started Guide](ind You can start a standalone master server by executing: - ./run spark.deploy.master.Master + ./spark-class spark.deploy.master.Master Once started, the master will print out a `spark://IP:PORT` URL for itself, which you can use to connect workers to it, or pass as the "master" argument to `SparkContext` to connect a job to the cluster. You can also find this URL on @@ -28,7 +28,7 @@ the master's web UI, which is [http://localhost:8080](http://localhost:8080) by Similarly, you can start one or more workers and connect them to the master via: - ./run spark.deploy.worker.Worker spark://IP:PORT + ./spark-class spark.deploy.worker.Worker spark://IP:PORT Once you have started a worker, look at the master's web UI ([http://localhost:8080](http://localhost:8080) by default). You should see the new node listed there, along with its number of CPUs and memory (minus one gigabyte left for the OS). diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index a74c17bdb7a..3330e635986 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -234,7 +234,7 @@ $ nc -lk 9999 Then, in a different terminal, you can start NetworkWordCount by using {% highlight bash %} -$ ./run spark.streaming.examples.NetworkWordCount local[2] localhost 9999 +$ ./run-example spark.streaming.examples.NetworkWordCount local[2] localhost 9999 {% endhighlight %} This will make NetworkWordCount connect to the netcat server. Any lines typed in the terminal running the netcat server will be counted and printed on screen. @@ -272,7 +272,7 @@ Time: 1357008430000 ms
    Pool Name Minimum Share
    {d}
    {worker.state} {worker.cores} ({worker.coresUsed} Used) - {Utils.memoryMegabytesToString(worker.memory)} - ({Utils.memoryMegabytesToString(worker.memoryUsed)} Used) + {Utils.megabytesToString(worker.memory)} + ({Utils.megabytesToString(worker.memoryUsed)} Used)
    - {Utils.memoryMegabytesToString(app.desc.memoryPerSlave)} + {Utils.megabytesToString(app.desc.memoryPerSlave)} {DeployWebUI.formatDate(app.submitDate)} {app.desc.user} {executor.execId} {executor.cores} - {Utils.memoryMegabytesToString(executor.memory)} + {Utils.megabytesToString(executor.memory)}
      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 22295069dc7..717619f80d1 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -114,7 +114,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I .format(appId, executorId, logType, math.max(startByte-byteLength, 0), byteLength)}> } @@ -129,7 +129,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala index 2a09a956ad0..e5531011c2b 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/spark/scheduler/SparkListener.scala @@ -153,7 +153,7 @@ object StatsReportListener extends Logging { } def showBytesDistribution(heading: String, dist: Distribution) { - showDistribution(heading, dist, (d => Utils.memoryBytesToString(d.toLong)): Double => String) + showDistribution(heading, dist, (d => Utils.bytesToString(d.toLong)): Double => String) } def showMillisDistribution(heading: String, dOpt: Option[Distribution]) { diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 55d6c0a47ef..42c3b4a6cf6 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -77,7 +77,7 @@ private[spark] class SparkDeploySchedulerBackend( override def executorAdded(executorId: String, workerId: String, hostPort: String, cores: Int, memory: Int) { logInfo("Granted executor ID %s on hostPort %s with %d cores, %s RAM".format( - executorId, hostPort, cores, Utils.memoryMegabytesToString(memory))) + executorId, hostPort, cores, Utils.megabytesToString(memory))) } override def executorRemoved(executorId: String, message: String, exitStatus: Option[Int]) { diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala index 07e3db30fe3..568783d8931 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/spark/storage/BlockFetcherIterator.scala @@ -111,7 +111,7 @@ object BlockFetcherIterator { 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)) + req.blocks.size, Utils.bytesToString(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)) @@ -310,7 +310,7 @@ object BlockFetcherIterator { } logDebug("Sending request for %d blocks (%s) from %s".format( - req.blocks.size, Utils.memoryBytesToString(req.size), req.address.host)) + req.blocks.size, Utils.bytesToString(req.size), req.address.host)) val cmId = new ConnectionManagerId(req.address.host, req.address.nettyPort) val cpier = new ShuffleCopier cpier.getBlocks(cmId, req.blocks, putResult) diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala index 011bb6b83d7..2a2e1785504 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala @@ -332,7 +332,7 @@ object BlockManagerMasterActor { private val _blocks = new JHashMap[String, BlockStatus] logInfo("Registering block manager %s with %s RAM".format( - blockManagerId.hostPort, Utils.memoryBytesToString(maxMem))) + blockManagerId.hostPort, Utils.bytesToString(maxMem))) def updateLastSeenMs() { _lastSeenMs = System.currentTimeMillis() @@ -358,12 +358,12 @@ object BlockManagerMasterActor { if (storageLevel.useMemory) { _remainingMem -= memSize logInfo("Added %s in memory on %s (size: %s, free: %s)".format( - blockId, blockManagerId.hostPort, Utils.memoryBytesToString(memSize), - Utils.memoryBytesToString(_remainingMem))) + blockId, blockManagerId.hostPort, Utils.bytesToString(memSize), + Utils.bytesToString(_remainingMem))) } if (storageLevel.useDisk) { logInfo("Added %s on disk on %s (size: %s)".format( - blockId, blockManagerId.hostPort, Utils.memoryBytesToString(diskSize))) + blockId, blockManagerId.hostPort, Utils.bytesToString(diskSize))) } } else if (_blocks.containsKey(blockId)) { // If isValid is not true, drop the block. @@ -372,12 +372,12 @@ object BlockManagerMasterActor { if (blockStatus.storageLevel.useMemory) { _remainingMem += blockStatus.memSize logInfo("Removed %s on %s in memory (size: %s, free: %s)".format( - blockId, blockManagerId.hostPort, Utils.memoryBytesToString(memSize), - Utils.memoryBytesToString(_remainingMem))) + blockId, blockManagerId.hostPort, Utils.bytesToString(memSize), + Utils.bytesToString(_remainingMem))) } if (blockStatus.storageLevel.useDisk) { logInfo("Removed %s on %s on disk (size: %s)".format( - blockId, blockManagerId.hostPort, Utils.memoryBytesToString(diskSize))) + blockId, blockManagerId.hostPort, Utils.bytesToString(diskSize))) } } } diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala index 3ebfe173b1e..b14497157e9 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/spark/storage/DiskStore.scala @@ -147,7 +147,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) channel.close() val finishTime = System.currentTimeMillis logDebug("Block %s stored as %s file on disk in %d ms".format( - blockId, Utils.memoryBytesToString(bytes.limit), (finishTime - startTime))) + blockId, Utils.bytesToString(bytes.limit), (finishTime - startTime))) } private def getFileBytes(file: File): ByteBuffer = { @@ -181,7 +181,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String) val timeTaken = System.currentTimeMillis - startTime logDebug("Block %s stored as %s file on disk in %d ms".format( - blockId, Utils.memoryBytesToString(length), timeTaken)) + blockId, Utils.bytesToString(length), timeTaken)) if (returnValues) { // Return a byte buffer for the contents of the file diff --git a/core/src/main/scala/spark/storage/MemoryStore.scala b/core/src/main/scala/spark/storage/MemoryStore.scala index b5a86b85a77..5a51f5cf313 100644 --- a/core/src/main/scala/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/spark/storage/MemoryStore.scala @@ -38,7 +38,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) // blocks from the memory store. private val putLock = new Object() - logInfo("MemoryStore started with capacity %s.".format(Utils.memoryBytesToString(maxMemory))) + logInfo("MemoryStore started with capacity %s.".format(Utils.bytesToString(maxMemory))) def freeMemory: Long = maxMemory - currentMemory @@ -164,10 +164,10 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) currentMemory += size if (deserialized) { logInfo("Block %s stored as values to memory (estimated size %s, free %s)".format( - blockId, Utils.memoryBytesToString(size), Utils.memoryBytesToString(freeMemory))) + blockId, Utils.bytesToString(size), Utils.bytesToString(freeMemory))) } else { logInfo("Block %s stored as bytes to memory (size %s, free %s)".format( - blockId, Utils.memoryBytesToString(size), Utils.memoryBytesToString(freeMemory))) + blockId, Utils.bytesToString(size), Utils.bytesToString(freeMemory))) } true } else { diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala index 2aeed4ea3cf..123b8f63457 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/spark/storage/StorageUtils.scala @@ -42,9 +42,9 @@ case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, numCachedPartitions: Int, numPartitions: Int, memSize: Long, diskSize: Long) extends Ordered[RDDInfo] { override def toString = { - import Utils.memoryBytesToString + import Utils.bytesToString "RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; DiskSize: %s".format(name, id, - storageLevel.toString, numCachedPartitions, numPartitions, memoryBytesToString(memSize), memoryBytesToString(diskSize)) + storageLevel.toString, numCachedPartitions, numPartitions, bytesToString(memSize), bytesToString(diskSize)) } override def compare(that: RDDInfo) = { diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index 43e0c20b19a..c6db56115fe 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -50,10 +50,10 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
    {kv(1)} {kv(2)} - {Utils.memoryBytesToString(kv(3).toLong)} / {Utils.memoryBytesToString(kv(4).toLong)} + {Utils.bytesToString(kv(3).toLong)} / {Utils.bytesToString(kv(4).toLong)} - {Utils.memoryBytesToString(kv(5).toLong)} + {Utils.bytesToString(kv(5).toLong)} {kv(6)} {kv(7)} {dateFmt.format(new Date(info.launchTime))}{metrics.flatMap{m => m.shuffleReadMetrics}.map{s => - Utils.memoryBytesToString(s.remoteBytesRead)}.getOrElse("")}{metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => - Utils.memoryBytesToString(s.shuffleBytesWritten)}.getOrElse("")} {if (gcTime > 0) parent.formatDuration(gcTime) else ""} diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/spark/ui/jobs/StageTable.scala index 19b07ccedac..bdf8d919583 100644 --- a/core/src/main/scala/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/spark/ui/jobs/StageTable.scala @@ -71,11 +71,11 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU val shuffleRead = listener.stageToShuffleRead.getOrElse(s.id, 0L) match { case 0 => "" - case b => Utils.memoryBytesToString(b) + case b => Utils.bytesToString(b) } val shuffleWrite = listener.stageToShuffleWrite.getOrElse(s.id, 0L) match { case 0 => "" - case b => Utils.memoryBytesToString(b) + case b => Utils.bytesToString(b) } val startedTasks = listener.stageToTasksActive.getOrElse(s.id, HashSet[TaskInfo]()).size diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index f76192eba84..0751f9e8f91 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -58,8 +58,8 @@ private[spark] class IndexPage(parent: BlockManagerUI) { {rdd.numCachedPartitions} {rdd.numCachedPartitions / rdd.numPartitions.toDouble}{Utils.memoryBytesToString(rdd.memSize)}{Utils.memoryBytesToString(rdd.diskSize)}{Utils.bytesToString(rdd.memSize)}{Utils.bytesToString(rdd.diskSize)}
    - {Utils.memoryBytesToString(block.memSize)} + {Utils.bytesToString(block.memSize)} - {Utils.memoryBytesToString(block.diskSize)} + {Utils.bytesToString(block.diskSize)} {locations.map(l => {l}
    )} @@ -122,10 +122,10 @@ private[spark] class RDDPage(parent: BlockManagerUI) {
    {status.blockManagerId.host + ":" + status.blockManagerId.port} - {Utils.memoryBytesToString(status.memUsed(prefix))} - ({Utils.memoryBytesToString(status.memRemaining)} Remaining) + {Utils.bytesToString(status.memUsed(prefix))} + ({Utils.bytesToString(status.memRemaining)} Remaining) {Utils.memoryBytesToString(status.diskUsed(prefix))}{Utils.bytesToString(status.diskUsed(prefix))}
    {info.taskId} {info.status} {info.taskLocality}{info.hostPort}{info.host} {dateFmt.format(new Date(info.launchTime))} {formatDuration} diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala index ce6cec0451e..c21f3331d08 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala @@ -112,22 +112,22 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { "akka://spark@localhost:" + boundPort + "/user/MapOutputTracker") masterTracker.registerShuffle(10, 1) - masterTracker.incrementGeneration() - slaveTracker.updateGeneration(masterTracker.getGeneration) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } val compressedSize1000 = MapOutputTracker.compressSize(1000L) val size1000 = MapOutputTracker.decompressSize(compressedSize1000) masterTracker.registerMapOutput(10, 0, new MapStatus( BlockManagerId("a", "hostA", 1000, 0), Array(compressedSize1000))) - masterTracker.incrementGeneration() - slaveTracker.updateGeneration(masterTracker.getGeneration) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getServerStatuses(10, 0).toSeq === Seq((BlockManagerId("a", "hostA", 1000, 0), size1000))) masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000, 0)) - masterTracker.incrementGeneration() - slaveTracker.updateGeneration(masterTracker.getGeneration) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } // failure should be cached diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala index 05afcd65676..63271551574 100644 --- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala @@ -72,7 +72,9 @@ class DummyTaskSetManager( override def executorLost(executorId: String, host: String): Unit = { } - override def slaveOffer(execId: String, host: String, avaiableCpus: Double, overrideLocality: TaskLocality.TaskLocality = null): Option[TaskDescription] = { + override def resourceOffer(execId: String, host: String, availableCpus: Double) + : Option[TaskDescription] = + { if (tasksFinished + runningTasks < numTasks) { increaseRunningTasks(1) return Some(new TaskDescription(0, execId, "task 0:0", null)) @@ -118,7 +120,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging 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 { + taskSet.resourceOffer("execId_1", "hostname_1", 1) match { case Some(task) => return taskSet.stageId case None => {} diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala index caaf3209fd6..3b4a0d52fc7 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala @@ -59,7 +59,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont override def stop() = {} override def submitTasks(taskSet: TaskSet) = { // normally done by TaskSetManager - taskSet.tasks.foreach(_.generation = mapOutputTracker.getGeneration) + taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) taskSets += taskSet } override def setListener(listener: TaskSchedulerListener) = {} @@ -299,10 +299,10 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val reduceRdd = makeRdd(2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) // pretend we were told hostA went away - val oldGeneration = mapOutputTracker.getGeneration + val oldEpoch = mapOutputTracker.getEpoch runEvent(ExecutorLost("exec-hostA")) - val newGeneration = mapOutputTracker.getGeneration - assert(newGeneration > oldGeneration) + val newEpoch = mapOutputTracker.getEpoch + assert(newEpoch > oldEpoch) val noAccum = Map[Long, Any]() val taskSet = taskSets(0) // should be ignored for being too old @@ -311,8 +311,8 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), noAccum, null, null)) // should be ignored for being too old runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum, null, null)) - // should work because it's a new generation - taskSet.tasks(1).generation = newGeneration + // should work because it's a new epoch + taskSet.tasks(1).epoch = newEpoch runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum, null, null)) assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) @@ -401,12 +401,14 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assert(results === Map(0 -> 42)) } - /** 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.map(loc => spark.Utils.parseHostPort(loc)._1) === taskLocs) + /** + * Assert that the supplied TaskSet has exactly the given hosts as its preferred locations. + * Note that this checks only the host and not the executor ID. + */ + private def assertLocations(taskSet: TaskSet, hosts: Seq[Seq[String]]) { + assert(hosts.size === taskSet.tasks.size) + for ((taskLocs, expectedLocs) <- taskSet.tasks.map(_.preferredLocations).zip(hosts)) { + assert(taskLocs.map(_.host) === expectedLocs) } } From cf39d45d14e0256bbd3ffe206c14997f02429cb3 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 13 Aug 2013 12:32:40 -0700 Subject: [PATCH 0774/2521] More scheduling fixes: - Added periodic revival of offers in StandaloneSchedulerBackend - Replaced task scheduling aggression with multi-level delay scheduling in ClusterTaskSetManager - Fixed ZippedRDD preferred locations because they can't currently be process-local - Fixed some uses of hostPort --- core/src/main/scala/spark/SparkEnv.scala | 19 +-- core/src/main/scala/spark/Utils.scala | 31 +---- .../scala/spark/deploy/DeployMessage.scala | 1 + .../spark/deploy/worker/ExecutorRunner.scala | 6 +- .../scala/spark/deploy/worker/Worker.scala | 2 +- .../main/scala/spark/rdd/CartesianRDD.scala | 2 +- .../scala/spark/rdd/ZippedPartitionsRDD.scala | 32 ++--- core/src/main/scala/spark/rdd/ZippedRDD.scala | 23 +--- .../scheduler/cluster/ClusterScheduler.scala | 68 +--------- .../cluster/ClusterTaskSetManager.scala | 116 +++++++++++++----- .../cluster/StandaloneSchedulerBackend.scala | 7 +- 11 files changed, 117 insertions(+), 190 deletions(-) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala index 0adbf1d96e9..bca90886a32 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/spark/SparkEnv.scala @@ -54,11 +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 - var executorIdToHostPort: Option[(String, String) => String]) { + val metricsSystem: MetricsSystem) { private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() @@ -83,16 +79,6 @@ class SparkEnv ( pythonWorkers.getOrElseUpdate(key, new PythonWorkerFactory(pythonExec, envVars)).create() } } - - def resolveExecutorIdToHostPort(executorId: String, defaultHostPort: String): String = { - val env = SparkEnv.get - if (env.executorIdToHostPort.isEmpty) { - // default to using host, not host port. Relevant to non cluster modes. - return defaultHostPort - } - - env.executorIdToHostPort.get(executorId, defaultHostPort) - } } object SparkEnv extends Logging { @@ -236,7 +222,6 @@ object SparkEnv extends Logging { connectionManager, httpFileServer, sparkFilesDir, - metricsSystem, - None) + metricsSystem) } } diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala index 885a7391d63..a05dcdcd97d 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/spark/Utils.scala @@ -393,41 +393,14 @@ 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 = "") { - // Currently catches only ipv4 pattern, this is just a debugging tool - not rigourous ! - // if (host.matches("^[0-9]+(\\.[0-9]+)*$")) { - if (ipPattern.matcher(host).matches()) { - Utils.logErrorWithStack("Unexpected to have host " + host + " which matches IP pattern. Message " + message) - } - if (Utils.parseHostPort(host)._2 != 0){ - Utils.logErrorWithStack("Unexpected to have host " + host + " which has port in it. Message " + message) - } + assert(host.indexOf(':') == -1, message) } - // Used by DEBUG code : remove when all testing done def checkHostPort(hostPort: String, message: String = "") { - val (host, port) = Utils.parseHostPort(hostPort) - checkHost(host) - if (port <= 0){ - Utils.logErrorWithStack("Unexpected to have port " + port + " which is not valid in " + hostPort + ". Message " + message) - } + assert(hostPort.indexOf(':') != -1, 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 = "") {} - // Used by DEBUG code : remove when all testing done def logErrorWithStack(msg: String) { try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } } diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala index 31861f3ac2d..0db13ffc987 100644 --- a/core/src/main/scala/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/spark/deploy/DeployMessage.scala @@ -80,6 +80,7 @@ private[deploy] object DeployMessages { case class RegisteredApplication(appId: String) extends DeployMessage + // TODO(matei): replace hostPort with host case class ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) { Utils.checkHostPort(hostPort, "Required hostport") } diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index f661accd2ff..5e53d95ac2f 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -40,13 +40,11 @@ private[spark] class ExecutorRunner( val memory: Int, val worker: ActorRef, val workerId: String, - val hostPort: String, + val host: String, val sparkHome: File, val workDir: File) extends Logging { - Utils.checkHostPort(hostPort, "Expected hostport") - val fullId = appId + "/" + execId var workerThread: Thread = null var process: Process = null @@ -92,7 +90,7 @@ private[spark] class ExecutorRunner( /** Replace variables such as {{EXECUTOR_ID}} and {{CORES}} in a command argument passed to us */ def substituteVariables(argument: String): String = argument match { case "{{EXECUTOR_ID}}" => execId.toString - case "{{HOSTNAME}}" => Utils.parseHostPort(hostPort)._1 + case "{{HOSTNAME}}" => host case "{{CORES}}" => cores.toString case other => other } diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index d4b58fc34e8..053ac552260 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -132,7 +132,7 @@ private[spark] class Worker( case LaunchExecutor(appId, execId, appDesc, cores_, memory_, execSparkHome_) => logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name)) val manager = new ExecutorRunner( - appId, execId, appDesc, cores_, memory_, self, workerId, host + ":" + port, new File(execSparkHome_), workDir) + appId, execId, appDesc, cores_, memory_, self, workerId, host, new File(execSparkHome_), workDir) executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ diff --git a/core/src/main/scala/spark/rdd/CartesianRDD.scala b/core/src/main/scala/spark/rdd/CartesianRDD.scala index 150e5bca29d..91b3e69d6fb 100644 --- a/core/src/main/scala/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/spark/rdd/CartesianRDD.scala @@ -64,7 +64,7 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest]( override def getPreferredLocations(split: Partition): Seq[String] = { val currSplit = split.asInstanceOf[CartesianPartition] - rdd1.preferredLocations(currSplit.s1) ++ rdd2.preferredLocations(currSplit.s2) + (rdd1.preferredLocations(currSplit.s1) ++ rdd2.preferredLocations(currSplit.s2)).distinct } override def compute(split: Partition, context: TaskContext) = { diff --git a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala index 51f5cc32517..9a0831bd899 100644 --- a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala @@ -55,29 +55,15 @@ abstract class ZippedPartitionsBaseRDD[V: ClassManifest]( } override def getPreferredLocations(s: Partition): Seq[String] = { - // TODO(matei): Fix this for hostPort - - // 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 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 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 + val parts = s.asInstanceOf[ZippedPartitionsPartition].partitions + val prefs = rdds.zip(parts).map { case (rdd, p) => rdd.preferredLocations(p) } + // Check whether there are any hosts that match all RDDs; otherwise return the union + val exactMatchLocations = prefs.reduce((x, y) => x.intersect(y)) + if (!exactMatchLocations.isEmpty) { + exactMatchLocations + } else { + prefs.flatten.distinct + } } override def clearDependencies() { diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala index b1c43b3195a..4074e50e440 100644 --- a/core/src/main/scala/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala @@ -65,27 +65,16 @@ 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) - - // exact match - instance local and host local. + // Check whether there are any hosts that match both RDDs; otherwise return the union val exactMatchLocations = pref1.intersect(pref2) - - // 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 ! - exactMatchLocations ++ otherNodeLocalLocations + if (!exactMatchLocations.isEmpty) { + exactMatchLocations + } else { + (pref1 ++ pref2).distinct + } } override def clearDependencies() { diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 036e36bca07..ec76e901855 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -45,39 +45,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext) // Threshold above which we warn user initial TaskSet may be starved val STARVATION_TIMEOUT = System.getProperty("spark.starvation.timeout", "15000").toLong - // How often to revive offers in case there are pending tasks - that is how often to try to get - // tasks scheduled in case there are nodes available : default 0 is to disable it - to preserve existing behavior - // Note that this is required due to delay scheduling due to data locality waits, etc. - // TODO(matei): move to StandaloneSchedulerBackend? - val TASK_REVIVAL_INTERVAL = System.getProperty("spark.scheduler.revival.interval", "1000").toLong - - // TODO(matei): replace this with something that only affects levels past PROCESS_LOCAL; - // basically it can be a "cliff" for locality - /* - 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 : - 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 - - 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 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.withName( - System.getProperty("spark.tasks.schedule.aggression", "NODE_LOCAL")) - val activeTaskSets = new HashMap[String, TaskSetManager] val taskIdToTaskSetId = new HashMap[Long, String] @@ -136,14 +103,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } } schedulableBuilder.buildPools() - // resolve executorId to hostPort mapping. - def executorToHostPort(executorId: String, defaultHostPort: String): String = { - executorIdToHost.getOrElse(executorId, defaultHostPort) - } - - // Unfortunately, this means that SparkEnv is indirectly referencing ClusterScheduler - // Will that be a design violation ? - SparkEnv.get.executorIdToHostPort = Some(executorToHostPort) } def newTaskId(): Long = nextTaskId.getAndIncrement() @@ -168,28 +127,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } }.start() } - - - // Change to always run with some default if TASK_REVIVAL_INTERVAL <= 0 ? - // TODO(matei): remove this thread - if (TASK_REVIVAL_INTERVAL > 0) { - new Thread("ClusterScheduler task offer revival check") { - setDaemon(true) - - override def run() { - logInfo("Starting speculative task offer revival thread") - while (true) { - try { - Thread.sleep(TASK_REVIVAL_INTERVAL) - } catch { - case e: InterruptedException => {} - } - - if (hasPendingTasks()) backend.reviveOffers() - } - } - }.start() - } } override def submitTasks(taskSet: TaskSet) { @@ -329,7 +266,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext) backend.reviveOffers() } if (taskFailed) { - // Also revive offers if a task had failed for some reason other than host lost backend.reviveOffers() } @@ -384,7 +320,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } // Check for pending tasks in all our active jobs. - def hasPendingTasks(): Boolean = { + def hasPendingTasks: Boolean = { synchronized { rootPool.hasPendingTasks() } @@ -416,10 +352,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext) /** Remove an executor from all our data structures and mark it as lost */ private def removeExecutor(executorId: String) { - // TODO(matei): remove HostPort activeExecutorIds -= executorId val host = executorIdToHost(executorId) - val execs = executorsByHost.getOrElse(host, new HashSet) execs -= executorId if (execs.isEmpty) { diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 1947c516dbc..cf406f876f5 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -42,9 +42,6 @@ import spark.TaskResultTooBigFailure 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 @@ -74,8 +71,6 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: 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 // Set of pending tasks for each executor. These collections are actually // treated as stacks, in which new tasks are added to the end of the @@ -114,11 +109,9 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: 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. + // 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 epoch and set it on all tasks @@ -134,6 +127,16 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: addPendingTask(i) } + // Figure out which locality levels we have in our TaskSet, so we can do delay scheduling + val myLocalityLevels = computeValidLocalityLevels() + val localityWaits = myLocalityLevels.map(getLocalityWait) // spark.locality.wait + + // Delay scheduling variables: we keep track of our current locality level and the time we + // last launched a task at that level, and move up a level when localityWaits[curLevel] expires. + // We then move down if we manage to launch a "more local" task. + var currentLocalityIndex = 0 // Index of our current locality level in validLocalityLevels + var lastLaunchTime = System.currentTimeMillis() // Time we last launched a task at this level + /** * Add a task to all the pending-task lists that it should be on. If readding is set, we are * re-adding the task so only include it in each list if it's not already there. @@ -169,7 +172,9 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: addTo(pendingTasksWithNoPrefs) } - addTo(allPendingTasks) + if (!readding) { + allPendingTasks += index // No point scanning this whole list to find the old task there + } } /** @@ -324,18 +329,9 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: : Option[TaskDescription] = { if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { - val curTime = System.currentTimeMillis - - // If explicitly specified, use that - val locality = { - // expand only if we have waited for more than LOCALITY_WAIT for a host local task ... - // TODO(matei): Multi-level delay scheduling - if (curTime - lastPreferredLaunchTime < LOCALITY_WAIT) { - TaskLocality.NODE_LOCAL - } else { - TaskLocality.ANY - } - } + val curTime = System.currentTimeMillis() + + val locality = getAllowedLocalityLevel(curTime) findTask(execId, host, locality) match { case Some((index, taskLocality)) => { @@ -350,16 +346,16 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: val info = new TaskInfo(taskId, index, curTime, execId, host, taskLocality) taskInfos(taskId) = info taskAttempts(index) = info :: taskAttempts(index) - if (taskLocality == TaskLocality.PROCESS_LOCAL || taskLocality == TaskLocality.NODE_LOCAL) { - lastPreferredLaunchTime = curTime - } + // Update our locality level for delay scheduling + currentLocalityIndex = getLocalityIndex(locality) + lastLaunchTime = curTime // Serialize and return the task - val startTime = System.currentTimeMillis + 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 + 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)) @@ -374,6 +370,34 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: return None } + /** + * Get the level we can launch tasks according to delay scheduling, based on current wait time. + */ + private def getAllowedLocalityLevel(curTime: Long): TaskLocality.TaskLocality = { + while (curTime - lastLaunchTime >= localityWaits(currentLocalityIndex) && + currentLocalityIndex < myLocalityLevels.length - 1) + { + // Jump to the next locality level, and remove our waiting time for the current one since + // we don't want to count it again on the next one + lastLaunchTime += localityWaits(currentLocalityIndex) + currentLocalityIndex += 1 + } + myLocalityLevels(currentLocalityIndex) + } + + /** + * Find the index in myLocalityLevels for a given locality. This is also designed to work with + * localities that are not in myLocalityLevels (in case we somehow get those) by returning the + * next-biggest level we have. Uses the fact that the last value in myLocalityLevels is ANY. + */ + def getLocalityIndex(locality: TaskLocality.TaskLocality): Int = { + var index = 0 + while (locality > myLocalityLevels(index)) { + index += 1 + } + index + } + /** Called by cluster scheduler when one of our tasks changes state */ override def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) { SparkEnv.set(env) @@ -467,7 +491,7 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: case ef: ExceptionFailure => sched.listener.taskEnded(tasks(index), ef, null, null, info, ef.metrics.getOrElse(null)) val key = ef.description - val now = System.currentTimeMillis + val now = System.currentTimeMillis() val (printFull, dupCount) = { if (recentExceptions.contains(key)) { val (dupCount, printTime) = recentExceptions(key) @@ -631,4 +655,38 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: override def hasPendingTasks(): Boolean = { numTasks > 0 && tasksFinished < numTasks } + + private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { + val defaultWait = System.getProperty("spark.locality.wait", "3000") + level match { + case TaskLocality.PROCESS_LOCAL => + System.getProperty("spark.locality.wait.process", defaultWait).toLong + case TaskLocality.NODE_LOCAL => + System.getProperty("spark.locality.wait.node", defaultWait).toLong + case TaskLocality.RACK_LOCAL => + System.getProperty("spark.locality.wait.rack", defaultWait).toLong + case TaskLocality.ANY => + 0L + } + } + + /** + * Compute the locality levels used in this TaskSet. Assumes that all tasks have already been + * added to queues using addPendingTask. + */ + private def computeValidLocalityLevels(): Array[TaskLocality.TaskLocality] = { + import TaskLocality.{PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY} + val levels = new ArrayBuffer[TaskLocality.TaskLocality] + if (!pendingTasksForExecutor.isEmpty && getLocalityWait(PROCESS_LOCAL) != 0) { + levels += PROCESS_LOCAL + } + if (!pendingTasksForHost.isEmpty && getLocalityWait(NODE_LOCAL) != 0) { + levels += NODE_LOCAL + } + if (!pendingTasksForRack.isEmpty && getLocalityWait(RACK_LOCAL) != 0) { + levels += RACK_LOCAL + } + levels += ANY + levels.toArray + } } diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 3b49af12585..3203be10294 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -26,6 +26,7 @@ import akka.dispatch.Await import akka.pattern.ask import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent} import akka.util.Duration +import akka.util.duration._ import spark.{Utils, SparkException, Logging, TaskState} import spark.scheduler.cluster.StandaloneClusterMessages._ @@ -39,8 +40,6 @@ private[spark] class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: ActorSystem) extends SchedulerBackend with Logging { - // TODO(matei): periodically revive offers as in MesosScheduler - // Use an atomic variable to track total number of cores in the cluster for simplicity and speed var totalCoreCount = new AtomicInteger(0) @@ -55,6 +54,10 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor override def preStart() { // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) + + // Periodically revive offers to allow delay scheduling to work + val reviveInterval = System.getProperty("spark.scheduler.revive.interval", "1000").toLong + context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers) } def receive = { From 222c8971285190761354456c2fe07f5c31edf330 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 14 Aug 2013 13:56:40 -0700 Subject: [PATCH 0775/2521] Comment cleanup (via Kay) and some debug messages --- .../scheduler/cluster/ClusterScheduler.scala | 30 +++++++------------ .../cluster/ClusterTaskSetManager.scala | 5 ++-- .../{ => cluster}/ClusterSchedulerSuite.scala | 2 +- .../{ => local}/LocalSchedulerSuite.scala | 2 +- 4 files changed, 16 insertions(+), 23 deletions(-) rename core/src/test/scala/spark/scheduler/{ => cluster}/ClusterSchedulerSuite.scala (99%) rename core/src/test/scala/spark/scheduler/{ => local}/LocalSchedulerSuite.scala (99%) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index ec76e901855..028f4d3283a 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -385,25 +385,17 @@ private[spark] class ClusterScheduler(val sc: SparkContext) object ClusterScheduler { - - // Used to 'spray' available containers across the available set to ensure too many containers on same host - // are not used up. Used in yarn mode and in task scheduling (when there are multiple containers available - // to execute a task) - // For example: yarn can returns more containers than we would have requested under ANY, this method - // prioritizes how to use the allocated containers. - // flatten the map such that the array buffer entries are spread out across the returned value. - // given == , , , , , i - // the return value would be something like : h1c1, h2c1, h3c1, h4c1, h5c1, h1c2, h2c2, h3c2, h1c3, h2c3, h1c4, h1c5 - // We then 'use' the containers in this order (consuming only the top K from this list where - // K = number to be user). This is to ensure that if we have multiple eligible allocations, - // they dont end up allocating all containers on a small number of hosts - increasing probability of - // multiple container failure when a host goes down. - // Note, there is bias for keys with higher number of entries in value to be picked first (by design) - // Also note that invocation of this method is expected to have containers of same 'type' - // (host-local, rack-local, off-rack) and not across types : so that reordering is simply better from - // the available list - everything else being same. - // That is, we we first consume data local, then rack local and finally off rack nodes. So the - // prioritization from this method applies to within each category + /** + * Used to balance containers across hosts. + * + * Accepts a map of hosts to resource offers for that host, and returns a prioritized list of + * resource offers representing the order in which the offers should be used. The resource + * offers are ordered such that we'll allocate one container on each host before allocating a + * second container on any host, and so on, in order to reduce the damage if a host fails. + * + * For example, given , , , returns + * [o1, o5, o4, 02, o6, o3] + */ def prioritizeContainers[K, T] (map: HashMap[K, ArrayBuffer[T]]): List[T] = { val _keyList = new ArrayBuffer[K](map.size) _keyList ++= map.keys diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index cf406f876f5..5316a7aed10 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -116,7 +116,7 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: // Figure out the current map output tracker epoch and set it on all tasks val epoch = sched.mapOutputTracker.getEpoch - logDebug("Epoch for " + taskSet.id + ": " + epoch) + logDebug("Epoch for " + taskSet + ": " + epoch) for (t <- tasks) { t.epoch = epoch } @@ -129,7 +129,7 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: // Figure out which locality levels we have in our TaskSet, so we can do delay scheduling val myLocalityLevels = computeValidLocalityLevels() - val localityWaits = myLocalityLevels.map(getLocalityWait) // spark.locality.wait + val localityWaits = myLocalityLevels.map(getLocalityWait) // Time to wait at each level // Delay scheduling variables: we keep track of our current locality level and the time we // last launched a task at that level, and move up a level when localityWaits[curLevel] expires. @@ -687,6 +687,7 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: levels += RACK_LOCAL } levels += ANY + logDebug("Valid locality levels for " + taskSet + ": " + levels.mkString(", ")) levels.toArray } } diff --git a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/cluster/ClusterSchedulerSuite.scala similarity index 99% rename from core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala rename to core/src/test/scala/spark/scheduler/cluster/ClusterSchedulerSuite.scala index 63271551574..8618009ea64 100644 --- a/core/src/test/scala/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/cluster/ClusterSchedulerSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler +package spark.scheduler.cluster import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter diff --git a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/local/LocalSchedulerSuite.scala similarity index 99% rename from core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala rename to core/src/test/scala/spark/scheduler/local/LocalSchedulerSuite.scala index a79b8bf2560..d28ee47fa33 100644 --- a/core/src/test/scala/spark/scheduler/LocalSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/local/LocalSchedulerSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler +package spark.scheduler.local import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter From 2a4ed10210f9ee32f472e2465094d88561c0ff18 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 15 Aug 2013 17:22:49 -0700 Subject: [PATCH 0776/2521] Address some review comments: - When a resourceOffers() call has multiple offers, force the TaskSets to consider them in increasing order of locality levels so that they get a chance to launch stuff locally across all offers - Simplify ClusterScheduler.prioritizeContainers - Add docs on the new configuration options --- .../scheduler/cluster/ClusterScheduler.scala | 21 ++++++------- .../cluster/ClusterTaskSetManager.scala | 17 +++++++---- .../scheduler/cluster/TaskSetManager.scala | 6 +++- .../scheduler/local/LocalScheduler.scala | 3 +- .../scheduler/local/LocalTaskSetManager.scala | 6 +++- .../cluster/ClusterSchedulerSuite.scala | 8 +++-- docs/configuration.md | 30 +++++++++++++++++-- 7 files changed, 68 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala index 028f4d3283a..e88edc5b2ac 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala @@ -184,27 +184,29 @@ private[spark] class ClusterScheduler(val sc: SparkContext) } } - // Build a list of tasks to assign to each slave + // Build a list of tasks to assign to each worker val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) val availableCpus = offers.map(o => o.cores).toArray - val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue() - for (manager <- sortedTaskSetQueue) { + val sortedTaskSets = rootPool.getSortedTaskSetQueue() + for (taskSet <- sortedTaskSets) { logDebug("parentName: %s, name: %s, runningTasks: %s".format( - manager.parent.name, manager.name, manager.runningTasks)) + taskSet.parent.name, taskSet.name, taskSet.runningTasks)) } + // Take each TaskSet in our scheduling order, and then offer it each node in increasing order + // of locality levels so that it gets a chance to launch local tasks on all of them. var launchedTask = false - for (manager <- sortedTaskSetQueue; offer <- offers) { + for (taskSet <- sortedTaskSets; maxLocality <- TaskLocality.values) { do { launchedTask = false for (i <- 0 until offers.size) { val execId = offers(i).executorId val host = offers(i).host - for (task <- manager.resourceOffer(execId, host, availableCpus(i))) { + for (task <- taskSet.resourceOffer(execId, host, availableCpus(i), maxLocality)) { tasks(i) += task val tid = task.taskId - taskIdToTaskSetId(tid) = manager.taskSet.id - taskSetTaskIds(manager.taskSet.id) += tid + taskIdToTaskSetId(tid) = taskSet.taskSet.id + taskSetTaskIds(taskSet.taskSet.id) += tid taskIdToExecutorId(tid) = execId activeExecutorIds += execId executorsByHost(host) += execId @@ -402,8 +404,7 @@ object ClusterScheduler { // order keyList based on population of value in map val keyList = _keyList.sortWith( - // TODO(matei): not sure why we're using getOrElse if keyList = map.keys... see if it matters - (left, right) => map.get(left).getOrElse(Set()).size > map.get(right).getOrElse(Set()).size + (left, right) => map(left).size > map(right).size ) val retval = new ArrayBuffer[T](keyList.size * 2) diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala index 5316a7aed10..91de25254c0 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -43,7 +43,7 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: extends TaskSetManager with Logging { // CPUs to request per task - val CPUS_PER_TASK = System.getProperty("spark.task.cpus", "1").toDouble + val CPUS_PER_TASK = System.getProperty("spark.task.cpus", "1").toInt // Maximum times a task is allowed to fail before failing the job val MAX_TASK_FAILURES = System.getProperty("spark.task.maxFailures", "4").toInt @@ -325,15 +325,22 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: /** * Respond to an offer of a single slave from the scheduler by finding a task */ - override def resourceOffer(execId: String, host: String, availableCpus: Double) + override def resourceOffer( + execId: String, + host: String, + availableCpus: Int, + maxLocality: TaskLocality.TaskLocality) : Option[TaskDescription] = { if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) { val curTime = System.currentTimeMillis() - val locality = getAllowedLocalityLevel(curTime) + var allowedLocality = getAllowedLocalityLevel(curTime) + if (allowedLocality > maxLocality) { + allowedLocality = maxLocality // We're not allowed to search for farther-away tasks + } - findTask(execId, host, locality) match { + findTask(execId, host, allowedLocality) match { case Some((index, taskLocality)) => { // Found a task; do some bookkeeping and return a task description val task = tasks(index) @@ -347,7 +354,7 @@ private[spark] class ClusterTaskSetManager(sched: ClusterScheduler, val taskSet: taskInfos(taskId) = info taskAttempts(index) = info :: taskAttempts(index) // Update our locality level for delay scheduling - currentLocalityIndex = getLocalityIndex(locality) + currentLocalityIndex = getLocalityIndex(allowedLocality) lastLaunchTime = curTime // Serialize and return the task val startTime = System.currentTimeMillis() diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index 277654edc07..5ab6ab9aadb 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -29,7 +29,11 @@ private[spark] trait TaskSetManager extends Schedulable { def taskSet: TaskSet - def resourceOffer(execId: String, hostPort: String, availableCpus: Double) + def resourceOffer( + execId: String, + host: String, + availableCpus: Int, + maxLocality: TaskLocality.TaskLocality) : Option[TaskDescription] def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala index a4f5f46777a..5be4dbd9f0b 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala @@ -141,8 +141,7 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: for (manager <- sortedTaskSetQueue) { do { launchTask = false - // TODO(matei): don't pass null here? - manager.resourceOffer(null, null, freeCpuCores) match { + manager.resourceOffer(null, null, freeCpuCores, null) match { case Some(task) => tasks += task taskIdToTaskSetId(task.taskId) = manager.taskSet.id diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala index 698c777bec6..3ef636ff075 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala @@ -98,7 +98,11 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas return None } - override def resourceOffer(execId: String, host: String, availableCpus: Double) + override def resourceOffer( + execId: String, + host: String, + availableCpus: Int, + maxLocality: TaskLocality.TaskLocality) : Option[TaskDescription] = { SparkEnv.set(sched.env) diff --git a/core/src/test/scala/spark/scheduler/cluster/ClusterSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/cluster/ClusterSchedulerSuite.scala index 8618009ea64..aeeed147864 100644 --- a/core/src/test/scala/spark/scheduler/cluster/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/spark/scheduler/cluster/ClusterSchedulerSuite.scala @@ -72,7 +72,11 @@ class DummyTaskSetManager( override def executorLost(executorId: String, host: String): Unit = { } - override def resourceOffer(execId: String, host: String, availableCpus: Double) + override def resourceOffer( + execId: String, + host: String, + availableCpus: Int, + maxLocality: TaskLocality.TaskLocality) : Option[TaskDescription] = { if (tasksFinished + runningTasks < numTasks) { @@ -120,7 +124,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging 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.resourceOffer("execId_1", "hostname_1", 1) match { + taskSet.resourceOffer("execId_1", "hostname_1", 1, TaskLocality.ANY) match { case Some(task) => return taskSet.stageId case None => {} diff --git a/docs/configuration.md b/docs/configuration.md index 99624a44aa9..dff08a06f56 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -243,8 +243,34 @@ Apart from these, the following properties are also available, and may be useful 3000 Number of milliseconds to wait to launch a data-local task before giving up and launching it - in a non-data-local location. You should increase this if your tasks are long and you are seeing - poor data locality, but the default generally works well. + on a less-local node. The same wait will be used to step through multiple locality levels + (process-local, node-local, rack-local and then any). It is also possible to customize the + waiting time for each level by setting spark.locality.wait.node, etc. + You should increase this setting if your tasks are long and see poor locality, but the + default usually works well. +
    spark.locality.wait.processspark.locality.wait + Customize the locality wait for process locality. This affects tasks that attempt to access + cached data in a particular executor process. +
    spark.locality.wait.nodespark.locality.wait + Customize the locality wait for node locality. For example, you can set this to 0 to skip + node locality and search immediately for rack locality (if your cluster has rack information). +
    spark.locality.wait.rackspark.locality.wait + Customize the locality wait for rack locality.
    spark.ui.port330003030 Port for your application's dashboard, which shows memory and workload data
    -You can find more examples in `/streaming/src/main/scala/spark/streaming/examples/`. They can be run in the similar manner using `./run spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files. +You can find more examples in `/streaming/src/main/scala/spark/streaming/examples/`. They can be run in the similar manner using `./run-example spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files. # DStream Persistence Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is, using `persist()` method on a DStream would automatically persist every RDD of that DStream in memory. This is useful if the data in the DStream will be computed multiple times (e.g., multiple operations on the same data). For window-based operations like `reduceByWindow` and `reduceByKeyAndWindow` and state-based operations like `updateStateByKey`, this is implicitly true. Hence, DStreams generated by window-based operations are automatically persisted in memory, without the developer calling `persist()`. diff --git a/examples/pom.xml b/examples/pom.xml index 0db52b8691c..d24bd404fa2 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -47,6 +47,11 @@ spark-mllib ${project.version}
    + + org.spark-project + spark-bagel + ${project.version} + org.apache.hbase hbase diff --git a/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala b/examples/src/main/scala/spark/examples/bagel/PageRankUtils.scala similarity index 99% rename from bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala rename to examples/src/main/scala/spark/examples/bagel/PageRankUtils.scala index de65e27fe0e..c23ee9895f2 100644 --- a/bagel/src/main/scala/spark/bagel/examples/PageRankUtils.scala +++ b/examples/src/main/scala/spark/examples/bagel/PageRankUtils.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.bagel.examples +package spark.examples.bagel import spark._ import spark.SparkContext._ diff --git a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala b/examples/src/main/scala/spark/examples/bagel/WikipediaPageRank.scala similarity index 99% rename from bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala rename to examples/src/main/scala/spark/examples/bagel/WikipediaPageRank.scala index a0c5ac9c183..00635a7ffa2 100644 --- a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala +++ b/examples/src/main/scala/spark/examples/bagel/WikipediaPageRank.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.bagel.examples +package spark.examples.bagel import spark._ import spark.SparkContext._ diff --git a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala b/examples/src/main/scala/spark/examples/bagel/WikipediaPageRankStandalone.scala similarity index 99% rename from bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala rename to examples/src/main/scala/spark/examples/bagel/WikipediaPageRankStandalone.scala index 3c54a85f425..c416ddbc584 100644 --- a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala +++ b/examples/src/main/scala/spark/examples/bagel/WikipediaPageRankStandalone.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.bagel.examples +package spark.examples.bagel import spark._ import serializer.{DeserializationStream, SerializationStream, SerializerInstance} diff --git a/make-distribution.sh b/make-distribution.sh index 70aff418c70..df7bbf1e748 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -47,7 +47,7 @@ VERSION=$($FWDIR/sbt/sbt "show version" | tail -1 | cut -f 2 | sed 's/^\([a-zA-Z # Initialize defaults SPARK_HADOOP_VERSION=1.0.4 -SPARK_WITH_YARN=false +SPARK_YARN=false MAKE_TGZ=false # Parse arguments @@ -58,7 +58,7 @@ while (( "$#" )); do shift ;; --with-yarn) - SPARK_WITH_YARN=true + SPARK_YARN=true ;; --tgz) MAKE_TGZ=true @@ -74,7 +74,7 @@ else fi echo "Hadoop version set to $SPARK_HADOOP_VERSION" -if [ "$SPARK_WITH_YARN" == "true" ]; then +if [ "$SPARK_YARN" == "true" ]; then echo "YARN enabled" else echo "YARN disabled" @@ -82,21 +82,22 @@ fi # Build fat JAR export SPARK_HADOOP_VERSION -export SPARK_WITH_YARN -"$FWDIR/sbt/sbt" "repl/assembly" +export SPARK_YARN +"$FWDIR/sbt/sbt" "assembly/assembly" # Make directories rm -rf "$DISTDIR" mkdir -p "$DISTDIR/jars" -echo "$VERSION" > "$DISTDIR/RELEASE" +echo "Spark $VERSION built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE" # Copy jars -cp $FWDIR/repl/target/*.jar "$DISTDIR/jars/" +cp $FWDIR/assembly/target/*/*assembly*.jar "$DISTDIR/jars/" # Copy other things cp -r "$FWDIR/bin" "$DISTDIR" cp -r "$FWDIR/conf" "$DISTDIR" -cp "$FWDIR/run" "$FWDIR/spark-shell" "$DISTDIR" +cp "$FWDIR/spark-class" "$DISTDIR" +cp "$FWDIR/spark-shell" "$DISTDIR" cp "$FWDIR/spark-executor" "$DISTDIR" diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5fdcf19b624..b3bf3ef89b0 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -26,30 +26,35 @@ import AssemblyKeys._ object SparkBuild extends Build { // Hadoop version to build against. For example, "1.0.4" for Apache releases, or // "2.0.0-mr1-cdh4.2.0" for Cloudera Hadoop. Note that these variables can be set - // through the environment variables SPARK_HADOOP_VERSION and SPARK_WITH_YARN. + // through the environment variables SPARK_HADOOP_VERSION and SPARK_YARN. val DEFAULT_HADOOP_VERSION = "1.0.4" - val DEFAULT_WITH_YARN = false + val DEFAULT_YARN = false // HBase version; set as appropriate. val HBASE_VERSION = "0.94.6" - lazy val root = Project("root", file("."), settings = rootSettings) aggregate(allProjects:_*) + lazy val root = Project("root", file("."), settings = rootSettings) aggregate(allProjects: _*) lazy val core = Project("core", file("core"), settings = coreSettings) - lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn(core) dependsOn(bagel) dependsOn(mllib) dependsOn(maybeYarn:_*) + lazy val repl = Project("repl", file("repl"), settings = replSettings) + .dependsOn(core, bagel, mllib) dependsOn(maybeYarn: _*) - lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) dependsOn (streaming) dependsOn(mllib) + lazy val examples = Project("examples", file("examples"), settings = examplesSettings) + .dependsOn(core, mllib, bagel, streaming) - lazy val tools = Project("tools", file("tools"), settings = examplesSettings) dependsOn (core) dependsOn (streaming) + lazy val tools = Project("tools", file("tools"), settings = toolsSettings) dependsOn(core) dependsOn(streaming) - lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn (core) + lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn(core) - lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn (core) + lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn(core) - lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn (core) + lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn(core) - lazy val yarn = Project("yarn", file("yarn"), settings = yarnSettings) dependsOn (core) + lazy val yarn = Project("yarn", file("yarn"), settings = yarnSettings) dependsOn(core) + + lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings) + .dependsOn(core, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*) // A configuration to set an alternative publishLocalConfiguration lazy val MavenCompile = config("m2r") extend(Compile) @@ -57,15 +62,16 @@ object SparkBuild extends Build { // Allows build configuration to be set through environment variables lazy val hadoopVersion = scala.util.Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION) - lazy val isYarnEnabled = scala.util.Properties.envOrNone("SPARK_WITH_YARN") match { - case None => DEFAULT_WITH_YARN + lazy val isYarnEnabled = scala.util.Properties.envOrNone("SPARK_YARN") match { + case None => DEFAULT_YARN case Some(v) => v.toBoolean } // Conditionally include the yarn sub-project lazy val maybeYarn = if(isYarnEnabled) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]() lazy val maybeYarnRef = if(isYarnEnabled) Seq[ProjectReference](yarn) else Seq[ProjectReference]() - lazy val allProjects = Seq[ProjectReference](core, repl, examples, bagel, streaming, mllib, tools) ++ maybeYarnRef + lazy val allProjects = Seq[ProjectReference]( + core, repl, examples, bagel, streaming, mllib, tools, assemblyProj) ++ maybeYarnRef def sharedSettings = Defaults.defaultSettings ++ Seq( organization := "org.spark-project", @@ -100,8 +106,8 @@ object SparkBuild extends Build { http://spark-project.org/ - BSD License - https://github.com/mesos/spark/blob/master/LICENSE + Apache 2.0 License + http://www.apache.org/licenses/LICENSE-2.0.html repo @@ -195,7 +201,7 @@ object SparkBuild extends Build { "com.twitter" % "chill_2.9.3" % "0.3.1", "com.twitter" % "chill-java" % "0.3.1" ) - ) ++ assemblySettings ++ extraAssemblySettings + ) def rootSettings = sharedSettings ++ Seq( publish := {} @@ -204,7 +210,7 @@ object SparkBuild extends Build { def replSettings = sharedSettings ++ Seq( name := "spark-repl", libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _) - ) ++ assemblySettings ++ extraAssemblySettings + ) def examplesSettings = sharedSettings ++ Seq( name := "spark-examples", @@ -223,7 +229,7 @@ object SparkBuild extends Build { exclude("org.apache.cassandra.deps", "avro") excludeAll(excludeSnappy) ) - ) + ) ++ assemblySettings ++ extraAssemblySettings def toolsSettings = sharedSettings ++ Seq( name := "spark-tools" @@ -251,7 +257,7 @@ object SparkBuild extends Build { "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty), "com.typesafe.akka" % "akka-zeromq" % "2.0.5" excludeAll(excludeNetty) ) - ) ++ assemblySettings ++ extraAssemblySettings + ) def yarnSettings = sharedSettings ++ Seq( name := "spark-yarn" @@ -271,7 +277,13 @@ object SparkBuild extends Build { ) ) - def extraAssemblySettings() = Seq(test in assembly := {}) ++ Seq( + def assemblyProjSettings = sharedSettings ++ Seq( + name := "spark-assembly", + jarName in assembly <<= version map { v => "spark-assembly-" + v + "-hadoop" + hadoopVersion + ".jar" } + ) ++ assemblySettings ++ extraAssemblySettings + + def extraAssemblySettings() = Seq( + test in assembly := {}, mergeStrategy in assembly := { case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard case m if m.toLowerCase.matches("meta-inf.*\\.sf$") => MergeStrategy.discard diff --git a/project/build.properties b/project/build.properties index 08e17131f64..96472771622 100644 --- a/project/build.properties +++ b/project/build.properties @@ -15,4 +15,4 @@ # limitations under the License. # -sbt.version=0.12.3 +sbt.version=0.12.4 diff --git a/project/plugins.sbt b/project/plugins.sbt index 783b40d4f5d..cfcd85082a8 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -4,7 +4,7 @@ resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/release resolvers += "Spray Repository" at "http://repo.spray.cc/" -addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.5") +addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.9.1") addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.2.0") diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index e503fb7621e..18011c0dc9c 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -28,7 +28,7 @@ def launch_gateway(): # Launch the Py4j gateway using Spark's run command so that we pick up the # proper classpath and SPARK_MEM settings from spark-env.sh - command = [os.path.join(SPARK_HOME, "run"), "py4j.GatewayServer", + command = [os.path.join(SPARK_HOME, "spark-class"), "py4j.GatewayServer", "--die-on-broken-pipe", "0"] proc = Popen(command, stdout=PIPE, stdin=PIPE) # Determine which ephemeral port the server started on: diff --git a/run-example b/run-example new file mode 100755 index 00000000000..e1b26257e14 --- /dev/null +++ b/run-example @@ -0,0 +1,76 @@ +#!/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. +# + +SCALA_VERSION=2.9.3 + +# Figure out where the Scala framework is installed +FWDIR="$(cd `dirname $0`; pwd)" + +# Export this as SPARK_HOME +export SPARK_HOME="$FWDIR" + +# 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 + +if [ -z "$1" ]; then + echo "Usage: run-example []" >&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. +EXAMPLES_DIR="$FWDIR"/examples +SPARK_EXAMPLES_JAR="" +if [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9T].jar ]; then + # Use the JAR from the SBT build + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9T].jar` +fi +if [ -e "$EXAMPLES_DIR"/target/spark-examples*[0-9T].jar ]; then + # Use the JAR from the Maven build + # TODO: this also needs to become an assembly! + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/spark-examples*[0-9T].jar` +fi +if [[ -z $SPARK_EXAMPLES_JAR ]]; then + echo "Failed to find Spark examples assembly in $FWDIR/examples/target" >&2 + echo "You need to compile Spark before running this program" >&2 + exit 1 +fi + +# Find java binary +if [ -n "${JAVA_HOME}" ]; then + RUNNER="${JAVA_HOME}/bin/java" +else + if [ `command -v java` ]; then + RUNNER="java" + else + echo "JAVA_HOME is not set" >&2 + exit 1 + fi +fi + +if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then + echo -n "Spark Command: " + echo "$RUNNER" -cp "$SPARK_EXAMPLES_JAR" "$@" + echo "========================================" + echo +fi + +exec "$RUNNER" -cp "$SPARK_EXAMPLES_JAR" "$@" diff --git a/run b/spark-class similarity index 52% rename from run rename to spark-class index 715bbf93d55..5ef3de97737 100755 --- a/run +++ b/spark-class @@ -66,51 +66,27 @@ 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. -# We still fall back on java for the shell if this is a "release" created -# from make-distribution.sh since it's possible scala is not installed -# but we have everything we need to run the shell. -if [[ "$SPARK_LAUNCH_WITH_SCALA" == "1" && ! -f "$FWDIR/RELEASE" ]]; 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 +# Find the java binary +if [ -n "${JAVA_HOME}" ]; then + RUNNER="${JAVA_HOME}/bin/java" else - if [ -n "${JAVA_HOME}" ]; then - RUNNER="${JAVA_HOME}/bin/java" + if [ `command -v java` ]; then + RUNNER="java" else - if [ `command -v java` ]; then - RUNNER="java" - else - echo "JAVA_HOME is not set" >&2 - exit 1 - fi + echo "JAVA_HOME is not set" >&2 + exit 1 fi - if [[ ! -f "$FWDIR/RELEASE" && -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 +if [[ ! -f "$FWDIR/RELEASE" && -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 -# 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 - SPARK_MEM="512m" -fi +# Set SPARK_MEM if it isn't already set since we also use it for this process +SPARK_MEM=${SPARK_MEM:-512m} export SPARK_MEM # Set JAVA_OPTS to be able to load native libraries and to set heap size @@ -125,51 +101,24 @@ export JAVA_OPTS # Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! if [ ! -f "$FWDIR/RELEASE" ]; then - CORE_DIR="$FWDIR/core" - EXAMPLES_DIR="$FWDIR/examples" - REPL_DIR="$FWDIR/repl" - # 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 + ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*.jar >& /dev/null + if [[ $? != 0 ]]; then + echo "Failed to find Spark assembly in $FWDIR/assembly/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 - - # 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 CLASSPATH=`$FWDIR/bin/compute-classpath.sh` export CLASSPATH -if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then - EXTRA_ARGS="" # Java options will be passed to scala as JAVA_OPTS -else - # The JVM doesn't read JAVA_OPTS by default so we need to pass it in - EXTRA_ARGS="$JAVA_OPTS" -fi - -command="$RUNNER -cp \"$CLASSPATH\" $EXTRA_ARGS $@" if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then - echo "Spark Command: $command" + echo -n "Spark Command: " + echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" echo "========================================" echo fi -exec "$RUNNER" -cp "$CLASSPATH" $EXTRA_ARGS "$@" +exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" diff --git a/spark-executor b/spark-executor index feccbf5cc2a..63692bd46c9 100755 --- a/spark-executor +++ b/spark-executor @@ -19,4 +19,4 @@ FWDIR="`dirname $0`" echo "Running spark-executor with framework dir = $FWDIR" -exec $FWDIR/run spark.executor.MesosExecutorBackend +exec $FWDIR/spark-class spark.executor.MesosExecutorBackend diff --git a/spark-shell b/spark-shell index 62fc18550df..4d379c5cfb9 100755 --- a/spark-shell +++ b/spark-shell @@ -79,8 +79,7 @@ if [[ ! $? ]]; then saved_stty="" fi -export SPARK_LAUNCH_WITH_SCALA=${SPARK_LAUNCH_WITH_SCALA:-1} -$FWDIR/run $OPTIONS spark.repl.Main "$@" +$FWDIR/spark-class $OPTIONS spark.repl.Main "$@" # record the exit status lest it be overwritten: # then reenable echo and propagate the code. From ab0e625d9e0abd62a20754125952e3a00f2c275a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 22 Aug 2013 23:02:09 -0700 Subject: [PATCH 0868/2521] Fix PySpark for assembly run and include it in dist --- .gitignore | 1 + core/lib/PY4J_LICENSE.txt | 27 +++++++++++++++++++++++++++ core/lib/PY4J_VERSION.txt | 1 + {python => core}/lib/py4j0.7.jar | Bin make-distribution.sh | 5 ++++- pyspark | 12 ++++++++---- 6 files changed, 41 insertions(+), 5 deletions(-) create mode 100644 core/lib/PY4J_LICENSE.txt create mode 100644 core/lib/PY4J_VERSION.txt rename {python => core}/lib/py4j0.7.jar (100%) diff --git a/.gitignore b/.gitignore index 00fbff6a2ca..e1f64a11339 100644 --- a/.gitignore +++ b/.gitignore @@ -40,3 +40,4 @@ checkpoint derby.log dist/ spark-*-bin.tar.gz +unit-tests.log diff --git a/core/lib/PY4J_LICENSE.txt b/core/lib/PY4J_LICENSE.txt new file mode 100644 index 00000000000..a70279ca14a --- /dev/null +++ b/core/lib/PY4J_LICENSE.txt @@ -0,0 +1,27 @@ + +Copyright (c) 2009-2011, Barthelemy Dagenais 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. + +- The name of the author may not 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. diff --git a/core/lib/PY4J_VERSION.txt b/core/lib/PY4J_VERSION.txt new file mode 100644 index 00000000000..04a0cd52a8d --- /dev/null +++ b/core/lib/PY4J_VERSION.txt @@ -0,0 +1 @@ +b7924aabe9c5e63f0a4d8bbd17019534c7ec014e diff --git a/python/lib/py4j0.7.jar b/core/lib/py4j0.7.jar similarity index 100% rename from python/lib/py4j0.7.jar rename to core/lib/py4j0.7.jar diff --git a/make-distribution.sh b/make-distribution.sh index df7bbf1e748..92b2706126c 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -94,11 +94,14 @@ echo "Spark $VERSION built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE cp $FWDIR/assembly/target/*/*assembly*.jar "$DISTDIR/jars/" # Copy other things +mkdir "$DISTDIR"/conf +cp -r "$FWDIR/conf/*.template" "$DISTDIR" cp -r "$FWDIR/bin" "$DISTDIR" -cp -r "$FWDIR/conf" "$DISTDIR" +cp -r "$FWDIR/python" "$DISTDIR" cp "$FWDIR/spark-class" "$DISTDIR" cp "$FWDIR/spark-shell" "$DISTDIR" cp "$FWDIR/spark-executor" "$DISTDIR" +cp "$FWDIR/pyspark" "$DISTDIR" if [ "$MAKE_TGZ" == "true" ]; then diff --git a/pyspark b/pyspark index 801239c108b..155ccd4fdfa 100755 --- a/pyspark +++ b/pyspark @@ -24,10 +24,14 @@ FWDIR="$(cd `dirname $0`; pwd)" export SPARK_HOME="$FWDIR" # Exit if the user hasn't compiled Spark -if [ ! -e "$SPARK_HOME/repl/target" ]; then - echo "Failed to find Spark classes in $SPARK_HOME/repl/target" >&2 - echo "You need to compile Spark before running this program" >&2 - exit 1 +if [ ! -f "$FWDIR/RELEASE" ]; then + # Exit if the user hasn't compiled Spark + ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*.jar >& /dev/null + if [[ $? != 0 ]]; then + echo "Failed to find Spark assembly in $FWDIR/assembly/target" >&2 + echo "You need to compile Spark before running this program" >&2 + exit 1 + fi fi # Load environment variables from conf/spark-env.sh, if it exists From 8d81358a05e885a76a27475075a46973c7540a0c Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 23 Aug 2013 23:25:49 -0700 Subject: [PATCH 0869/2521] Provide more memory for tests --- pom.xml | 2 +- project/SparkBuild.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 85bcd8696c2..ea9548359f6 100644 --- a/pom.xml +++ b/pom.xml @@ -492,7 +492,7 @@ ${project.build.directory}/surefire-reports . ${project.build.directory}/SparkTestSuite.txt - -Xms64m -Xmx1024m + -Xms64m -Xmx3g diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b3bf3ef89b0..8797e65b8d4 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -86,7 +86,7 @@ object SparkBuild extends Build { // Fork new JVMs for tests and set Java options for those fork := true, - javaOptions += "-Xmx2500m", + javaOptions += "-Xmx3g", // 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), From aab345c46318264a58ee86704567a2586de32b13 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 27 Aug 2013 11:37:00 -0700 Subject: [PATCH 0870/2521] Fix finding of assembly JAR, as well as some pointers to ./run --- bin/compute-classpath.sh | 2 +- .../src/main/scala/spark/deploy/worker/ExecutorRunner.scala | 1 + .../spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala | 2 +- core/src/main/scala/spark/ui/UIWorkloadGenerator.scala | 2 +- .../scala/spark/streaming/examples/ActorWordCount.scala | 4 ++-- .../main/scala/spark/streaming/examples/HdfsWordCount.scala | 2 +- .../scala/spark/streaming/examples/KafkaWordCount.scala | 2 +- .../scala/spark/streaming/examples/NetworkWordCount.scala | 2 +- .../spark/streaming/examples/StatefulNetworkWordCount.scala | 2 +- .../scala/spark/streaming/examples/ZeroMQWordCount.scala | 4 ++-- .../streaming/examples/clickstream/PageViewGenerator.scala | 4 ++-- .../streaming/examples/clickstream/PageViewStream.scala | 6 +++--- sbt/sbt | 2 +- 13 files changed, 18 insertions(+), 17 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 5dc86c51a4a..c7819d49324 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -35,7 +35,7 @@ CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" if [ -f "$FWDIR/RELEASE" ]; then ASSEMBLY_JAR=`ls "$FWDIR"/jars/spark-assembly*.jar` else - ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*.jar` + ASSEMBLY_JAR=`ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar` fi CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala index 5e53d95ac2f..34665ce451a 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala @@ -153,6 +153,7 @@ private[spark] class ExecutorRunner( // Launch the process val command = buildCommandSeq() + logInfo("Launch 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/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index ebfc21392dd..eef3ee14251 100644 --- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -132,7 +132,7 @@ private[spark] class CoarseMesosSchedulerBackend( // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". val basename = uri.split('/').last.split('.').head - command.setValue("cd %s*; ./run spark.executor.StandaloneExecutorBackend %s %s %s %d".format( + command.setValue("cd %s*; ./spark-class spark.executor.StandaloneExecutorBackend %s %s %s %d".format( basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala index f96419520fb..5ff0572f0a3 100644 --- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala @@ -35,7 +35,7 @@ private[spark] object UIWorkloadGenerator { def main(args: Array[String]) { if (args.length < 2) { - println("usage: ./run spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]") + println("usage: ./spark-class spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]") System.exit(1) } val master = args(0) diff --git a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala index f97174aeaea..05d31764781 100644 --- a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala @@ -132,9 +132,9 @@ object FeederActor { * and describe the AkkaSystem that Spark Sample feeder is running on. * * To run this example locally, you may run Feeder Actor as - * `$ ./run spark.streaming.examples.FeederActor 127.0.1.1 9999` + * `$ ./run-example spark.streaming.examples.FeederActor 127.0.1.1 9999` * and then run the example - * `$ ./run spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999` + * `$ ./run-example spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999` */ object ActorWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala index f5baec242da..30af01a26f4 100644 --- a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala @@ -28,7 +28,7 @@ import spark.streaming.StreamingContext._ * is the directory that Spark Streaming will use to find and read new text files. * * To run this on your local machine on directory `localdir`, run this example - * `$ ./run spark.streaming.examples.HdfsWordCount local[2] localdir` + * `$ ./run-example spark.streaming.examples.HdfsWordCount local[2] localdir` * Then create a text file in `localdir` and the words in the file will get counted. */ object HdfsWordCount { diff --git a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala index 4929703ba2f..d9c76d1a33b 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 @@ import spark.streaming.util.RawTextHelper._ * is the number of threads the kafka consumer should use * * Example: - * `./run spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1` + * `./run-example spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1` */ object KafkaWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala index 150fb5eb9c3..b29d79aac53 100644 --- a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala @@ -29,7 +29,7 @@ import spark.streaming.StreamingContext._ * 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.NetworkWordCount local[2] localhost 9999` + * `$ ./run-example spark.streaming.examples.NetworkWordCount local[2] localhost 9999` */ object NetworkWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala index 33ab324732c..b709fc3c87c 100644 --- a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -29,7 +29,7 @@ import spark.streaming.StreamingContext._ * 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` + * `$ ./run-example spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` */ object StatefulNetworkWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala index e264fae6097..a0cae06c309 100644 --- a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala @@ -60,9 +60,9 @@ object SimpleZeroMQPublisher { * and describe where zeroMq publisher is running. * * To run this example locally, you may run publisher as - * `$ ./run spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` + * `$ ./run-example spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` * and run the example as - * `$ ./run spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo` + * `$ ./run-example spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo` */ object ZeroMQWordCount { def main(args: Array[String]) { 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 375d5c9d220..dd36bbbf320 100644 --- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala @@ -37,8 +37,8 @@ object PageView { /** Generates streaming events to simulate page views on a website. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./run spark.streaming.examples.clickstream.PageViewGenerator 44444 10 - * $ ./run spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 + * $ ./run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10 + * $ ./run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 * */ object PageViewGenerator { val pages = Map("http://foo.com/" -> .7, 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 a24342bebf9..152da234898 100644 --- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala @@ -25,9 +25,9 @@ import spark.SparkContext._ * operators available in Spark streaming. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./run spark.streaming.examples.clickstream.PageViewGenerator 44444 10 - * $ ./run spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 - * */ + * $ ./run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10 + * $ ./run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 + */ object PageViewStream { def main(args: Array[String]) { if (args.length != 3) { diff --git a/sbt/sbt b/sbt/sbt index 2227bc46969..c31a0280ff9 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -25,4 +25,4 @@ fi 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=256m $EXTRA_ARGS $SBT_OPTS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@" +java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=256m $EXTRA_ARGS $SBT_OPTS -jar "$SPARK_HOME"/sbt/sbt-launch-*.jar "$@" From 3ff105f87db217c0c0af0f176626a416b2669740 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 27 Aug 2013 15:46:23 -0700 Subject: [PATCH 0871/2521] Find assembly correctly in pyspark --- pyspark | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pyspark b/pyspark index 155ccd4fdfa..aedc51e0483 100755 --- a/pyspark +++ b/pyspark @@ -23,10 +23,12 @@ FWDIR="$(cd `dirname $0`; pwd)" # Export this as SPARK_HOME export SPARK_HOME="$FWDIR" +SCALA_VERSION=2.9.3 + # Exit if the user hasn't compiled Spark if [ ! -f "$FWDIR/RELEASE" ]; then # Exit if the user hasn't compiled Spark - ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*.jar >& /dev/null + ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null if [[ $? != 0 ]]; then echo "Failed to find Spark assembly in $FWDIR/assembly/target" >&2 echo "You need to compile Spark before running this program" >&2 From d7dec938e503b86d1b338c4df3439d3649a76294 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 27 Aug 2013 15:52:03 -0700 Subject: [PATCH 0872/2521] Don't use SPARK_LAUNCH_WITH_SCALA in pyspark --- pyspark | 5 ----- 1 file changed, 5 deletions(-) diff --git a/pyspark b/pyspark index aedc51e0483..2dba2ceb21c 100755 --- a/pyspark +++ b/pyspark @@ -54,11 +54,6 @@ export PYTHONPATH=$SPARK_HOME/python/:$PYTHONPATH export OLD_PYTHONSTARTUP=$PYTHONSTARTUP export PYTHONSTARTUP=$FWDIR/python/pyspark/shell.py -# Launch with `scala` by default: -if [[ "$SPARK_LAUNCH_WITH_SCALA" != "0" ]] ; then - export SPARK_LAUNCH_WITH_SCALA=1 -fi - if [ -n "$IPYTHON_OPTS" ]; then IPYTHON=1 fi From 666d93c294458cb056cb590eb11bb6cf979861e5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 27 Aug 2013 19:23:54 -0700 Subject: [PATCH 0873/2521] Update Maven build to create assemblies expected by new scripts This includes the following changes: - The "assembly" package now builds in Maven by default, and creates an assembly containing both hadoop-client and Spark, unlike the old BigTop distribution assembly that skipped hadoop-client - There is now a bigtop-dist package to build the old BigTop assembly - The repl-bin package is no longer built by default since the scripts don't reply on it; instead it can be enabled with -Prepl-bin - Py4J is now included in the assembly/lib folder as a local Maven repo, so that the Maven package can link to it - run-example now adds the original Spark classpath as well because the Maven examples assembly lists spark-core and such as provided - The various Maven projects add a spark-yarn dependency correctly --- {core => assembly}/lib/PY4J_LICENSE.txt | 0 {core => assembly}/lib/PY4J_VERSION.txt | 0 .../lib/net/sf/py4j/py4j/0.7/py4j-0.7.jar | Bin .../lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom | 9 ++ .../net/sf/py4j/py4j/maven-metadata-local.xml | 12 ++ assembly/pom.xml | 138 +++++++++++++++--- assembly/src/main/assembly/assembly.xml | 19 ++- examples/pom.xml | 62 +++++++- pom.xml | 16 +- project/SparkBuild.scala | 4 +- run-example | 9 +- 11 files changed, 222 insertions(+), 47 deletions(-) rename {core => assembly}/lib/PY4J_LICENSE.txt (100%) rename {core => assembly}/lib/PY4J_VERSION.txt (100%) rename core/lib/py4j0.7.jar => assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.jar (100%) create mode 100644 assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom create mode 100644 assembly/lib/net/sf/py4j/py4j/maven-metadata-local.xml diff --git a/core/lib/PY4J_LICENSE.txt b/assembly/lib/PY4J_LICENSE.txt similarity index 100% rename from core/lib/PY4J_LICENSE.txt rename to assembly/lib/PY4J_LICENSE.txt diff --git a/core/lib/PY4J_VERSION.txt b/assembly/lib/PY4J_VERSION.txt similarity index 100% rename from core/lib/PY4J_VERSION.txt rename to assembly/lib/PY4J_VERSION.txt diff --git a/core/lib/py4j0.7.jar b/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.jar similarity index 100% rename from core/lib/py4j0.7.jar rename to assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.jar diff --git a/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom b/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom new file mode 100644 index 00000000000..1c730e19b4b --- /dev/null +++ b/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom @@ -0,0 +1,9 @@ + + + 4.0.0 + net.sf.py4j + py4j + 0.7 + POM was created from install:install-file + diff --git a/assembly/lib/net/sf/py4j/py4j/maven-metadata-local.xml b/assembly/lib/net/sf/py4j/py4j/maven-metadata-local.xml new file mode 100644 index 00000000000..6942ff45e75 --- /dev/null +++ b/assembly/lib/net/sf/py4j/py4j/maven-metadata-local.xml @@ -0,0 +1,12 @@ + + + net.sf.py4j + py4j + + 0.7 + + 0.7 + + 20130828020333 + + diff --git a/assembly/pom.xml b/assembly/pom.xml index ca20ccadba8..74990b63612 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -1,4 +1,21 @@ + + 4.0.0 @@ -13,29 +30,13 @@ Spark Project Assembly http://spark-project.org/ - - - - org.apache.maven.plugins - maven-assembly-plugin - 2.4 - - - dist - package - - single - - - - src/main/assembly/assembly.xml - - - - - - - + + + + lib + file://${project.basedir}/lib + + @@ -63,5 +64,96 @@ spark-streaming ${project.version} + + net.sf.py4j + py4j + 0.7 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + false + ${project.build.directory}/scala-${scala.version}/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + package + + shade + + + + + + reference.conf + + + + + + + + + + + + hadoop2-yarn + + + org.spark-project + spark-yarn + ${project.version} + + + + + bigtop-dist + + + + + org.apache.maven.plugins + maven-assembly-plugin + 2.4 + + + dist + package + + single + + + + src/main/assembly/assembly.xml + + + + + + + + + diff --git a/assembly/src/main/assembly/assembly.xml b/assembly/src/main/assembly/assembly.xml index 14485b7181e..4543b52c931 100644 --- a/assembly/src/main/assembly/assembly.xml +++ b/assembly/src/main/assembly/assembly.xml @@ -1,3 +1,19 @@ + dist @@ -36,7 +52,8 @@ /bin - run* + run-example* + spark-class* spark-shell* spark-executor* diff --git a/examples/pom.xml b/examples/pom.xml index d24bd404fa2..687fbcca8f3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -36,21 +36,25 @@ org.spark-project spark-core ${project.version} + provided org.spark-project spark-streaming ${project.version} + provided org.spark-project spark-mllib ${project.version} + provided org.spark-project spark-bagel ${project.version} + provided org.apache.hbase @@ -67,10 +71,6 @@ - - org.scala-lang - scala-library - org.eclipse.jetty jetty-server @@ -126,13 +126,63 @@ + + + + hadoop2-yarn + + + org.spark-project + spark-yarn + ${project.version} + provided + + + + + target/scala-${scala.version}/classes target/scala-${scala.version}/test-classes - org.scalatest - scalatest-maven-plugin + org.apache.maven.plugins + maven-shade-plugin + + false + ${project.build.directory}/scala-${scala.version}/${project.artifactId}-assembly-${project.version}.jar + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + package + + shade + + + + + + reference.conf + + + + + diff --git a/pom.xml b/pom.xml index ea9548359f6..e2fd54a9660 100644 --- a/pom.xml +++ b/pom.xml @@ -62,6 +62,7 @@ tools streaming repl + assembly @@ -75,7 +76,7 @@ 1.7.2 1.2.17 1.0.4 - + 0.94.6 64m 512m @@ -743,21 +744,10 @@ - assembly + repl-bin false - - assembly - - - - expensive-modules - - - !noExpensive - - repl-bin diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 8797e65b8d4..2e268126713 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -41,7 +41,7 @@ object SparkBuild extends Build { .dependsOn(core, bagel, mllib) dependsOn(maybeYarn: _*) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) - .dependsOn(core, mllib, bagel, streaming) + .dependsOn(core, mllib, bagel, streaming) dependsOn(maybeYarn: _*) lazy val tools = Project("tools", file("tools"), settings = toolsSettings) dependsOn(core) dependsOn(streaming) @@ -261,7 +261,7 @@ object SparkBuild extends Build { def yarnSettings = sharedSettings ++ Seq( name := "spark-yarn" - ) ++ extraYarnSettings ++ assemblySettings ++ extraAssemblySettings + ) ++ extraYarnSettings // Conditionally include the YARN dependencies because some tools look at all sub-projects and will complain // if we refer to nonexistent dependencies (e.g. hadoop-yarn-api from a Hadoop version without YARN). diff --git a/run-example b/run-example index e1b26257e14..ccd4356bdf1 100755 --- a/run-example +++ b/run-example @@ -54,6 +54,11 @@ if [[ -z $SPARK_EXAMPLES_JAR ]]; then exit 1 fi +# Since the examples JAR ideally shouldn't include spark-core (that dependency should be +# "provided"), also add our standard Spark classpath, built using compute-classpath.sh. +CLASSPATH=`$FWDIR/bin/compute-classpath.sh` +CLASSPATH="$SPARK_EXAMPLES_JAR:$CLASSPATH" + # Find java binary if [ -n "${JAVA_HOME}" ]; then RUNNER="${JAVA_HOME}/bin/java" @@ -68,9 +73,9 @@ fi if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then echo -n "Spark Command: " - echo "$RUNNER" -cp "$SPARK_EXAMPLES_JAR" "$@" + echo "$RUNNER" -cp "$CLASSPATH" "$@" echo "========================================" echo fi -exec "$RUNNER" -cp "$SPARK_EXAMPLES_JAR" "$@" +exec "$RUNNER" -cp "$CLASSPATH" "$@" From 2de756ff195e580007d5d96d49fa27634e04c765 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 27 Aug 2013 19:39:54 -0700 Subject: [PATCH 0874/2521] Update some build instructions because only sbt assembly and mvn package are now needed --- README.md | 10 +++++----- docs/building-with-maven.md | 14 +++++++------- docs/index.md | 2 +- docs/python-programming-guide.md | 2 +- docs/quick-start.md | 2 +- 5 files changed, 15 insertions(+), 15 deletions(-) diff --git a/README.md b/README.md index e7af34f5131..89b5a0abfd7 100644 --- a/README.md +++ b/README.md @@ -16,7 +16,7 @@ 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: - sbt/sbt package assembly + sbt/sbt assembly Spark also supports building using Maven. If you would like to build using Maven, see the [instructions for building Spark with Maven](http://spark-project.org/docs/latest/building-with-maven.html) @@ -52,19 +52,19 @@ For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions without YARN, use: # Apache Hadoop 1.2.1 - $ SPARK_HADOOP_VERSION=1.2.1 sbt/sbt package assembly + $ SPARK_HADOOP_VERSION=1.2.1 sbt/sbt assembly # Cloudera CDH 4.2.0 with MapReduce v1 - $ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt/sbt package assembly + $ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt/sbt assembly For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, also set `SPARK_WITH_YARN=true`: # Apache Hadoop 2.0.5-alpha - $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_WITH_YARN=true sbt/sbt package assembly + $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_WITH_YARN=true sbt/sbt assembly # Cloudera CDH 4.2.0 with MapReduce v2 - $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_WITH_YARN=true sbt/sbt package assembly + $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_WITH_YARN=true sbt/sbt assembly For convenience, these variables may also be set through the `conf/spark-env.sh` file described below. diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index a9f2cb8a7ae..72d37fec0ae 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -15,18 +15,18 @@ To enable support for HDFS and other Hadoop-supported storage systems, specify t For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions without YARN, use: # Apache Hadoop 1.2.1 - $ mvn -Dhadoop.version=1.2.1 clean install + $ mvn -Dhadoop.version=1.2.1 clean package # Cloudera CDH 4.2.0 with MapReduce v1 - $ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 clean install + $ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 clean package For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, enable the "hadoop2-yarn" profile: # Apache Hadoop 2.0.5-alpha - $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.5-alpha clean install + $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.5-alpha clean package # Cloudera CDH 4.2.0 with MapReduce v2 - $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.0-cdh4.2.0 clean install + $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.0-cdh4.2.0 clean package ## Spark Tests in Maven ## @@ -35,7 +35,7 @@ Tests are run by default via the scalatest-maven-plugin. With this you can do th Skip test execution (but not compilation): - $ mvn -Dhadoop.version=... -DskipTests clean install + $ mvn -Dhadoop.version=... -DskipTests clean package To run a specific test suite: @@ -72,8 +72,8 @@ This setup works fine in IntelliJ IDEA 11.1.4. After opening the project via the ## Building Spark Debian Packages ## -It includes support for building a Debian package containing a 'fat-jar' which includes the repl, the examples and bagel. This can be created by specifying the deb profile: +It includes support for building a Debian package containing a 'fat-jar' which includes the repl, the examples and bagel. This can be created by specifying the following profiles: - $ mvn -Pdeb clean install + $ mvn -Prepl-bin -Pdeb clean package The debian package can then be found under repl/target. We added the short commit hash to the file name so that we can distinguish individual packages build for SNAPSHOT versions. diff --git a/docs/index.md b/docs/index.md index e51a6998f6c..ec9c7dd4f30 100644 --- a/docs/index.md +++ b/docs/index.md @@ -20,7 +20,7 @@ of these methods on slave nodes on your cluster. Spark uses [Simple Build Tool](http://www.scala-sbt.org), which is bundled with it. To compile the code, go into the top-level Spark directory and run - sbt/sbt package + sbt/sbt assembly Spark also supports building using Maven. If you would like to build using Maven, see the [instructions for building Spark with Maven](building-with-maven.html). diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 794bff56474..15d3ebfcae6 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -70,7 +70,7 @@ The script automatically adds the `pyspark` package to the `PYTHONPATH`. The `pyspark` script launches a Python interpreter that is configured to run PySpark jobs. To use `pyspark` interactively, first build Spark, then launch it directly from the command line without any options: {% highlight bash %} -$ sbt/sbt package +$ sbt/sbt assembly $ ./pyspark {% endhighlight %} diff --git a/docs/quick-start.md b/docs/quick-start.md index 335643536aa..4e9deadbaa8 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -12,7 +12,7 @@ See the [programming guide](scala-programming-guide.html) for a more complete re To follow along with this guide, you only need to have successfully built Spark on one machine. Simply go into your Spark directory and run: {% highlight bash %} -$ sbt/sbt package +$ sbt/sbt assembly {% endhighlight %} # Interactive Analysis with the Spark Shell From d8a4008685996756c3fc871a5a196a60d8e8989c Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 27 Aug 2013 19:44:59 -0700 Subject: [PATCH 0875/2521] Fix path to assembly in make-distribution.sh --- make-distribution.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/make-distribution.sh b/make-distribution.sh index 92b2706126c..91f62784912 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -91,7 +91,7 @@ mkdir -p "$DISTDIR/jars" echo "Spark $VERSION built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE" # Copy jars -cp $FWDIR/assembly/target/*/*assembly*.jar "$DISTDIR/jars/" +cp $FWDIR/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/jars/" # Copy other things mkdir "$DISTDIR"/conf From e11bc18294d9e3f2ea155f5398faf3fb08aa2a59 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 27 Aug 2013 20:02:30 -0700 Subject: [PATCH 0876/2521] Update Maven docs --- docs/building-with-maven.md | 55 ++++++++++++++++++------------------- 1 file changed, 26 insertions(+), 29 deletions(-) diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index 72d37fec0ae..7ecb601dddc 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -8,53 +8,51 @@ title: Building Spark with Maven Building Spark using Maven Requires Maven 3 (the build process is tested with Maven 3.0.4) and Java 1.6 or newer. -## Specifying the Hadoop version ## -To enable support for HDFS and other Hadoop-supported storage systems, specify the exact Hadoop version by setting the "hadoop.version" property. If unset, Spark will build against Hadoop 1.0.4 by default. +## Setting up Maven's Memory Usage ## -For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions without YARN, use: +You'll need to configure Maven to use more memory than usual by setting `MAVEN_OPTS`. We recommend the following settings: - # Apache Hadoop 1.2.1 - $ mvn -Dhadoop.version=1.2.1 clean package + export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" - # Cloudera CDH 4.2.0 with MapReduce v1 - $ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 clean package +If you don't run this, you may see errors like the following: -For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, enable the "hadoop2-yarn" profile: - - # Apache Hadoop 2.0.5-alpha - $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.5-alpha clean package + [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_VERSION}}/classes... + [ERROR] PermGen space -> [Help 1] - # Cloudera CDH 4.2.0 with MapReduce v2 - $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.0-cdh4.2.0 clean package + [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_VERSION}}/classes... + [ERROR] Java heap space -> [Help 1] +You can fix this by setting the `MAVEN_OPTS` variable as discussed before. -## Spark Tests in Maven ## +## Specifying the Hadoop version ## -Tests are run by default via the scalatest-maven-plugin. With this you can do things like: +Because HDFS is not protocol-compatible across versions, if you want to read from HDFS, you'll need to build Spark against the specific HDFS version in your environment. You can do this through the "hadoop.version" property. If unset, Spark will build against Hadoop 1.0.4 by default. -Skip test execution (but not compilation): +For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions without YARN, use: - $ mvn -Dhadoop.version=... -DskipTests clean package + # Apache Hadoop 1.2.1 + $ mvn -Dhadoop.version=1.2.1 -DskipTests clean package -To run a specific test suite: + # Cloudera CDH 4.2.0 with MapReduce v1 + $ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 -DskipTests clean package - $ mvn -Dhadoop.version=... -Dsuites=spark.repl.ReplSuite test +For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you should also enable the "hadoop2-yarn" profile: + # Apache Hadoop 2.0.5-alpha + $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.5-alpha -DskipTests clean package -## Setting up JVM Memory Usage Via Maven ## + # Cloudera CDH 4.2.0 with MapReduce v2 + $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.0-cdh4.2.0 -DskipTests clean package -You might run into the following errors if you're using a vanilla installation of Maven: - [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_VERSION}}/classes... - [ERROR] PermGen space -> [Help 1] +## Spark Tests in Maven ## - [INFO] Compiling 203 Scala sources and 9 Java sources to /Users/me/Development/spark/core/target/scala-{{site.SCALA_VERSION}}/classes... - [ERROR] Java heap space -> [Help 1] +Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). Some of the require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. You can then run the tests with `mvn -Dhadoop.version=... test`. -To fix these, you can do the following: +The ScalaTest plugin also supports running only a specific test suite as follows: - export MAVEN_OPTS="-Xmx1024m -XX:MaxPermSize=128M" + $ mvn -Dhadoop.version=... -Dsuites=spark.repl.ReplSuite test ## Continuous Compilation ## @@ -63,8 +61,7 @@ We use the scala-maven-plugin which supports incremental and continuous compilat $ mvn scala:cc -…should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively. - +should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively. ## Using With IntelliJ IDEA ## From adc700582b52c730a4dbf2a45d4cc794be43cfe5 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Fri, 30 Aug 2013 00:16:32 -0700 Subject: [PATCH 0877/2521] Fix broken build by removing addIntercept --- .../spark/mllib/regression/LinearRegression.scala | 9 ++++----- .../scala/spark/mllib/regression/RidgeRegression.scala | 10 ++++------ 2 files changed, 8 insertions(+), 11 deletions(-) diff --git a/mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala index 5b3743f2fa1..885ff5a30d4 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala @@ -47,8 +47,7 @@ class LinearRegressionModel( class LinearRegressionWithSGD private ( var stepSize: Double, var numIterations: Int, - var miniBatchFraction: Double, - var addIntercept: Boolean) + var miniBatchFraction: Double) extends GeneralizedLinearAlgorithm[LinearRegressionModel] with Serializable { @@ -61,7 +60,7 @@ class LinearRegressionWithSGD private ( /** * Construct a LinearRegression object with default parameters */ - def this() = this(1.0, 100, 1.0, true) + def this() = this(1.0, 100, 1.0) def createModel(weights: Array[Double], intercept: Double) = { new LinearRegressionModel(weights, intercept) @@ -94,7 +93,7 @@ object LinearRegressionWithSGD { initialWeights: Array[Double]) : LinearRegressionModel = { - new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction, true).run(input, + new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction).run(input, initialWeights) } @@ -115,7 +114,7 @@ object LinearRegressionWithSGD { miniBatchFraction: Double) : LinearRegressionModel = { - new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction, true).run(input) + new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction).run(input) } /** diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala index ccf73648061..cb1303dd99e 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala @@ -48,8 +48,7 @@ class RidgeRegressionWithSGD private ( var stepSize: Double, var numIterations: Int, var regParam: Double, - var miniBatchFraction: Double, - var addIntercept: Boolean) + var miniBatchFraction: Double) extends GeneralizedLinearAlgorithm[RidgeRegressionModel] with Serializable { @@ -71,7 +70,7 @@ class RidgeRegressionWithSGD private ( /** * Construct a RidgeRegression object with default parameters */ - def this() = this(1.0, 100, 1.0, 1.0, true) + def this() = this(1.0, 100, 1.0, 1.0) def createModel(weights: Array[Double], intercept: Double) = { val weightsMat = new DoubleMatrix(weights.length + 1, 1, (Array(intercept) ++ weights):_*) @@ -134,7 +133,7 @@ object RidgeRegressionWithSGD { initialWeights: Array[Double]) : RidgeRegressionModel = { - new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction, true).run( + new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run( input, initialWeights) } @@ -157,8 +156,7 @@ object RidgeRegressionWithSGD { miniBatchFraction: Double) : RidgeRegressionModel = { - new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction, true).run( - input) + new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input) } /** From 1b0f69c6234bc97d331cf4b1785078789cb69b47 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 30 Aug 2013 10:15:58 -0700 Subject: [PATCH 0878/2521] Change docs color theme for 0.8 --- docs/css/bootstrap.min.css | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/css/bootstrap.min.css b/docs/css/bootstrap.min.css index eb48138e08d..cf4c519cdc7 100644 --- a/docs/css/bootstrap.min.css +++ b/docs/css/bootstrap.min.css @@ -6,4 +6,4 @@ * 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{height:auto;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;line-height:0;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:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-static-top .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;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-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.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:20px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:1;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1{font-size:36px;line-height:40px}h2{font-size:30px;line-height:40px}h3{font-size:24px;line-height:40px}h4{font-size:18px;line-height:20px}h5{font-size:14px;line-height:20px}h6{font-size:12px;line-height:20px}h1 small{font-size:24px}h2 small{font-size:18px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.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:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}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 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:25px}blockquote small{display:block;line-height:20px;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}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,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:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;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:20px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}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:20px;padding:4px 6px;margin-bottom:9px;font-size:14px;line-height:20px;color:#555;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}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-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;-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:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal;cursor:pointer}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;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}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-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)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.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[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .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}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"]{float:left}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}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;-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)}.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:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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;-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)}.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:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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;-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)}.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:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{margin-bottom:5px;font-size:0;white-space:nowrap}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;font-size:14px;vertical-align:top;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn{margin-left:-1px;vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .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 .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}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 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;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{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:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;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:10px;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:20px}.table th,.table td{padding:8px;line-height:20px;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:collapse;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,.table-bordered tfoot: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,.table-bordered tfoot: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-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-right-topleft:4px}.table-striped tbody tr:nth-child(odd) td,.table-striped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table-hover tbody tr:hover td,.table-hover tbody tr:hover th{background-color:#f5f5f5}table [class*=span],.row-fluid table [class*=span]{display:table-cell;float:none;margin-left:0}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}.table tbody tr.success td{background-color:#dff0d8}.table tbody tr.error td{background-color:#f2dede}.table tbody tr.info td{background-color:#d9edf7}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*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}.icon-white,.nav>.active>a>[class^="icon-"],.nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"]{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{width:16px;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{width:16px;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:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 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:6px;-moz-border-radius:6px;border-radius:6px;-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:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu li>a:focus,.dropdown-submenu:hover>a{color:#fff;text-decoration:none;background-color:#0098cc;background-color:#0098cc;background-image:-moz-linear-gradient(top,#0098cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0098cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0098cc,#0087b3);background-image:-o-linear-gradient(top,#0098cc,#0087b3);background-image:linear-gradient(to bottom,#0098cc,#0087b3);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0098cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#0098cc;background-color:#0081c2;background-image:linear-gradient(to bottom,#0098cc,#0087b3);background-image:-moz-linear-gradient(top,#0098cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0098cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0098cc,#0087b3);background-image:-o-linear-gradient(top,#0098cc,#0087b3);background-repeat:repeat-x;outline:0;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0098cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .disabled>a,.dropdown-menu .disabled>a:hover{color:#999}.dropdown-menu .disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.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}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover .dropdown-menu{display:block}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.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 #e3e3e3;-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;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;overflow:visible \9;-webkit-transition:height .35s ease;-moz-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:20px;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 14px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;*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:-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(to bottom,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #bbb;*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:#a2a2a2;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',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]{color:#333;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;-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:16px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:2px}.btn-small{padding:3px 9px;font-size:12px;line-height:18px}.btn-small [class^="icon-"]{margin-top:0}.btn-mini{padding:2px 6px;font-size:11px;line-height:16px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}.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:#c5c5c5;border-color:rgba(0,0,0,0.15) rgba(0,0,0,0.15) rgba(0,0,0,0.25)}.btn-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-image:-moz-linear-gradient(top,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;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='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#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(to bottom,#fbb450,#f89406);background-image:-moz-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='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#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:linear-gradient(to bottom,#ee5f5b,#bd362f);background-image:-moz-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='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#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:linear-gradient(to bottom,#62c462,#51a351);background-image:-moz-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='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#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:linear-gradient(to bottom,#5bc0de,#2f96b4);background-image:-moz-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='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-image:-moz-linear-gradient(top,#444,#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='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}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-link,.btn-link:active{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover{color:#005580;text-decoration:underline;background-color:transparent}.btn-group{position:relative;*margin-left:.3em;font-size:0;white-space:nowrap}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-toolbar .btn+.btn,.btn-toolbar .btn-group+.btn,.btn-toolbar .btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu{font-size:14px}.btn-group>.btn-mini{font-size:11px}.btn-group>.btn-small{font-size:12px}.btn-group>.btn-large{font-size:16px}.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>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;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-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;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:#04c}.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:8px;margin-left:0}.btn-mini .caret,.btn-small .caret,.btn-large .caret{margin-top:6px}.btn-large .caret{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}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical .btn{display:block;float:none;width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical .btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical .btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical .btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical .btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical .btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;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 h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.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:20px;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-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;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:9px 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;line-height:0;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:20px;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-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft: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 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover{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;line-height:0;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}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible;color:#555}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#e2f2e2);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e2f2e2));background-image:-webkit-linear-gradient(top,#fff,#e2f2e2);background-image:-o-linear-gradient(top,#fff,#e2f2e2);background-image:linear-gradient(to bottom,#fff,#e2f2e2);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffddff',endColorstr='#ffe2f2e2',GradientType=0);-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#555;text-shadow:0 1px 0 #fff}.navbar .brand:hover{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px}.navbar-link{color:#555}.navbar-link:hover{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:6px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;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,.navbar-form .btn{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:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;width:100%;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.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,.navbar-static-top .navbar-inner{border: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-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#555;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;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='#fff2f2f2',endColorstr='#ffe5e5e5',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]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#ccc \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 .nav>li>.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 .nav>li>.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 .nav>li>.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 .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse{color:#999}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover{color:#fff}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-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;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;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-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-image:-moz-linear-gradient(top,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #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='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#ccc}.breadcrumb .active{color:#999}.pagination{height:40px;margin:20px 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,.pagination span{float:left;padding:0 14px;line-height:38px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a,.pagination .active span{background-color:#f5f5f5}.pagination .active a,.pagination .active span{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,.pagination li:first-child span{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,.pagination li:last-child span{-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:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;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:2080}.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;-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-header h3{margin:0;line-height:30px}.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;line-height:0;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:1030;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:-3px}.tooltip.right{margin-left:3px}.tooltip.bottom{margin-top:3px}.tooltip.left{margin-left:-3px}.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;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;width:236px;padding:1px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-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}.popover.top{margin-bottom:10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-right:10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.popover .arrow,.popover .arrow:after{position:absolute;display:inline-block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow:after{z-index:-1;content:""}.popover.top .arrow{bottom:-10px;left:50%;margin-left:-10px;border-top-color:#fff;border-width:10px 10px 0}.popover.top .arrow:after{bottom:-1px;left:-11px;border-top-color:rgba(0,0,0,0.25);border-width:11px 11px 0}.popover.right .arrow{top:50%;left:-10px;margin-top:-10px;border-right-color:#fff;border-width:10px 10px 10px 0}.popover.right .arrow:after{bottom:-11px;left:-1px;border-right-color:rgba(0,0,0,0.25);border-width:11px 11px 11px 0}.popover.bottom .arrow{top:-10px;left:50%;margin-left:-10px;border-bottom-color:#fff;border-width:0 10px 10px}.popover.bottom .arrow:after{top:-1px;left:-11px;border-bottom-color:rgba(0,0,0,0.25);border-width:0 11px 11px}.popover.left .arrow{top:50%;right:-10px;margin-top:-10px;border-left-color:#fff;border-width:10px 0 10px 10px}.popover.left .arrow:after{right:-1px;bottom:-11px;border-left-color:rgba(0,0,0,0.25);border-width:11px 0 11px 11px}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}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;color:#555}.label,.badge{font-size:11.844px;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}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-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:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-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(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#fff9f9f9',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{float:left;width:0;height:100%;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(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',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;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;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:-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);-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,.progress .bar-danger{background-color:#dd514c;background-image:-moz-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(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{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:-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,.progress .bar-success{background-color:#5eb95e;background-image:-moz-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(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{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:-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,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-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(to bottom,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff339bb9',GradientType=0)}.progress-info.progress-striped .bar,.progress-striped .bar-info{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:-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,.progress .bar-warning{background-color:#faa732;background-image:-moz-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(to bottom,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0)}.progress-warning.progress-striped .bar,.progress-striped .bar-warning{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:-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:20px}.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:20px;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;-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:15px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{line-height:20px;color:#fff}.carousel-caption h4{margin:0 0 5px}.carousel-caption p{margin-bottom:0}.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:30px;color:inherit}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden}.affix{position:fixed} + */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{height:auto;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;line-height:0;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:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-static-top .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;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-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.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:20px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:1;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1{font-size:36px;line-height:40px}h2{font-size:30px;line-height:40px}h3{font-size:24px;line-height:40px}h4{font-size:18px;line-height:20px}h5{font-size:14px;line-height:20px}h6{font-size:12px;line-height:20px}h1 small{font-size:24px}h2 small{font-size:18px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.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:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}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 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:25px}blockquote small{display:block;line-height:20px;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}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,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:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;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:20px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}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:20px;padding:4px 6px;margin-bottom:9px;font-size:14px;line-height:20px;color:#555;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}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-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;-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:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal;cursor:pointer}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;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}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-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)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.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[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .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}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"]{float:left}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}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;-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)}.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:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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;-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)}.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:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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;-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)}.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:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{margin-bottom:5px;font-size:0;white-space:nowrap}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;font-size:14px;vertical-align:top;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn{margin-left:-1px;vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .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 .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}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 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;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{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:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;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:10px;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:20px}.table th,.table td{padding:8px;line-height:20px;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:collapse;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,.table-bordered tfoot: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,.table-bordered tfoot: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-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-right-topleft:4px}.table-striped tbody tr:nth-child(odd) td,.table-striped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table-hover tbody tr:hover td,.table-hover tbody tr:hover th{background-color:#f5f5f5}table [class*=span],.row-fluid table [class*=span]{display:table-cell;float:none;margin-left:0}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}.table tbody tr.success td{background-color:#dff0d8}.table tbody tr.error td{background-color:#f2dede}.table tbody tr.info td{background-color:#d9edf7}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*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}.icon-white,.nav>.active>a>[class^="icon-"],.nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"]{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{width:16px;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{width:16px;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:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 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:6px;-moz-border-radius:6px;border-radius:6px;-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:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu li>a:focus,.dropdown-submenu:hover>a{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0088cc;background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0081c2;background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-repeat:repeat-x;outline:0;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .disabled>a,.dropdown-menu .disabled>a:hover{color:#999}.dropdown-menu .disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.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}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover .dropdown-menu{display:block}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.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 #e3e3e3;-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;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;overflow:visible \9;-webkit-transition:height .35s ease;-moz-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:20px;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 14px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;*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:-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(to bottom,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #bbb;*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:#a2a2a2;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',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]{color:#333;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;-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:16px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:2px}.btn-small{padding:3px 9px;font-size:12px;line-height:18px}.btn-small [class^="icon-"]{margin-top:0}.btn-mini{padding:2px 6px;font-size:11px;line-height:16px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}.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:#c5c5c5;border-color:rgba(0,0,0,0.15) rgba(0,0,0,0.15) rgba(0,0,0,0.25)}.btn-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-image:-moz-linear-gradient(top,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;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='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#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(to bottom,#fbb450,#f89406);background-image:-moz-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='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#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:linear-gradient(to bottom,#ee5f5b,#bd362f);background-image:-moz-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='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#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:linear-gradient(to bottom,#62c462,#51a351);background-image:-moz-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='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#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:linear-gradient(to bottom,#5bc0de,#2f96b4);background-image:-moz-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='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-image:-moz-linear-gradient(top,#444,#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='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}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-link,.btn-link:active{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover{color:#005580;text-decoration:underline;background-color:transparent}.btn-group{position:relative;*margin-left:.3em;font-size:0;white-space:nowrap}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-toolbar .btn+.btn,.btn-toolbar .btn-group+.btn,.btn-toolbar .btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu{font-size:14px}.btn-group>.btn-mini{font-size:11px}.btn-group>.btn-small{font-size:12px}.btn-group>.btn-large{font-size:16px}.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>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;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-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;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:#04c}.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:8px;margin-left:0}.btn-mini .caret,.btn-small .caret,.btn-large .caret{margin-top:6px}.btn-large .caret{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}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical .btn{display:block;float:none;width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical .btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical .btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical .btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical .btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical .btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;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 h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.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:20px;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-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;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:9px 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;line-height:0;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:20px;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-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft: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 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover{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;line-height:0;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}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible;color:#555}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#e2f2e2);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e2f2e2));background-image:-webkit-linear-gradient(top,#fff,#d8e8f8);background-image:-o-linear-gradient(top,#fff,#d8e8f8);background-image:linear-gradient(to bottom,#fff,#d8e8f8);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffddff',endColorstr='#ffe2f2e2',GradientType=0);-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#555;text-shadow:0 1px 0 #fff}.navbar .brand:hover{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px}.navbar-link{color:#555}.navbar-link:hover{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:6px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;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,.navbar-form .btn{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:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;width:100%;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.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,.navbar-static-top .navbar-inner{border: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-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#555;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;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='#fff2f2f2',endColorstr='#ffe5e5e5',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]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#ccc \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 .nav>li>.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 .nav>li>.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 .nav>li>.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 .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse{color:#999}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover{color:#fff}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-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;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;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-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-image:-moz-linear-gradient(top,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #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='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#ccc}.breadcrumb .active{color:#999}.pagination{height:40px;margin:20px 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,.pagination span{float:left;padding:0 14px;line-height:38px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a,.pagination .active span{background-color:#f5f5f5}.pagination .active a,.pagination .active span{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,.pagination li:first-child span{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,.pagination li:last-child span{-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:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;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:2080}.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;-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-header h3{margin:0;line-height:30px}.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;line-height:0;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:1030;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:-3px}.tooltip.right{margin-left:3px}.tooltip.bottom{margin-top:3px}.tooltip.left{margin-left:-3px}.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;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;width:236px;padding:1px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-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}.popover.top{margin-bottom:10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-right:10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.popover .arrow,.popover .arrow:after{position:absolute;display:inline-block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow:after{z-index:-1;content:""}.popover.top .arrow{bottom:-10px;left:50%;margin-left:-10px;border-top-color:#fff;border-width:10px 10px 0}.popover.top .arrow:after{bottom:-1px;left:-11px;border-top-color:rgba(0,0,0,0.25);border-width:11px 11px 0}.popover.right .arrow{top:50%;left:-10px;margin-top:-10px;border-right-color:#fff;border-width:10px 10px 10px 0}.popover.right .arrow:after{bottom:-11px;left:-1px;border-right-color:rgba(0,0,0,0.25);border-width:11px 11px 11px 0}.popover.bottom .arrow{top:-10px;left:50%;margin-left:-10px;border-bottom-color:#fff;border-width:0 10px 10px}.popover.bottom .arrow:after{top:-1px;left:-11px;border-bottom-color:rgba(0,0,0,0.25);border-width:0 11px 11px}.popover.left .arrow{top:50%;right:-10px;margin-top:-10px;border-left-color:#fff;border-width:10px 0 10px 10px}.popover.left .arrow:after{right:-1px;bottom:-11px;border-left-color:rgba(0,0,0,0.25);border-width:11px 0 11px 11px}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}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;color:#555}.label,.badge{font-size:11.844px;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}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-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:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-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(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#fff9f9f9',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{float:left;width:0;height:100%;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(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',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;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;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:-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);-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,.progress .bar-danger{background-color:#dd514c;background-image:-moz-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(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{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:-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,.progress .bar-success{background-color:#5eb95e;background-image:-moz-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(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{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:-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,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-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(to bottom,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff339bb9',GradientType=0)}.progress-info.progress-striped .bar,.progress-striped .bar-info{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:-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,.progress .bar-warning{background-color:#faa732;background-image:-moz-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(to bottom,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0)}.progress-warning.progress-striped .bar,.progress-striped .bar-warning{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:-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:20px}.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:20px;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;-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:15px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{line-height:20px;color:#fff}.carousel-caption h4{margin:0 0 5px}.carousel-caption p{margin-bottom:0}.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:30px;color:inherit}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden}.affix{position:fixed} From 23762efda25c67a347d2bb2383f6272993a431e4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 30 Aug 2013 10:16:26 -0700 Subject: [PATCH 0879/2521] New hardware provisioning doc, and updates to menus --- docs/_layouts/global.html | 22 ++++++----- docs/hardware-provisioning.md | 70 +++++++++++++++++++++++++++++++++++ 2 files changed, 83 insertions(+), 9 deletions(-) create mode 100644 docs/hardware-provisioning.md diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index a76346f4286..a0145544623 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -61,20 +61,24 @@ Programming Guides @@ -91,10 +95,10 @@ diff --git a/docs/hardware-provisioning.md b/docs/hardware-provisioning.md new file mode 100644 index 00000000000..d21e2a3d703 --- /dev/null +++ b/docs/hardware-provisioning.md @@ -0,0 +1,70 @@ +--- +layout: global +title: Hardware Provisioning +--- + +A common question received by Spark developers is how to configure hardware for it. While the right +hardware will depend on the situation, we make the following recommendations. + +# Storage Systems + +Because most Spark jobs will likely have to read input data from an external storage system (e.g. +the Hadoop File System, or HBase), it is important to place it **as close to this system as +possible**. We recommend the following: + +* If at all possible, run Spark on the same nodes as HDFS. The simplest way is to set up a Spark +[standalone mode cluster](spark-standalone.html) on the same nodes, and configure Spark and +Hadoop's memory and CPU usage to avoid interference (for Hadoop, the relevant options are +`mapred.child.java.opts` for the per-task memory and `mapred.tasktracker.map.tasks.maximum` +and `mapred.tasktracker.reduce.tasks.maximum` for number of tasks). Alternatively, you can run +Hadoop and Spark on a common cluster manager like [Mesos](running-on-mesos.html) or +[Hadoop YARN](running-on-yarn.html). + +* If this is not possible, run Spark on different nodes in the same local-area network as HDFS. +If your cluster spans multiple racks, include some Spark nodes on each rack. + +* For low-latency data stores like HBase, it may be preferrable to run computing jobs on different +nodes than the storage system to avoid interference. + +# Local Disks + +While Spark can perform a lot of its computation in memory, it still uses local disks to store +data that doesn't fit in RAM, as well as to preserve intermediate output between stages. We +recommend having **4-8 disks** per node, configured _without_ RAID (just as separate mount points). +In Linux, mount the disks with the [`noatime` option](http://www.centos.org/docs/5/html/Global_File_System/s2-manage-mountnoatime.html) +to reduce unnecessary writes. In Spark, [configure](configuration.html) the `spark.local.dir` +variable to be a comma-separated list of the local disks. If you are running HDFS, it's fine to +use the same disks as HDFS. + +# Memory + +In general, Spark can run well with anywhere from **8 GB to hundreds of gigabytes** of memory per +machine. In all cases, we recommend allocating only at most 75% of the memory for Spark; leave the +rest for the operating system and buffer cache. + +How much memory you will need will depend on your application. To determine how much your +application uses for a certain dataset size, load part of your dataset in a Spark RDD and use the +Storage tab of Spark's monitoring UI (`http://:3030`) to see its size in memory. +Note that memory usage is greatly affected by storage level and serialization format -- see +the [tuning guide](tuning.html) for tips on how to reduce it. + +Finally, note that the Java VM does not always behave well with more than 200 GB of RAM. If you +purchase machines with more RAM than this, you can run _multiple worker JVMs per node_. In +Spark's [standalone mode](spark-standalone.html), you can set the number of workers per node +with the `SPARK_WORKER_INSTANCES` variable in `conf/spark-env.sh`, and the number of cores +per worker with `SPARK_WORKER_CORES`. + +# Network + +In our experience, when the data is in memory, a lot of Spark applications are network-bound. +Using a **10 Gigabit** or higher network is the best way to make these applications faster. +This is especially true for "distributed reduce" applications such as group-bys, reduce-bys, and +SQL joins. In any given application, you can see how much data Spark shuffles across the network +from the application's monitoring UI (`http://:3030`). + +# CPU Cores + +Spark scales well to tens of CPU cores per machine because it performes minimal sharing between +threads. You should likely provision at least **8-16 cores** per machine. Depending on the CPU +cost of your workload, you may also need more: once data is in memory, most applications are +either CPU- or network-bound. From 5e30172f706090ccf49ecdac7983cb4b1c091417 Mon Sep 17 00:00:00 2001 From: Mikhail Bautin Date: Wed, 14 Aug 2013 11:51:00 -0700 Subject: [PATCH 0880/2521] Make HadoopRDD's configuration accessible --- core/src/main/scala/spark/rdd/HadoopRDD.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index 6c41b977809..e512423fd63 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -34,7 +34,7 @@ import org.apache.hadoop.util.ReflectionUtils import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, SparkEnv, TaskContext} import spark.util.NextIterator -import org.apache.hadoop.conf.Configurable +import org.apache.hadoop.conf.{Configuration, Configurable} /** @@ -132,4 +132,6 @@ class HadoopRDD[K, V]( override def checkpoint() { // Do nothing. Hadoop RDD should not be checkpointed. } + + def getConf: Configuration = confBroadcast.value.value } From 35090958b3df8ba06360be6a48489b33b22216e7 Mon Sep 17 00:00:00 2001 From: Mikhail Bautin Date: Fri, 30 Aug 2013 11:03:57 -0700 Subject: [PATCH 0881/2521] Also add getConf to NewHadoopRDD --- core/src/main/scala/spark/rdd/NewHadoopRDD.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index 184685528e3..b1877dc06e5 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -120,4 +120,7 @@ class NewHadoopRDD[K, V]( val theSplit = split.asInstanceOf[NewHadoopPartition] theSplit.serializableHadoopSplit.value.getLocations.filter(_ != "localhost") } + + def getConf: Configuration = confBroadcast.value.value } + From f3a964848dd2ba65491f3eea8a54439069aa1b29 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 30 Aug 2013 12:38:23 -0700 Subject: [PATCH 0882/2521] More doc improvements + better warnings when you haven't built Spark --- docs/_layouts/global.html | 15 +++++++--- docs/img/incubator-logo.png | Bin 0 -> 11651 bytes docs/index.md | 53 ++++++++++++++++++++++-------------- docs/running-on-yarn.md | 39 ++++++++------------------ pyspark | 2 +- run-example | 2 +- spark-class | 4 +-- 7 files changed, 58 insertions(+), 57 deletions(-) create mode 100644 docs/img/incubator-logo.png diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index a0145544623..91a4a2eaee6 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -66,6 +66,7 @@
  • Spark in Python
  • Spark Streaming
  • +
  • MLlib (Machine Learning)
  • Bagel (Pregel on Spark)
  • @@ -77,8 +78,8 @@
  • Spark Core for Python
  • Spark Streaming
  • -
  • Bagel (Pregel on Spark)
  • MLlib (Machine Learning)
  • +
  • Bagel (Pregel on Spark)
  • @@ -140,9 +141,15 @@

    Heading


    --> - +
    +
    +

    + Apache Spark is an effort undergoing incubation at the Apache Software Foundation. + + + +

    +
    diff --git a/docs/img/incubator-logo.png b/docs/img/incubator-logo.png new file mode 100644 index 0000000000000000000000000000000000000000..33ca7f62272560f3da274086a7532bcdec7b76ac GIT binary patch literal 11651 zcmZX41yCJrvgpCx-QnQw?(Xg$4iMZO5`x3Q-Q8US!QC~uyL)g55N`gx_w8=go2qZ7 z=WFk&>FVwou$qc23L+sQ002Odmy^=?d!6{ZnBZamf!dW2006``8%aquc}YofHCLz4 zHujbPfE-Fqs+ShlD&EMpl5Y`Ayvj)z5;VHp2?TaAVj*Utm^FJCG)4nO6k~m1gj`cW zlm;{-c~oQsNV>sn`!CSWf@@#?MpHj1+;g}7;qNrnO(!g8bSXDU_fd z6OaMfKnMI8-a^ij8|DY?JX(TW-^R5Fe*322q?uIRJM;Y|l& zZ-Qx?RtvEKa+?t<6Gz45{&E^2moOVz5C98{^U^v>ID(z@MDaAFuD>M`45O_Myd8ui>Ao;RAwr0LEVYnCg$;MZaH!K}h9CSZH9K zKu~@{wGW_lh9C_Hlp;dz7sHN#ih_XqB~F7Lcu0;`4HcS?Zx6dJ#=Qdb7zDLL&Ih?B zj>iWz9Xzok(27C|pvQ+`?ZbZt(2&FTi(^M1+DQD4N3ek5mvA3Nx($~ofTu%M2+<&i zpbEUre^6uBg4LGv%ZEK-bB7dwG7f>w_xp*?h1eC^-Ak8_T-D!RPj3%5Z$?85Dccvd zgY8X_jfB^mwWH9A>KL}Z!}-hdnTILVw11y209de~4ogRbO>!88C^=M=s197P)xxNR z)DE{5_bB|W#;dNez_?(=g{v$*|C7fW%^h8Ua7vt2vQ8!~ziJllr^AWKz4AS}FY-Hy zWZ%iBYPKtU+|Lj+Fipgo5f!})=CS6}W{&1EY-II%)dEY%=aDG=!)EjKj1AN)Br7`h zyq*BTXp)g#=S)7vY+4rNkzue)Xe)ye=M~o#tRwUjf@=TyKFJNlvv(&TFRreacTaWK zX(w-&7G41wD@c$cx|h0|JQ?#5LjYkJo-zcXkYhIbL_Qi=im{Br2xcC18w?-xqQV@D zKaj~(RiLh+)=orCV2KZt%Nt{`P;6x74`iZn0$$U4C0!?MtJ7g?A_ilsVrcTH6Vh4H z$I<6umm{_yGQrgL6NVCo@`Ms0*5M#T(nKl^4Mny^;zY{g)X{fq0O>LqUKkoubQl-u za?}X52z5&t1nI)*tr*hico|H!{A+_W*|l$UcPp{WFSPlqIFzK-W6Ha=P(6e_$R3D2 zq;RTiOUE^oHJNpU%h&1zN=plQ44Gw^Wou+Ei!-%IN*pVvHD=UUwM7g4W#1>5HuWd< zJN1dInkU=Y&x^uZU?qdg)Qe{;T@3f5Xq4woIgQ<-K6e~ZGJj7kS^83QXdu`;ZPPWU zv-tS)$QsXvY__`&>Rj(k>ZE#MwaQJ}%1Bq0S53Fvu*6U2DJ8d^RFYM>&geRMt6W5T zTTxWI$LQnxAEZpe82O2O?Kxi?b65dbaahb?Vc2(Atgz2vvtch`HDPhl5i;&Y7DcW_ zq($uJU-#RFeTJ2XpOoQg+_AdHJd;F{;z~c4Hk1-6tt;nc*`zm2x=v+Il4fd7c1@*a zEN5&t_8RWi`wu$zk4t z)1u0e$TrJq-SKs8dWB`nG0He^)638575Vk+BO0+5aS+KZi2%u#fPsK3u^q7!zj`)g z_7a~5Usr|$cNVYT@kPh9)^Xc3QMO-pPg`qygpq-fjB$)Hma&~N+-c>#voFGX5Yh0| zh(nBF+Pak&o>#*w##4`wz0jO6p>XaDd)7c!lhbs4Spn_3TU^;f)KSVw^Sa`h!_L%W z?VW!=8_1d9A=*j_=`&$d#>?8smh6Gs#&JG|>b%;LdJ#<<~&y@W`fC?+@?L=vcA2VGI4VDdaFrG*{%&o*_IBA5-7;#Z?;JP?6(~}CZIK?PkN4jz4>>UHFm^~`-w@v_D%1+pwD?;4%m-@oW<|^XZ1l|9m@f`r zOK%h^4m}Rx44EqOQK$L4JeS<2jVX_1{Z=$q@Nyh>#I^5k8uFV6S^!mD*)Bbvu+bitgQR;!?*WlOmB(jOuRClfWyAI{S;9=-t`w{6;5lxEt#KGCY z+Iqo;$ss6^QW^uZ@lZu7ow@;CB@!DD1O@&;*{uS?y6NfE~)4H*y z9^Jm3+a*1hndi;1=Z?#Qa7W15?Ym$1CzX4$j-47Djk|@fAA5671ij#}U`xE(yz;gu zTAz;mAK$@nc4Qz@Twx7gtsA{#m(6S3o*xRv#kApXVYNcwFSno4Ip1#MsztkFv4u*6 zYP?H7s(0J=?`mfQX3tr1yYYSKzwqBw{!%>@wKL7?5qZShuVXYv8kX6L*i!0d_9=UD z{CHix_8T4;A^SG^#&*+il5|uCm3Nx=+CAv+!_)A_cx89U8|9R&jgaCYApysZ z08~x@oA?pps1Wfk9c(~O_5q2yB{SpdmynmbPb+ z(&ObTpj}jHd6mPi`7bp9AmlCZ7j&?6HzW6Uuy=G5@D>LC zi$dTp{LeNkko;dH?smdJJ!Lg=NheoJa&8uO77m~YA~`v^kgLUK0Szgc|6~7qCk(W9 zcXt+GW%cs%V)5c+adNd{W#{MTXJzAH<=|lcOTp~s0%Z$pvJ)b{00)f6M;MD)i5; zfSQfBrM;e%jf175+g}|b9PB(o|Dyl@F#pl`U!3~?;pF1`Z^r*Z{2!wb>p%AYmp}g- ztbgtPjg|ilmt9;;UR+E+=Gx2ibV1Cq zb0x4y7Y9S*NWnZ{q{ynH6P-~$=NAeENYN``1j9pMzz6=sQt(|@=2>sD{QP>A{QB*> z&5y^+%8J*pvH0zk+G~dSINR^K?O51U*vdW(eOSJ$OW4}MLEO&{J;55Er-Ov2V&2RP z$SsUSPhYX$Ibd-ERRCjPXlRz^bbj};C!%6$N$q*L7OLO$x$pXE;M9G1%Bw#2dkh?t zu<&MiMJ@WLm0jiAzC%|@M`EUt5wVzCCngJm#+hRW6243Cs$;O3XO8 zT}}{9<|C-Yo-1aE@txNbtUb|`v3x-tuc7|G=kxYPME|g3Vq*SWU+*x@H7j&#cRN0oS z#mqp1N5pz9hS-UsU9x8MNxI(B5a#IEr{U)nZ9>$Z%%#GYqdr2`Sh z4UdcsxE{f;XlExTRi}WRmQ1e}B1yrupj%*gkJ%9;h7sGZ+=TOpEgrc?cfbIdX3fV0 z8xXN4H$>g^sBfAn@>QV42@kz}QvrH(a#BiJIfiicK{=JhxKb*_=kAC?P>^UMm07d# zsqHv73QJW;-D=?PIk4GonV~JK&|F*`Vw!Isnb_~5;oYYdEAi>uSw-2eSy{3Q-%^#F zN*TQ+r$Z)eMffdd!K zk&;=N6M?VJCw0*nwI~@VZ8Cs}>j|B?Mi(Yc2_0E{dus+UxlnitqE;k&it4!;CVIXv z6RkNbxP7^F`eu0fac=t2|8ktSZQ28ioiY~|2CH@rylErzIlwUTT(vw~Idr8bFNc~S z8$bE6?cX6F*>Ih#sUj{d9ZC%+RXr;0;NT#tr}vahuPK%wjWwaJLC-X2cC1?Z)TqR)G6f5n>-e#*9utE^;jt~coTOv~f&4e!M#yol{9#T;>)9GGuc zmNha-BmVrNH)zuR%Yspqw2gkb^=6>GuGDW;^uSzC?3as`4PZ7Me-Nn`oPZ|Kc`QGh zgBneHPDC?~9|AmWGn&Z(r>QXM&p+#6zP*kjQDCBVwGo*(w>sBA4B9;tU#vyMqMIWi zz6~T%YU8)qwpk^H1P001(zD0>j+T6+rcPVK8&uC4#CXZtexp%PR9tKjsY7M@_Dn%9 z^7i+6Y7vKtmHHQe^SzIAynZe$grJGOq05r`6t~}pi@u3l^--r86$;4s`U?2pWo^@` zNU2jXGFFMAdFk;EVa2w}l70Q&{y5H97eJ#khvxqaYDj%F##-Fg-d<~?Owqmf`pdB( z!%X0xaH!!dr|t?^d4qmFJ|n+fOkyih#A6)eC>w$i;LmOxMxN~$Gyz?Zax3b@8N)A? z4zf7y=KallBD70uu{6oSmH;%5u9CTMT#)3s*0a!^W&8u>MnY7f7=+rP)Owh$_tpx; z{#N+xpHY$(-|Z6b_di(m7vrkCIYcUhMZ7uS%X5<$!JU{N%;Agr&kaAEj9HC7g}Hln z2#Lk5%>I}VMu7;Yvw?;U6f+*8fVH3w0z1yh!g?JFgk*_f4ydS@hnqze>P}AQkX(M1 zfiwoW(+lDZ(O?R)wXKz;nM9ht?cv!Ln;%p1RS&B8^}*nkYo0IHjdVS2MLDdu&|@%j zbK~%Cx+f7Xjyargl3U9u(fe}q@&;4VgZD5~j4%biR3AoV2jx>WTt%_8%U$Z~>IQ?> zIcJ25V#nnDADi+j-lwNkQZ%qFUfuHf}py>*;iGULPFvNs(N|&DMrw#|eAXFIq zwyuaz>XK3<7>hgD1j0;W7`@U(3wFg&qqxXCF-S}X*E%O1?+-#D99U#Tuo$TS0wwjiP2%?As7@`u$w7X=0b!Z_VvaXY}MT(sZGdGnHY6z zwWL=l_~$-u(|WYTive;)o+p5`pw|E(_e{=T&f;R>A~;hWLQi5r2y zp-${E>&GSEXqNT4IoDn&X;H?9KAZVc`dY>MWYaF_z79u8ke_hr@3o&R84;b(ysk+( z>6zKFMy7}@7grMPfWpCyz_4Uo?>PaTBiX+3W+|5Ppwz7Mu)3z>pmjvw3t$Yj1J6mQ3(AALxyo%z$6U-p{j zGiCN(>+~8VkkRSiLAmYZ-&P2wh3=H7D`)v-)xjGPOXOz~4GKLhc&zVpY2C%yehcO& zgXY4-ITcW9W=DdgOoNZ+>+uSz{A3mL$Gb6ttP-mA5Cx76_G7*4SRCH&i#m`g;}1Ns8RdtPQ!C1ZL2) z*jMB6jzY~)Y`#)4z)TViHC%o7Aa}Q=d6S)#Rq4Y?7x}sya1>{Uvfg-VUTkFnjz|uA zY{}P)tBl`?9}g+wvy^gYkT!A0m9LoiS;z6yUMWtPnvF9?;TTC)G9tnn`$$9+LV*uc z!qJ4CqsbLkXh9Cu|8Hrw|GNFrJ+2@&nL)|a8OP(W7P8u8RaJED|J3ox!~tbrFr-5A zVRno*FZ$b`yS)6)soCtq?{p)bNO&w-`R1OHDi@R38MUo@k^c2(?W6S8FO8Y8&6>9l z%cdiHLs8mG+EqHct^2W>VnsmJtCG9ZcDl$ISnu|Yz*I*Ae>Bf(Ux+D8+?37^l9w$l zF77Bi(SyDXXd)gDYO_p10&VAr=L2zx6$o7o5uS*vy;PM0JR}1Ypfv`PzTbDetz-*# zg3EF|EVteh2~8*Fv!DlSM@&W#h(|+s=ob}L-?fsol`6pI%Kl!VvvB)rDB;w95`%2o z?_u3V!vwYE`x3?KZeJ%CS?2Kv+NR~yXx3ROgb;8qlc;N$X_Ehfr??cS>N5#+YHcm* zMfFM{aU6C>B?BWgwkH-9(35|t|Il6D7B4@7MWR-MM2eklYWRStd<79W@vIoj>wPQF>8d|Lj|p={p0D z<3_m@oWLBHm4(ll0N47Qz>qU)q~d)-{7GDctE$1oGchzljdQ0;LTrGUtW_5w`2twv zc?U`x&4a>~S(%Hs=WktIiR@9**vLWr?d>L9iyXLa`*4Q_tnuT<^n_8H%ZUif)Pe3m zp1h+i5=xGUphO4-w(Anj&7$yPUH~hBAvuugr2Kg-AcciXya4wiK7OQ+DztKO{^flM zA9E=CX{lMsLH#RCHcxQT2F-SD5k#LHZRUDbNK*vEjyvU879(t6d3r9V`iWC70iF~3 z=7yL|yW+TX!cJnJX+ZXlYd$CqLUkO>=XeuWE)x%TQGjy3jdw0nfEZa!hl@C2tx%h| zldPUnal~}L6vZZ8Z{2$zC=DU)j(Zt@;35Sj&4)NtSI?Itkziuh3{GeFBkx$h<9c;e z4L+PO{nL2McioXmYe(pS)cLkIztZ`I-m`i$!mY8t+!mcEekJJWfW}-?8*eZiJU5A~ z<+V1XjiFH~26hh3OeKYuc4Sn-!cYncwpT@Lfqe&_g|iLW;q`t8^Cg88PFgIuhsCoJ z+av|drk+UZCjWND`%LS1V_WjCP7l=38!U%%U` zYAl;y9w3F3+3wg#wu%QM^GDJ>y*5{6D75-ee}sypo!R;96fcOV&D&0qQi!Y@!pjgP zwdWafZJkg{C^EJ&KPZX zhe{k&9#-p)$)a3YOs8Xy<5^gMjZa$*4(ca?;-46PsFL=a!PImCR#q`+!C6VMJppJ7ZD@sas)SRR4&Zm^5|1r=WJNN1?I# z!rUttaGd7{JCSF|;u`^%Hg&T*mSQ)k22eUj0WDsU?h+yJ(Mv;9V*JPy5FT|6-AS0r zOALK<#M4L$s)EmRF-6`f~*Y! zbdpwLfP$`6FdH1gieFlH!-yUOLJMa{$huo3S0<^t1Cn7K$8 zXFB(I7W9=If!|V)<%#e^1sNXIcNxC3NK20a#U#2)T~%|WgDs~wD5>h_zDvcFfGc)1 zugb0x>BRDiKggKHir|^SZ-3}p*mV5sQHhqFdd*5g;khq{^_wPs8M*NSK?#PgllDR# zRVa2pE~RwDyE*p0oEvUyq$&$!w^73eojmyLp!a}G{$Q6}8yGg4T2-W?r%q4nP*J}~ zw{4a%nY@3R&BtJo#83{D=8KdBn`8GW^_EB4z+lx&Ny`-1g|;hvXXWmb_&TKZi(BvW zv;ozhY^rm&80#KM`=e}&;;&56-X+(;Tep&-*2Vb^z|_yv+5FbG+V;6$zbX~MF~>AW znM2ovgbO^JGj$NzTJV?$&S_h(70S1Y)+*|S-_B_+=2z&EaB}l%5_1l*Nd|^w(*QrQ zdl2C$%Rh2PSR1mXe2AyXIPdL-lU>0fHqGz#v!ytF*(ai;`|4~_fD?0csRv>rjKRsH znq_Ua7)uoYK^1|2>spSrXs>STn=H?ZiBMEjvj`2rYy^$W%kYRf4Pv+@r24ARJ!kj` z=qr^l)vKz^EKm44Hlg{|LL%f_#dbt1O}5qFykh8t!Z4g(DW7Q{2)2e;EFwCqlzzJ{ z&?b&QAY`~O5$M>%hlUzr4l1jg&uaKY1FhtYu68sWUhHL3g6jy92)25Cj6kz&PR&=s zxEz)+eeTDD#d$53T^C1V^Vy4cfLN>w?!!fR_*GD3+#2Z&o0KKTkYnt0@r1|*t(^BP z{fWN2Piy*emmI{=(ufIJjD~M7f5rhFlYwvE62kuW?$z^35-Xt4-!~(usysYL@z6O# zrQFinZ4Xh>pr}{XTTjQ;ow>DB2J>My_m-(PUXU$$e=D z&NHGPZ)!seOiI&D4UWDn6aQ3-5XT;*0vC@md9NGCTInc~_V@#jUDsdH956MTc|oK{ zmMbx_t)C6C@$mLj4IR~?#B!-Va?Y>hP-v$7C3lC z?d|>6<#p_tgy%70`D500NQ5LLmoi?uWfa{6l0iV~N@*?k)dMIZjta$Ej|!nlB5RxE zMvt%)=`cQln8;h@;nXQIqVwf$t1qp-DQnPd=0F6Qi;-Ne*tInNhv2is)tq77?%Nb` z9}j}k+g;?074C8-)$c!k5MgEr+50T@`Ta6!pbDPMZ{!eHwtQWz^XEPxJWp(6I%`+@ z6y;}JZ8nTRo6UBT=`JN1fd2~rKEkoN_KEboW8^A|V5RlC`}Oz42?>FH7&EOuQjT_s zJyOPz!=T@{b79N+1b-dj{Av;OrvM79*S#%W+~J?AA_)uAhfya_Fk%W9LF(hI|n*szrk5?9T&KXs3BA23h4DN&>8BhI+!Q@I%T3&VaO=6P_rUp6ma zag#<**RcBELcT!x|?Awip6>G!~R?h2<=k=?n6t`$XB%oe$M-j5r}-C`Y4K-T1Z zgNHo*DD|j9V?*Q$0))bshM>up%{`TTWQnnh_g2{x!C+poRZ(SgkA2xl4q{zS_wBk# zgR~TW4LbR0PD_B`g2Lp_$N2vEqX$O1BmtAw1%vA~iB7{60`<2w7dv8Rj+YaO?y=SQ zDGC!WE{F)3I(4WjPJQRcmhX9hV)WqAJs2wV;@%|!x}+710DHQ*poq<;q!wS6&c_}H zC)8=g7AU^MHn##D^P5iNhmSIGG6QtjvSrf4@T69M9xXnT*WZzNz{uu0L>bU-Z;!Z$ zd8RSjs_1y`yj|opnepe_0j=n15A)N0%*H#x-o-DHzf()XPqFp;5(qeS#-Zz|QyP*m{Vi|G5Qs;#*2)jUvIJ@oF;Mtx7lZdEEPX|(~~8xxMk;DN_}B|r9o z3^widD_yFKcBbB55D_u)^LyJLF&CGc{EA86)zX#5CG+9^hH9Qejp9hzY;lDN0W&NcZ(ssBAn5N}l~llQ`T`2^~X;AY(Q0 zytl-=FYt?h)to#WK^tNWRrZJ~8RU=dyf_D4;*X328wa4(^Gb99l{X-kOCP-u1~ z6MUX{$*Qr{`#qXzVOa;oAGT>^!e{d#LmDdVSc#(zVW!Cp0X0a5C(@K7F4XL|t*O`$ zwP~9V6>dz=x+^)>fb|}RG(u+PD#pj(A;XCF(Gc96X>xuP4xEjd`TGJR2%M8eVvHrb zvvO;65|!9B>QJYz`Bjni=wR9q=^?q0_{`{mG)Gcl%Oc%qMhUI zOML!kHqIZqwsrs_6m{=!XK?BrQJ9sIlo*^K$x(R(v|387e+vR>ZwJ1G=g^tx z!PZ!CMg5Z`T~GiQ7x(3MdqP=yh4$0bDSZ9Im^gZjC^*F9}~yPX_ajl;w#Sc)`c zfJH|!5aH!#)7>vnk%k(uu9$IWn&`t#t+X}|l7x_Q!9+oqm)0ie6=xzpg$u8D9WwT2 z1_jC0WU~hBCdxshJY+$>z|;qa(v>1}H;(wh4a*J1c3)ih$`^ZERv+svRl?s9dZfdV zeNn5Sw4V}O5K*qYs9qvhXIWEb%*7d$Od-1xYO5hF2%i`^ECc044;&sRJG+j7fh`F8 zuw!j>7}24>`y{X&b0cKtX26Ke89Qv^{cecZ2V58T{rta&SX?no(!NCwz_S6vDyquO zcNu7`fuooSac)+qG=;pG3Kz~KF(jE}!JORY;Bju$)oAD;(Z)9P!@JkOF z`-{?F1KQ#7Zlb7{W`1H20*57uyAV3N76(kJ@>jTxL|Rbrbj$GZXW-d$kE59_ea7s?0rlZfYLCw(*&l!d61>|(UBadpr+Xk$g5Hm6uX5KpH}H3X4uJwX%?W1W^h?y zO0Bc=IM*-vV#caq8bi?@kB|Z%q0eSp>lZbfHdr1z;dx8PJfc|$4*pMSY|5VgMfGx< zYRP8h>rZR3C}2-bF7A|5$*Z{iOF`@_UUmKF@~W!Dyi?S2>iBF|C^rgcmPG~?MZvEe zN>g!_*x!bVMHd=5=Y#2SQp_azxr|5`Gkc6hL47Z_0#0+Aq%VcJa-N8jLVbDlqc{#%5 zj8L8vzemo>eo=LYM(Ga5T0lp&xX+dq0wk~7PS^pd~>T}}YM#?h3jK6O}4g^Vy z665WhCv&&^T2yEY5S*UtDj;0~N)1?j5%`e;CEPeIw0qs3EtL&AA{>&qRw2VBkp6y1 zT*9MvZne;;<2wT0a$+v%TZ+?yGu!o14vFI?me2``c#O27W+|2t}(axi1+1 Q^M{GNw2D-%#HXPD0lH(mhX4Qo literal 0 HcmV?d00001 diff --git a/docs/index.md b/docs/index.md index ec9c7dd4f30..5aa7f740595 100644 --- a/docs/index.md +++ b/docs/index.md @@ -3,13 +3,13 @@ layout: global title: Spark Overview --- -Spark is a MapReduce-like cluster computing framework designed for low-latency iterative jobs and interactive use from an interpreter. -It provides clean, language-integrated APIs in [Scala](scala-programming-guide.html), [Java](java-programming-guide.html), and [Python](python-programming-guide.html), with a rich array of parallel operators. +Apache Spark is a cluster computing engine that aims to make data analytics both easier and faster. +It provides rich, language-integrated APIs in [Scala](scala-programming-guide.html), [Java](java-programming-guide.html), and [Python](python-programming-guide.html), and a powerful execution engine that supports general operator graphs. Spark can run on the Apache Mesos cluster manager, Hadoop YARN, Amazon EC2, or without an independent resource manager ("standalone mode"). # Downloading -Get Spark by visiting the [downloads page](http://spark-project.org/downloads.html) of the Spark website. This documentation is for Spark version {{site.SPARK_VERSION}}. +Get Spark from the [downloads page](http://spark.incubator.apache.org/downloads.html) of the Apache Spark site. This documentation is for Spark version {{site.SPARK_VERSION}}. # Building @@ -42,11 +42,17 @@ Finally, Spark can be used interactively from a modified version of the Scala in # A Note About Hadoop Versions -Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported +Spark uses the Hadoop-client library to talk to HDFS and other Hadoop-supported storage systems. Because the HDFS protocol has changed in different versions of -Hadoop, you must build Spark against the same version that your cluster runs. -You can change the version by setting the `HADOOP_VERSION` variable at the top -of `project/SparkBuild.scala`, then rebuilding Spark (`sbt/sbt clean compile`). +Hadoop, you must build Spark against the same version that your cluster uses. +You can do this by setting the `SPARK_HADOOP_VERSION` variable when compiling: + + SPARK_HADOOP_VERSION=1.2.1 sbt/sbt assembly + +In addition, if you wish to run Spark on [YARN](running-on-yarn.md), you should also +set `SPARK_YARN` to `true`: + + SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly # Where to Go from Here @@ -54,15 +60,20 @@ of `project/SparkBuild.scala`, then rebuilding Spark (`sbt/sbt clean compile`). * [Quick Start](quick-start.html): a quick introduction to the Spark API; start here! * [Spark Programming Guide](scala-programming-guide.html): an overview of Spark concepts, and details on the Scala API -* [Java Programming Guide](java-programming-guide.html): using Spark from Java -* [Python Programming Guide](python-programming-guide.html): using Spark from Python -* [Spark Streaming Guide](streaming-programming-guide.html): using the alpha release of Spark Streaming + * [Java Programming Guide](java-programming-guide.html): using Spark from Java + * [Python Programming Guide](python-programming-guide.html): using Spark from Python +* [Spark Streaming](streaming-programming-guide.html): using the alpha release of Spark Streaming +* [MLlib (Machine Learning)](mllib-programming-guide.html): Spark's built-in machine learning library +* [Bagel (Pregel on Spark)](bagel-programming-guide.html): simple graph processing model **API Docs:** -* [Spark Java/Scala (Scaladoc)](api/core/index.html) -* [Spark Python (Epydoc)](api/pyspark/index.html) -* [Spark Streaming Java/Scala (Scaladoc)](api/streaming/index.html) +* [Spark for Java/Scala (Scaladoc)](api/core/index.html) +* [Spark for Python (Epydoc)](api/pyspark/index.html) +* [Spark Streaming for Java/Scala (Scaladoc)](api/streaming/index.html) +* [MLlib (Machine Learning) for Java/Scala (Scaladoc)](api/mllib/index.html) +* [Bagel (Pregel on Spark) for Scala (Scaladoc)](api/bagel/index.html) + **Deployment guides:** @@ -74,27 +85,27 @@ of `project/SparkBuild.scala`, then rebuilding Spark (`sbt/sbt clean compile`). **Other documents:** -* [Building Spark With Maven](building-with-maven.html): Build Spark using the Maven build tool * [Configuration](configuration.html): customize Spark via its configuration system * [Tuning Guide](tuning.html): best practices to optimize performance and memory use -* [Bagel](bagel-programming-guide.html): an implementation of Google's Pregel on Spark +* [Hardware Provisioning](hardware-provisioning.html): recommendations for cluster hardware +* [Building Spark with Maven](building-with-maven.html): Build Spark using the Maven build tool * [Contributing to Spark](contributing-to-spark.html) **External resources:** -* [Spark Homepage](http://www.spark-project.org) -* [Mailing List](http://groups.google.com/group/spark-users): ask questions about Spark here -* [AMP Camp](http://ampcamp.berkeley.edu/): a two-day training camp at UC Berkeley that featured talks and exercises - about Spark, Shark, Mesos, and more. [Videos](http://ampcamp.berkeley.edu/agenda-2012), +* [Spark Homepage](http://spark.incubator.apache.org) +* [Mailing Lists](http://spark.incubator.apache.org/mailing-lists.html): ask questions about Spark here +* [AMP Camps](http://ampcamp.berkeley.edu/): a series of training camps at UC Berkeley that featured talks and + exercises about Spark, Shark, Mesos, and more. [Videos](http://ampcamp.berkeley.edu/agenda-2012), [slides](http://ampcamp.berkeley.edu/agenda-2012) and [exercises](http://ampcamp.berkeley.edu/exercises-2012) are available online for free. -* [Code Examples](http://spark-project.org/examples.html): more are also available in the [examples subfolder](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/examples) of Spark +* [Code Examples](http://spark.incubator.apache.org/examples.html): more are also available in the [examples subfolder](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/examples) of Spark * [Paper Describing Spark](http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf) * [Paper Describing Spark Streaming](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) # Community -To get help using Spark or keep up with Spark development, sign up for the [spark-users mailing list](http://groups.google.com/group/spark-users). +To get help using Spark or keep up with Spark development, sign up for the [user mailing list](http://spark.incubator.apache.org/mailing-lists.html). If you're in the San Francisco Bay Area, there's a regular [Spark meetup](http://www.meetup.com/spark-users/) every few weeks. Come by to meet the developers and other users. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 678cd57abac..fe5334ffdca 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -3,50 +3,33 @@ layout: global title: Launching Spark on YARN --- -Experimental support for running over a [YARN (Hadoop +Support for running on [YARN (Hadoop NextGen)](http://hadoop.apache.org/docs/r2.0.2-alpha/hadoop-yarn/hadoop-yarn-site/YARN.html) -cluster was added to Spark in version 0.6.0. This was merged into master as part of 0.7 effort. -To build spark with YARN support, please use the hadoop2-yarn profile. -Ex: mvn -Phadoop2-yarn clean install +was added to Spark in version 0.6.0, and improved in 0.7.0 and 0.8.0. -# Building spark core consolidated jar. +# Building a YARN-Enabled Assembly JAR -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. +We need a consolidated Spark JAR (which bundles all the required dependencies) to run Spark jobs on a YARN cluster. +This can be built by setting the Hadoop version and `SPARK_YARN` environment variable, as follows: -- Building spark assembled jar via sbt. -Enable YARN support by setting `SPARK_YARN=true` when invoking sbt: + SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true ./sbt/sbt assembly - SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true ./sbt/sbt clean assembly - -The assembled jar would typically be something like : -`./yarn/target/spark-yarn-assembly-0.8.0-SNAPSHOT.jar` - - -- Building spark assembled jar via Maven. - Use the hadoop2-yarn profile and execute the package target. - -Something like this. Ex: - - mvn -Phadoop2-yarn -Dhadoop.version=2.0.5-alpha clean package -DskipTests=true - - -This will build the shaded (consolidated) jar. Typically something like : -`./yarn/target/spark-yarn-bin--shaded.jar` +The assembled JAR will be something like this: +`./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly_{{site.SPARK_VERSION}}-hadoop2.0.5.jar`. # Preparations -- Building spark-yarn assembly (see above). +- Building a YARN-enabled assembly (see above). - Your application code must be packaged into a separate JAR file. -If you want to test out the YARN deployment mode, you can use the current Spark examples. A `spark-examples_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}` file can be generated by running `sbt/sbt package`. NOTE: since the documentation you're reading is for Spark version {{site.SPARK_VERSION}}, we are assuming here that you have downloaded Spark {{site.SPARK_VERSION}} or checked it out of source control. If you are using a different version of Spark, the version numbers in the jar generated by the sbt package command will obviously be different. +If you want to test out the YARN deployment mode, you can use the current Spark examples. A `spark-examples_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}` file can be generated by running `sbt/sbt assembly`. NOTE: since the documentation you're reading is for Spark version {{site.SPARK_VERSION}}, we are assuming here that you have downloaded Spark {{site.SPARK_VERSION}} or checked it out of source control. If you are using a different version of Spark, the version numbers in the jar generated by the sbt package command will obviously be different. # Configuration Most of the configs are the same for Spark on YARN as other deploys. See the Configuration page for more information on those. These are configs that are specific to SPARK on YARN. -* `SPARK_YARN_USER_ENV`, to add environment variables to the Spark processes launched on YARN. This can be a comma separated list of environment variables. ie SPARK_YARN_USER_ENV="JAVA_HOME=/jdk64,FOO=bar" +* `SPARK_YARN_USER_ENV`, to add environment variables to the Spark processes launched on YARN. This can be a comma separated list of environment variables, e.g. `SPARK_YARN_USER_ENV="JAVA_HOME=/jdk64,FOO=bar"`. # Launching Spark on YARN diff --git a/pyspark b/pyspark index 2dba2ceb21c..4941a36d0df 100755 --- a/pyspark +++ b/pyspark @@ -31,7 +31,7 @@ if [ ! -f "$FWDIR/RELEASE" ]; then ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null if [[ $? != 0 ]]; then echo "Failed to find Spark assembly in $FWDIR/assembly/target" >&2 - echo "You need to compile Spark before running this program" >&2 + echo "You need to build Spark with sbt/sbt assembly before running this program" >&2 exit 1 fi fi diff --git a/run-example b/run-example index ccd4356bdf1..24d83ba5cff 100755 --- a/run-example +++ b/run-example @@ -50,7 +50,7 @@ if [ -e "$EXAMPLES_DIR"/target/spark-examples*[0-9T].jar ]; then fi if [[ -z $SPARK_EXAMPLES_JAR ]]; then echo "Failed to find Spark examples assembly in $FWDIR/examples/target" >&2 - echo "You need to compile Spark before running this program" >&2 + echo "You need to build Spark with sbt/sbt assembly before running this program" >&2 exit 1 fi diff --git a/spark-class b/spark-class index 5ef3de97737..244b78b4e19 100755 --- a/spark-class +++ b/spark-class @@ -102,10 +102,10 @@ export JAVA_OPTS if [ ! -f "$FWDIR/RELEASE" ]; then # Exit if the user hasn't compiled Spark - ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*.jar >& /dev/null + ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null if [[ $? != 0 ]]; then echo "Failed to find Spark assembly in $FWDIR/assembly/target" >&2 - echo "You need to compile Spark before running this program" >&2 + echo "You need to build Spark with sbt/sbt assembly before running this program" >&2 exit 1 fi fi From bac46266a97a6096d6d772e023a3362fd48baac0 Mon Sep 17 00:00:00 2001 From: "Y.CORP.YAHOO.COM\\tgraves" Date: Fri, 30 Aug 2013 15:55:32 -0500 Subject: [PATCH 0883/2521] Link the Spark UI to the Yarn UI --- .../cluster/SparkDeploySchedulerBackend.scala | 2 +- core/src/main/scala/spark/ui/SparkUI.scala | 7 ++- core/src/main/scala/spark/ui/UIUtils.scala | 58 +++++++++++------ .../main/scala/spark/ui/jobs/PoolTable.scala | 3 +- .../main/scala/spark/ui/jobs/StageTable.scala | 5 +- .../scala/spark/ui/storage/IndexPage.scala | 2 +- docs/running-on-yarn.md | 4 ++ .../spark/deploy/yarn/ApplicationMaster.scala | 63 ++++++++++++++----- .../deploy/yarn/YarnAllocationHandler.scala | 10 ++- .../cluster/YarnClusterScheduler.scala | 2 + 10 files changed, 111 insertions(+), 45 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 42c3b4a6cf6..8bdc1469e62 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -49,7 +49,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.appUIAddress) + sc.ui.appHttpUIAddress) client = new Client(sc.env.actorSystem, master, appDesc, this) client.start() diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/spark/ui/SparkUI.scala index 23ded44ba38..e078c4a6b24 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/spark/ui/SparkUI.scala @@ -1,5 +1,4 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more +/* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with * this work for additional information regarding copyright ownership. * The ASF licenses this file to You under the Apache License, Version 2.0 @@ -78,7 +77,9 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { server.foreach(_.stop()) } - private[spark] def appUIAddress = "http://" + host + ":" + boundPort.getOrElse("-1") + private[spark] def appHttpUIAddress = "http://" + appUIAddress + private[spark] def appUIAddress = host + ":" + boundPort.getOrElse("-1") + } private[spark] object SparkUI { diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/spark/ui/UIUtils.scala index fe2afc11299..6b45679f9d5 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -25,33 +25,53 @@ import spark.SparkContext private[spark] object UIUtils { import Page._ + // Yarn has to go through a proxy so the base uri is provided and has to be on all links + private[spark] val uiRoot : String = Option(System.getenv("APPLICATION_WEB_PROXY_BASE")). + getOrElse("") + + def addBaseUri(resource: String = ""): String = { + return uiRoot + resource + } + + private[spark] val storageStr = addBaseUri("/storage") + private[spark] val stagesStr = addBaseUri("/stages") + private[spark] val envStr = addBaseUri("/environment") + private[spark] val executorsStr = addBaseUri("/executors") + private[spark] val bootstrapMinCssStr = addBaseUri("/static/bootstrap.min.css") + private[spark] val webuiCssStr = addBaseUri("/static/webui.css") + private[spark] val bootstrapResponsiveCssStr = addBaseUri("/static/bootstrap-responsive.min.css") + private[spark] val sortTableStr = addBaseUri("/static/sorttable.js") + private[spark] val sparkLogoHdStr = addBaseUri("/static/spark-logo-77x50px-hd.png") + private[spark] val sparkLogoStr = addBaseUri("/static/spark_logo.png") + + /** Returns a spark page with correctly formatted headers */ def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String, page: Page.Value) : Seq[Node] = { - val jobs = page match { - case Jobs =>
  • Jobs
  • - case _ =>
  • Jobs
  • - } val storage = page match { - case Storage =>
  • Storage
  • - case _ =>
  • Storage
  • + case Storage =>
  • Storage
  • + case _ =>
  • Storage
  • + } + val jobs = page match { + case Jobs =>
  • Jobs
  • + case _ =>
  • Jobs
  • } val environment = page match { - case Environment =>
  • Environment
  • - case _ =>
  • Environment
  • + case Environment =>
  • Environment
  • + case _ =>
  • Environment
  • } val executors = page match { - case Executors =>
  • Executors
  • - case _ =>
  • Executors
  • + case Executors =>
  • Executors
  • + case _ =>
  • Executors
  • } - - - - + + + + {sc.appName} - {title} -
    - -
    -
    - -
    + -
    +
    +

    {title}

    -
    {content}
    @@ -99,21 +87,16 @@ private[spark] object UIUtils { - + {title} - -
    -
    -
    - -
    -
    -

    +
    +
    +
    +

    + {title}

    diff --git a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala index b3e28ce317b..b1be1a27efe 100644 --- a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/spark/ui/env/EnvironmentUI.scala @@ -78,14 +78,11 @@ private[spark] class EnvironmentUI(sc: SparkContext) { val content =

    Runtime Information

    {jvmTable} -
    -

    {sparkProperties.size} Spark Properties

    +

    Spark Properties

    {sparkPropertyTable} -
    -

    {otherProperties.size} System Properties

    +

    System Properties

    {otherPropertyTable} -
    -

    {classPath.size} Classpath Entries

    +

    Classpath Entries

    {classPathTable}
    diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index f97860013e1..0a7021fbf86 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -62,7 +62,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val execTable = UIUtils.listingTable(execHead, execRow, execInfo) val content = -
    +
    • Memory: @@ -78,7 +78,7 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
    ; - UIUtils.headerSparkPage(content, sc, execInfo.size + " Executors", Executors) + UIUtils.headerSparkPage(content, sc, "Executors (" + execInfo.size + ")", Executors) } def getExecInfo(a: Int): Seq[String] = { diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/spark/ui/jobs/IndexPage.scala index cda6addd22a..8867a6c90c1 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/spark/ui/jobs/IndexPage.scala @@ -52,13 +52,9 @@ private[spark] class IndexPage(parent: JobProgressUI) {
    • - Duration: + Total Duration: {parent.formatDuration(now - listener.sc.startTime)}
    • -
    • - CPU Time: - {parent.formatDuration(listener.totalTime + activeTime)} -
    • Scheduling Mode: {parent.sc.getSchedulingMode}
    • Active Stages: @@ -77,18 +73,18 @@ private[spark] class IndexPage(parent: JobProgressUI) { val content = summary ++ {if (listener.sc.getSchedulingMode == SchedulingMode.FAIR) { -

      {pools.size} Fair Scheduler Pools

      ++ poolTable.toNodeSeq +

      {pools.size} Fair Scheduler Pools

      ++ poolTable.toNodeSeq } else { Seq() }} ++ -

      {activeStages.size} Active Stages

      ++ +

      Active Stages ({activeStages.size})

      ++ activeStagesTable.toNodeSeq++ -

      {completedStages.size} Completed Stages

      ++ +

      Completed Stages ({completedStages.size})

      ++ completedStagesTable.toNodeSeq++ -

      {failedStages.size} Failed Stages

      ++ +

      Failed Stages ({failedStages.size})

      ++ failedStagesTable.toNodeSeq - headerSparkPage(content, parent.sc, "Spark Stages", Jobs) + headerSparkPage(content, parent.sc, "Spark Stages", Stages) } } } diff --git a/core/src/main/scala/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/spark/ui/jobs/PoolPage.scala index e8f80ebfce7..7fb74dce406 100644 --- a/core/src/main/scala/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/spark/ui/jobs/PoolPage.scala @@ -24,10 +24,9 @@ private[spark] class PoolPage(parent: JobProgressUI) { val poolTable = new PoolTable(Seq(pool), listener) val content =

      Summary

      ++ poolTable.toNodeSeq() ++ -

      {activeStages.size} Active Stages

      ++ activeStagesTable.toNodeSeq() - headerSparkPage(content, parent.sc, "Fair Scheduler Pool: " + poolName, Jobs) + headerSparkPage(content, parent.sc, "Fair Scheduler Pool: " + poolName, Stages) } } } diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/spark/ui/jobs/StagePage.scala index 6948ea4dd90..c2341475c7f 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/spark/ui/jobs/StagePage.scala @@ -46,7 +46,7 @@ private[spark] class StagePage(parent: JobProgressUI) {

      Summary Metrics

      No tasks have started yet

      Tasks

      No tasks have started yet
    - return headerSparkPage(content, parent.sc, "Details for Stage %s".format(stageId), Jobs) + return headerSparkPage(content, parent.sc, "Details for Stage %s".format(stageId), Stages) } val tasks = listener.stageToTaskInfos(stageId).toSeq.sortBy(_._1.launchTime) @@ -133,9 +133,9 @@ private[spark] class StagePage(parent: JobProgressUI) { summary ++

    Summary Metrics for {numCompleted} Completed Tasks

    ++
    {summaryTable.getOrElse("No tasks have reported metrics yet.")}
    ++ -

    Tasks

    ++ taskTable; +

    Tasks

    ++ taskTable; - headerSparkPage(content, parent.sc, "Details for Stage %d".format(stageId), Jobs) + headerSparkPage(content, parent.sc, "Details for Stage %d".format(stageId), Stages) } } diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/spark/ui/jobs/StageTable.scala index b31f4abc26d..2b1bc984fca 100644 --- a/core/src/main/scala/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/spark/ui/jobs/StageTable.scala @@ -46,7 +46,7 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) val startWidth = "width: %s%%".format((started.toDouble/total)*100) -
    +
    {completed}/{total} {failed} diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/spark/ui/storage/IndexPage.scala index 0751f9e8f91..fc6273c6945 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/spark/ui/storage/IndexPage.scala @@ -38,13 +38,13 @@ private[spark] class IndexPage(parent: BlockManagerUI) { "RDD Name", "Storage Level", "Cached Partitions", - "Fraction Partitions Cached", + "Fraction Cached", "Size in Memory", "Size on Disk") val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) val content = listingTable(rddHeaders, rddRow, rdds) - headerSparkPage(content, parent.sc, "Spark Storage ", Storage) + headerSparkPage(content, parent.sc, "Storage ", Storage) } def rddRow(rdd: RDDInfo): Seq[Node] = { @@ -57,7 +57,7 @@ private[spark] class IndexPage(parent: BlockManagerUI) { {rdd.storageLevel.description} {rdd.numCachedPartitions} - {rdd.numCachedPartitions / rdd.numPartitions.toDouble} + {"%.0f%%".format(rdd.numCachedPartitions * 100.0 / rdd.numPartitions)} {Utils.bytesToString(rdd.memSize)} {Utils.bytesToString(rdd.diskSize)} diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/spark/ui/storage/RDDPage.scala index f0b711e6ecc..b128a5614dc 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/spark/ui/storage/RDDPage.scala @@ -55,7 +55,7 @@ private[spark] class RDDPage(parent: BlockManagerUI) { val blockTable = listingTable(blockHeaders, blockRow, blocks) val content = -
    +
    • @@ -81,15 +81,15 @@ private[spark] class RDDPage(parent: BlockManagerUI) {
    -
    -
    + +

    Data Distribution on {workers.size} Executors

    {workerTable}
    -
    -
    + +

    {blocks.size} Partitions

    {blockTable} From 250bddc255b719dd9a39b6bb3f1e60574579f983 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Sat, 31 Aug 2013 17:25:05 -0700 Subject: [PATCH 0891/2521] Don't require spark home to be set for standalone mode --- .../spark/scheduler/cluster/SparkDeploySchedulerBackend.scala | 3 +-- 1 file changed, 1 insertion(+), 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 42c3b4a6cf6..7ac574bdc8e 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -46,8 +46,7 @@ private[spark] class SparkDeploySchedulerBackend( StandaloneSchedulerBackend.ACTOR_NAME) val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}") 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 sparkHome = sc.getSparkHome().getOrElse(null) val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, sc.ui.appUIAddress) From 2ee6a7e32aaf6dbe81ea79bee1228bb09e48dd1a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 31 Aug 2013 17:31:07 -0700 Subject: [PATCH 0892/2521] Print output from spark-daemon only when it fails to launch --- bin/slaves.sh | 4 +--- bin/spark-daemon.sh | 16 +++++++++++++--- bin/start-slaves.sh | 2 -- bin/stop-all.sh | 1 + .../spark/deploy/worker/ui/WorkerWebUI.scala | 1 - 5 files changed, 15 insertions(+), 9 deletions(-) diff --git a/bin/slaves.sh b/bin/slaves.sh index c8fb5ca4736..752565b759f 100755 --- a/bin/slaves.sh +++ b/bin/slaves.sh @@ -42,7 +42,7 @@ bin=`cd "$bin"; pwd` . "$bin/spark-config.sh" # If the slaves file is specified in the command line, -# then it takes precedence over the definition in +# then it takes precedence over the definition in # spark-env.sh. Save it here. HOSTLIST=$SPARK_SLAVES @@ -58,8 +58,6 @@ if [ "$HOSTLIST" = "" ]; then fi fi -echo $"${@// /\\ }" - # By default disable strict host key checking if [ "$SPARK_SSH_OPTS" = "" ]; then SPARK_SSH_OPTS="-o StrictHostKeyChecking=no" diff --git a/bin/spark-daemon.sh b/bin/spark-daemon.sh index f01ff84d95a..5bfe967fbfa 100755 --- a/bin/spark-daemon.sh +++ b/bin/spark-daemon.sh @@ -75,6 +75,9 @@ if [ "$SPARK_IDENT_STRING" = "" ]; then export SPARK_IDENT_STRING="$USER" fi + +export SPARK_PRINT_LAUNCH_COMMAND="1" + # get log directory if [ "$SPARK_LOG_DIR" = "" ]; then export SPARK_LOG_DIR="$SPARK_HOME/logs" @@ -122,12 +125,19 @@ case $startStop in rsync -a -e ssh --delete --exclude=.svn --exclude='logs/*' --exclude='contrib/hod/logs/*' $SPARK_MASTER/ "$SPARK_HOME" fi - spark_rotate_log $log + spark_rotate_log "$log" echo starting $command, logging to $log cd "$SPARK_PREFIX" nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & - echo $! > $pid - sleep 1; head "$log" + newpid=$! + echo $newpid > $pid + sleep 2 + # Check if the process has died; in that case we'll tail the log so the user can see + if ! kill -0 $newpid >/dev/null 2>&1; then + echo "failed to launch $command:" + tail -2 "$log" | sed 's/^/ /' + echo "full log in $log" + fi ;; (stop) diff --git a/bin/start-slaves.sh b/bin/start-slaves.sh index dad7c3df765..00dc4888b2e 100755 --- a/bin/start-slaves.sh +++ b/bin/start-slaves.sh @@ -35,8 +35,6 @@ if [ "$SPARK_MASTER_IP" = "" ]; then SPARK_MASTER_IP=`hostname` fi -echo "Master IP: $SPARK_MASTER_IP" - # Launch the slaves if [ "$SPARK_WORKER_INSTANCES" = "" ]; then exec "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/start-slave.sh" 1 spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT diff --git a/bin/stop-all.sh b/bin/stop-all.sh index a043ac0095b..b6c83a7ba4c 100755 --- a/bin/stop-all.sh +++ b/bin/stop-all.sh @@ -20,6 +20,7 @@ # Start all spark daemons. # Run this on the master nde + bin=`dirname "$0"` bin=`cd "$bin"; pwd` 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 717619f80d1..0a75ad8cf4a 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala @@ -143,7 +143,6 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I {linkToMaster} -
    {backButton}
    {range}
    From 9ddad0dcb47e3326151a53e270448b5135805ae5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 31 Aug 2013 17:40:33 -0700 Subject: [PATCH 0893/2521] Fixes suggested by Patrick --- conf/spark-env.sh.template | 2 +- docs/hardware-provisioning.md | 1 - docs/index.md | 9 +++++---- docs/quick-start.md | 10 ++-------- 4 files changed, 8 insertions(+), 14 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index a367d59d64c..d92d2e2ae31 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -4,7 +4,7 @@ # spark-env.sh and edit that to configure Spark for your site. # # The following variables can be set in this file: -# - SPARK_LOCAL_IP, to override the IP address binds to +# - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node # - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos # - SPARK_JAVA_OPTS, to set node-specific JVM options for Spark. Note that # we recommend setting app-wide options in the application's driver program. diff --git a/docs/hardware-provisioning.md b/docs/hardware-provisioning.md index d21e2a3d703..e5f054cb14b 100644 --- a/docs/hardware-provisioning.md +++ b/docs/hardware-provisioning.md @@ -21,7 +21,6 @@ Hadoop and Spark on a common cluster manager like [Mesos](running-on-mesos.html) [Hadoop YARN](running-on-yarn.html). * If this is not possible, run Spark on different nodes in the same local-area network as HDFS. -If your cluster spans multiple racks, include some Spark nodes on each rack. * For low-latency data stores like HBase, it may be preferrable to run computing jobs on different nodes than the storage system to avoid interference. diff --git a/docs/index.md b/docs/index.md index bcd7dad6ae8..0ea0e103e43 100644 --- a/docs/index.md +++ b/docs/index.md @@ -40,12 +40,13 @@ Python interpreter (`./pyspark`). These are a great way to learn Spark. Spark uses the Hadoop-client library to talk to HDFS and other Hadoop-supported storage systems. Because the HDFS protocol has changed in different versions of Hadoop, you must build Spark against the same version that your cluster uses. -You can do this by setting the `SPARK_HADOOP_VERSION` variable when compiling: +By default, Spark links to Hadoop 1.0.4. You can change this by setting the +`SPARK_HADOOP_VERSION` variable when compiling: SPARK_HADOOP_VERSION=1.2.1 sbt/sbt assembly -In addition, if you wish to run Spark on [YARN](running-on-yarn.md), you should also -set `SPARK_YARN`: +In addition, if you wish to run Spark on [YARN](running-on-yarn.md), set +`SPARK_YARN` to `true`: SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly @@ -94,7 +95,7 @@ set `SPARK_YARN`: exercises about Spark, Shark, Mesos, and more. [Videos](http://ampcamp.berkeley.edu/agenda-2012), [slides](http://ampcamp.berkeley.edu/agenda-2012) and [exercises](http://ampcamp.berkeley.edu/exercises-2012) are available online for free. -* [Code Examples](http://spark.incubator.apache.org/examples.html): more are also available in the [examples subfolder](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/examples) of Spark +* [Code Examples](http://spark.incubator.apache.org/examples.html): more are also available in the [examples subfolder](https://github.com/mesos/spark/tree/master/examples/src/main/scala/) of Spark * [Paper Describing Spark](http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf) * [Paper Describing Spark Streaming](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) diff --git a/docs/quick-start.md b/docs/quick-start.md index bac5d690a69..11d4370a1d5 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -126,7 +126,7 @@ object SimpleJob { This job simply counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the job. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the job, the directory where Spark is installed, and a name for the jar file containing the job's sources. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes. -This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` which explains that Spark is a dependency. This file also adds two repositories which host Spark dependencies: +This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` which explains that Spark is a dependency. This file also adds a repository that Spark depends on: {% highlight scala %} name := "Simple Project" @@ -137,9 +137,7 @@ scalaVersion := "{{site.SCALA_VERSION}}" libraryDependencies += "org.spark-project" %% "spark-core" % "{{site.SPARK_VERSION}}" -resolvers ++= Seq( - "Akka Repository" at "http://repo.akka.io/releases/", - "Spray Repository" at "http://repo.spray.cc/") +resolvers += "Akka Repository" at "http://repo.akka.io/releases/" {% endhighlight %} If you also wish to read data from Hadoop's HDFS, you will also need to add a dependency on `hadoop-client` for your version of HDFS: @@ -210,10 +208,6 @@ To build the job, we also write a Maven `pom.xml` file that lists Spark as a dep jar 1.0 - - Spray.cc repository - http://repo.spray.cc - Akka repository http://repo.akka.io/releases From 7862c4a3c8b900db81f5c2af157bd4564d814bd9 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 31 Aug 2013 17:41:47 -0700 Subject: [PATCH 0894/2521] Another fix suggested by Patrick --- conf/spark-env.sh.template | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index d92d2e2ae31..0a35ee7c79b 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -12,7 +12,7 @@ # Examples of app-wide options : -Dspark.serializer # # If using the standalone deploy mode, you can also set variables for it here: -# - SPARK_MASTER_IP, to bind the master to a different IP address +# - SPARK_MASTER_IP, to bind the master to a different IP address or hostname # - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports # - 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) From e34bc3a8eecdb0e28a248fb5f7926f730fab736f Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 31 Aug 2013 17:47:15 -0700 Subject: [PATCH 0895/2521] Small tweak --- 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 719c5b5940b..51bb18d888d 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/spark/ui/UIUtils.scala @@ -96,7 +96,7 @@ private[spark] object UIUtils {

    - + {title}

    From 2c5a4b89ee034b7933b258cfc37bc6d91a06c186 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 31 Aug 2013 18:08:05 -0700 Subject: [PATCH 0896/2521] Small fixes to README --- README.md | 42 ++++++++++++++++-------------------------- 1 file changed, 16 insertions(+), 26 deletions(-) diff --git a/README.md b/README.md index 89b5a0abfd7..2ddfe862a23 100644 --- a/README.md +++ b/README.md @@ -1,12 +1,12 @@ -# Spark +# Apache Spark -Lightning-Fast Cluster Computing - +Lightning-Fast Cluster Computing - ## Online Documentation You can find the latest Spark documentation, including a programming -guide, on the project webpage at . +guide, on the project webpage at . This README file only contains basic setup instructions. @@ -18,16 +18,14 @@ Spark and its example programs, run: sbt/sbt assembly -Spark also supports building using Maven. If you would like to build using Maven, -see the [instructions for building Spark with Maven](http://spark-project.org/docs/latest/building-with-maven.html) -in the Spark documentation.. +Once you've built Spark, the easiest way to start using it is the shell: -To run Spark, you will need to have Scala's bin directory in your `PATH`, or -you will need to set the `SCALA_HOME` environment variable to point to where -you've installed Scala. Scala must be accessible through one of these -methods on your cluster's worker nodes as well as its master. + ./spark-shell -To run one of the examples, use `./run-example `. For example: +Or, for the Python API, the Python shell (`./pyspark`). + +Spark also comes with several sample programs in the `examples` directory. +To run one of them, use `./run-example `. For example: ./run-example spark.examples.SparkLR local[2] @@ -35,7 +33,7 @@ will run the Logistic Regression example locally on 2 CPUs. Each of the example programs prints usage help if no params are given. -All of the Spark samples take a `` parameter that is the cluster URL +All of the Spark samples take a `` parameter that is the cluster URL to connect to. This can be a mesos:// or spark:// URL, or "local" to run locally with one thread, or "local[N]" to run locally with N threads. @@ -58,13 +56,13 @@ versions without YARN, use: $ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt/sbt assembly For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions -with YARN, also set `SPARK_WITH_YARN=true`: +with YARN, also set `SPARK_YARN=true`: # Apache Hadoop 2.0.5-alpha - $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_WITH_YARN=true sbt/sbt assembly + $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly # Cloudera CDH 4.2.0 with MapReduce v2 - $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_WITH_YARN=true sbt/sbt assembly + $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_YARN=true sbt/sbt assembly For convenience, these variables may also be set through the `conf/spark-env.sh` file described below. @@ -81,22 +79,14 @@ If your project is built with Maven, add this to your POM file's ` org.apache.hadoop hadoop-client - - [1.2.1] + 1.2.1 ## Configuration -Please refer to the "Configuration" guide in the online documentation for a -full overview on how to configure Spark. At the minimum, you will need to -create a `conf/spark-env.sh` script (copy `conf/spark-env.sh.template`) and -set the following two variables: - -- `SCALA_HOME`: Location where Scala is installed. - -- `MESOS_NATIVE_LIBRARY`: Your Mesos library (only needed if you want to run - on Mesos). For example, this might be `/usr/local/lib/libmesos.so` on Linux. +Please refer to the [Configuration guide](http://spark.incubator.apache.org/docs/latest/configuration.html) +in the online documentation for an overview on how to configure Spark. ## Contributing to Spark From 6371febe189a6960401afd82985a793bf6c96a97 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 31 Aug 2013 17:28:07 -0700 Subject: [PATCH 0897/2521] Better docs --- docs/quick-start.md | 36 ++++++++++++++++++++++++++++-------- 1 file changed, 28 insertions(+), 8 deletions(-) diff --git a/docs/quick-start.md b/docs/quick-start.md index 11d4370a1d5..8cf4156f136 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -53,7 +53,7 @@ scala> textFile.filter(line => line.contains("Spark")).count() // How many lines res3: Long = 15 {% endhighlight %} -## More On RDD Operations +## More on RDD Operations RDD actions and transformations can be used for more complex computations. Let's say we want to find the line with the most words: {% highlight scala %} @@ -163,8 +163,6 @@ $ sbt run Lines with a: 46, Lines with b: 23 {% endhighlight %} -This example only runs the job locally; for a tutorial on running jobs across several machines, see the [Standalone Mode](spark-standalone.html) documentation, and consider using a distributed input source, such as HDFS. - # A Standalone Job In Java Now say we wanted to write a standalone job using the Java API. We will walk through doing this with Maven. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide. @@ -252,8 +250,6 @@ $ mvn exec:java -Dexec.mainClass="SimpleJob" Lines with a: 46, Lines with b: 23 {% endhighlight %} -This example only runs the job locally; for a tutorial on running jobs across several machines, see the [Standalone Mode](spark-standalone.html) documentation, and consider using a distributed input source, such as HDFS. - # A Standalone Job In Python Now we will show how to write a standalone job using the Python API (PySpark). @@ -290,6 +286,30 @@ $ ./pyspark SimpleJob.py Lines with a: 46, Lines with b: 23 {% endhighlight python %} -This example only runs the job locally; for a tutorial on running jobs across several machines, see the [Standalone Mode](spark-standalone.html) documentation, and consider using a distributed input source, such as HDFS. - -Also, this example links against the default version of HDFS that Spark builds with (1.0.4). You can run it against other HDFS versions by [building Spark with another HDFS version](index.html#a-note-about-hadoop-versions). +# Running Jobs on a Cluster + +There are a few additional considerations when running jobs on a +[Spark](spark-standalone.html), [YARN](running-on-yarn.html), or +[Mesos](running-on-mesos.html) cluster. + +### Including Your Dependencies +If your code depends on other projects, you will need to ensure they are also +present on the slave nodes. The most common way to do this is to create an +assembly jar (or "uber" jar) containing your code and its dependencies. You +may then submit the assembly jar when creating a SparkContext object. If you +do this, you should make Spark itself a `provided` dependency, since it will +already be present on the slave nodes. It is also possible to submit your +dependent jars one-by-one when creating a SparkContext. + +### Setting Configuration Options +Spark includes several configuration options which influence the behavior +of your job. These should be set as +[JVM system properties](configuration.html#system-properties) in your +program. The options will be captured and shipped to all slave nodes. + +### Accessing Hadoop Filesystems + +The examples here access a local file. To read data from a distributed +filesystem, such as HDFS, include +[Hadoop version information](index.html#a-note-about-hadoop-versions) +in your build file. By default, Spark builds against HDFS 1.0.4. From 0e375a3cc280880c430da00838b7231b942f96d4 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 1 Sep 2013 09:38:56 -0700 Subject: [PATCH 0898/2521] Add assmebly plug in links --- docs/quick-start.md | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/docs/quick-start.md b/docs/quick-start.md index 8cf4156f136..4507b21c5ed 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -294,12 +294,15 @@ There are a few additional considerations when running jobs on a ### Including Your Dependencies If your code depends on other projects, you will need to ensure they are also -present on the slave nodes. The most common way to do this is to create an -assembly jar (or "uber" jar) containing your code and its dependencies. You -may then submit the assembly jar when creating a SparkContext object. If you -do this, you should make Spark itself a `provided` dependency, since it will -already be present on the slave nodes. It is also possible to submit your -dependent jars one-by-one when creating a SparkContext. +present on the slave nodes. A popular approach is to create an +assembly jar (or "uber" jar) containing your code and its dependencies. Both +[sbt](https://github.com/sbt/sbt-assembly) and +[Maven](http://maven.apache.org/plugins/maven-assembly-plugin/) +have assembly plugins. When creating assembly jars, list Spark +itself as a `provided` dependency; it need not be bundled since it is +already present on the slaves. Once you have an assembled jar, +add it to the SparkContext as shown here. It is also possible to submit +your dependent jars one-by-one when creating a SparkContext. ### Setting Configuration Options Spark includes several configuration options which influence the behavior From 46eecd110a4017ea0c86cbb1010d0ccd6a5eb2ef Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 31 Aug 2013 19:27:07 -0700 Subject: [PATCH 0899/2521] Initial work to rename package to org.apache.spark --- README.md | 2 +- assembly/pom.xml | 16 +- assembly/src/main/assembly/assembly.xml | 10 +- bagel/pom.xml | 6 +- .../{ => org/apache}/spark/bagel/Bagel.scala | 35 ++-- .../apache/spark}/bagel/BagelSuite.scala | 10 +- bin/start-master.sh | 2 +- bin/start-slave.sh | 2 +- bin/stop-master.sh | 2 +- bin/stop-slaves.sh | 4 +- core/pom.xml | 4 +- .../spark/network/netty/FileClient.java | 2 +- .../netty/FileClientChannelInitializer.java | 2 +- .../network/netty/FileClientHandler.java | 2 +- .../spark/network/netty/FileServer.java | 2 +- .../netty/FileServerChannelInitializer.java | 2 +- .../network/netty/FileServerHandler.java | 2 +- .../spark/network/netty/PathResolver.java | 2 +- .../apache}/spark/ui/static/bootstrap.min.css | 0 .../apache}/spark/ui/static/sorttable.js | 0 .../spark/ui/static/spark-logo-77x50px-hd.png | Bin .../apache}/spark/ui/static/spark_logo.png | Bin .../apache}/spark/ui/static/webui.css | 0 .../{ => org/apache}/spark/Accumulators.scala | 10 +- .../{ => org/apache}/spark/Aggregator.scala | 2 +- .../spark/BlockStoreShuffleFetcher.scala | 10 +- .../{ => org/apache}/spark/CacheManager.scala | 4 +- .../apache}/spark/ClosureCleaner.scala | 2 +- .../{ => org/apache}/spark/Dependency.scala | 2 +- .../apache}/spark/DoubleRDDFunctions.scala | 14 +- .../apache}/spark/FetchFailedException.scala | 4 +- .../apache}/spark/HttpFileServer.scala | 2 +- .../{ => org/apache}/spark/HttpServer.scala | 2 +- .../apache}/spark/JavaSerializer.scala | 4 +- .../apache}/spark/KryoSerializer.scala | 10 +- .../{ => org/apache}/spark/Logging.scala | 2 +- .../apache}/spark/MapOutputTracker.scala | 8 +- .../apache}/spark/PairRDDFunctions.scala | 16 +- .../{ => org/apache}/spark/Partition.scala | 2 +- .../{ => org/apache}/spark/Partitioner.scala | 6 +- .../scala/{ => org/apache}/spark/RDD.scala | 62 +++--- .../apache}/spark/RDDCheckpointData.scala | 2 +- .../spark/SequenceFileRDDFunctions.scala | 4 +- .../apache}/spark/SerializableWritable.scala | 2 +- .../apache}/spark/ShuffleFetcher.scala | 6 +- .../apache}/spark/SizeEstimator.scala | 2 +- .../{ => org/apache}/spark/SparkContext.scala | 34 ++-- .../{ => org/apache}/spark/SparkEnv.scala | 27 ++- .../apache}/spark/SparkException.scala | 2 +- .../{ => org/apache}/spark/SparkFiles.java | 2 +- .../apache}/spark/SparkHadoopWriter.scala | 4 +- .../{ => org/apache}/spark/TaskContext.scala | 2 +- .../apache}/spark/TaskEndReason.scala | 6 +- .../{ => org/apache}/spark/TaskState.scala | 2 +- .../scala/{ => org/apache}/spark/Utils.scala | 6 +- .../spark/api/java/JavaDoubleRDD.scala | 20 +- .../apache}/spark/api/java/JavaPairRDD.scala | 24 +-- .../apache}/spark/api/java/JavaRDD.scala | 8 +- .../apache}/spark/api/java/JavaRDDLike.scala | 14 +- .../spark/api/java/JavaSparkContext.scala | 26 +-- .../JavaSparkContextVarargsWorkaround.java | 2 +- .../apache}/spark/api/java/JavaUtils.scala | 2 +- .../apache}/spark/api/java/StorageLevels.java | 4 +- .../java/function/DoubleFlatMapFunction.java | 2 +- .../api/java/function/DoubleFunction.java | 2 +- .../api/java/function/FlatMapFunction.scala | 2 +- .../api/java/function/FlatMapFunction2.scala | 2 +- .../spark/api/java/function/Function.java | 2 +- .../spark/api/java/function/Function2.java | 2 +- .../java/function/PairFlatMapFunction.java | 2 +- .../spark/api/java/function/PairFunction.java | 2 +- .../api/java/function/VoidFunction.scala | 2 +- .../api/java/function/WrappedFunction1.scala | 2 +- .../api/java/function/WrappedFunction2.scala | 2 +- .../spark/api/python/PythonPartitioner.scala | 8 +- .../apache}/spark/api/python/PythonRDD.scala | 12 +- .../api/python/PythonWorkerFactory.scala | 4 +- .../spark/broadcast/BitTorrentBroadcast.scala | 6 +- .../apache}/spark/broadcast/Broadcast.scala | 8 +- .../spark/broadcast/BroadcastFactory.scala | 2 +- .../spark/broadcast/HttpBroadcast.scala | 10 +- .../spark/broadcast/MultiTracker.scala | 4 +- .../apache}/spark/broadcast/SourceInfo.scala | 4 +- .../spark/broadcast/TreeBroadcast.scala | 6 +- .../spark/deploy/ApplicationDescription.scala | 2 +- .../apache}/spark/deploy/Command.scala | 2 +- .../apache}/spark/deploy/DeployMessage.scala | 10 +- .../apache}/spark/deploy/ExecutorState.scala | 2 +- .../apache}/spark/deploy/JsonProtocol.scala | 8 +- .../spark/deploy/LocalSparkCluster.scala | 10 +- .../spark/deploy/SparkHadoopUtil.scala | 2 +- .../{ => org/apache}/spark/deploy/WebUI.scala | 2 +- .../apache}/spark/deploy/client/Client.scala | 10 +- .../spark/deploy/client/ClientListener.scala | 2 +- .../spark/deploy/client/TestClient.scala | 8 +- .../spark/deploy/client/TestExecutor.scala | 2 +- .../spark/deploy/master/ApplicationInfo.scala | 4 +- .../deploy/master/ApplicationSource.scala | 4 +- .../deploy/master/ApplicationState.scala | 2 +- .../spark/deploy/master/ExecutorInfo.scala | 4 +- .../apache}/spark/deploy/master/Master.scala | 14 +- .../spark/deploy/master/MasterArguments.scala | 6 +- .../spark/deploy/master/MasterSource.scala | 4 +- .../spark/deploy/master/WorkerInfo.scala | 4 +- .../spark/deploy/master/WorkerState.scala | 2 +- .../deploy/master/ui/ApplicationPage.scala | 12 +- .../spark/deploy/master/ui/IndexPage.scala | 14 +- .../spark/deploy/master/ui/MasterWebUI.scala | 12 +- .../spark/deploy/worker/ExecutorRunner.scala | 8 +- .../apache}/spark/deploy/worker/Worker.scala | 16 +- .../spark/deploy/worker/WorkerArguments.scala | 8 +- .../spark/deploy/worker/WorkerSource.scala | 4 +- .../spark/deploy/worker/ui/IndexPage.scala | 12 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 14 +- .../apache}/spark/executor/Executor.scala | 10 +- .../spark/executor/ExecutorBackend.scala | 4 +- .../spark/executor/ExecutorExitCode.scala | 2 +- .../spark/executor/ExecutorSource.scala | 4 +- .../executor/ExecutorURLClassLoader.scala | 2 +- .../spark/executor/MesosExecutorBackend.scala | 8 +- .../executor/StandaloneExecutorBackend.scala | 10 +- .../apache}/spark/executor/TaskMetrics.scala | 2 +- .../apache}/spark/io/CompressionCodec.scala | 6 +- .../apache}/spark/metrics/MetricsConfig.scala | 6 +- .../apache}/spark/metrics/MetricsSystem.scala | 8 +- .../spark/metrics/sink/ConsoleSink.scala | 4 +- .../apache}/spark/metrics/sink/CsvSink.scala | 4 +- .../apache}/spark/metrics/sink/JmxSink.scala | 2 +- .../spark/metrics/sink/MetricsServlet.scala | 4 +- .../apache}/spark/metrics/sink/Sink.scala | 4 +- .../spark/metrics/source/JvmSource.scala | 2 +- .../apache}/spark/metrics/source/Source.scala | 2 +- .../apache}/spark/network/BufferMessage.scala | 4 +- .../apache}/spark/network/Connection.scala | 4 +- .../spark/network/ConnectionManager.scala | 4 +- .../spark/network/ConnectionManagerId.scala | 4 +- .../spark/network/ConnectionManagerTest.scala | 6 +- .../apache}/spark/network/Message.scala | 2 +- .../apache}/spark/network/MessageChunk.scala | 2 +- .../spark/network/MessageChunkHeader.scala | 2 +- .../apache}/spark/network/ReceiverTest.scala | 2 +- .../apache}/spark/network/SenderTest.scala | 2 +- .../spark/network/netty/FileHeader.scala | 4 +- .../spark/network/netty/ShuffleCopier.scala | 6 +- .../spark/network/netty/ShuffleSender.scala | 4 +- .../{ => org/apache}/spark/package.scala | 12 +- .../partial/ApproximateActionListener.scala | 6 +- .../spark/partial/ApproximateEvaluator.scala | 2 +- .../apache}/spark/partial/BoundedDouble.scala | 2 +- .../spark/partial/CountEvaluator.scala | 2 +- .../spark/partial/GroupedCountEvaluator.scala | 2 +- .../spark/partial/GroupedMeanEvaluator.scala | 4 +- .../spark/partial/GroupedSumEvaluator.scala | 4 +- .../apache}/spark/partial/MeanEvaluator.scala | 4 +- .../apache}/spark/partial/PartialResult.scala | 2 +- .../spark/partial/StudentTCacher.scala | 2 +- .../apache}/spark/partial/SumEvaluator.scala | 4 +- .../{ => org/apache}/spark/rdd/BlockRDD.scala | 6 +- .../apache}/spark/rdd/CartesianRDD.scala | 4 +- .../apache}/spark/rdd/CheckpointRDD.scala | 6 +- .../apache}/spark/rdd/CoGroupedRDD.scala | 6 +- .../apache}/spark/rdd/CoalescedRDD.scala | 4 +- .../{ => org/apache}/spark/rdd/EmptyRDD.scala | 4 +- .../apache}/spark/rdd/FilteredRDD.scala | 4 +- .../apache}/spark/rdd/FlatMappedRDD.scala | 4 +- .../spark/rdd/FlatMappedValuesRDD.scala | 4 +- .../apache}/spark/rdd/GlommedRDD.scala | 4 +- .../apache}/spark/rdd/HadoopRDD.scala | 6 +- .../{ => org/apache}/spark/rdd/JdbcRDD.scala | 6 +- .../apache}/spark/rdd/MapPartitionsRDD.scala | 4 +- .../spark/rdd/MapPartitionsWithIndexRDD.scala | 4 +- .../apache}/spark/rdd/MappedRDD.scala | 4 +- .../apache}/spark/rdd/MappedValuesRDD.scala | 4 +- .../apache}/spark/rdd/NewHadoopRDD.scala | 4 +- .../spark/rdd/OrderedRDDFunctions.scala | 4 +- .../spark/rdd/ParallelCollectionRDD.scala | 4 +- .../spark/rdd/PartitionPruningRDD.scala | 4 +- .../{ => org/apache}/spark/rdd/PipedRDD.scala | 6 +- .../apache}/spark/rdd/SampledRDD.scala | 4 +- .../apache}/spark/rdd/ShuffledRDD.scala | 4 +- .../apache}/spark/rdd/SubtractedRDD.scala | 18 +- .../{ => org/apache}/spark/rdd/UnionRDD.scala | 4 +- .../spark/rdd/ZippedPartitionsRDD.scala | 4 +- .../apache}/spark/rdd/ZippedRDD.scala | 4 +- .../apache}/spark/scheduler/ActiveJob.scala | 4 +- .../spark/scheduler/DAGScheduler.scala | 14 +- .../spark/scheduler/DAGSchedulerEvent.scala | 8 +- .../spark/scheduler/DAGSchedulerSource.scala | 4 +- .../spark/scheduler/InputFormatInfo.scala | 4 +- .../apache}/spark/scheduler/JobListener.scala | 2 +- .../apache}/spark/scheduler/JobLogger.scala | 8 +- .../apache}/spark/scheduler/JobResult.scala | 2 +- .../apache}/spark/scheduler/JobWaiter.scala | 2 +- .../apache}/spark/scheduler/MapStatus.scala | 4 +- .../apache}/spark/scheduler/ResultTask.scala | 4 +- .../spark/scheduler/ShuffleMapTask.scala | 10 +- .../spark/scheduler/SparkListener.scala | 12 +- .../spark/scheduler/SparkListenerBus.scala | 4 +- .../apache}/spark/scheduler/SplitInfo.scala | 2 +- .../apache}/spark/scheduler/Stage.scala | 6 +- .../apache}/spark/scheduler/StageInfo.scala | 6 +- .../apache}/spark/scheduler/Task.scala | 8 +- .../spark/scheduler/TaskLocation.scala | 2 +- .../apache}/spark/scheduler/TaskResult.scala | 6 +- .../spark/scheduler/TaskScheduler.scala | 6 +- .../scheduler/TaskSchedulerListener.scala | 8 +- .../apache}/spark/scheduler/TaskSet.scala | 2 +- .../scheduler/cluster/ClusterScheduler.scala | 10 +- .../cluster/ClusterTaskSetManager.scala | 18 +- .../cluster/ExecutorLossReason.scala | 4 +- .../spark/scheduler/cluster/Pool.scala | 6 +- .../spark/scheduler/cluster/Schedulable.scala | 4 +- .../cluster/SchedulableBuilder.scala | 6 +- .../scheduler/cluster/SchedulerBackend.scala | 4 +- .../cluster/SchedulingAlgorithm.scala | 2 +- .../scheduler/cluster/SchedulingMode.scala | 2 +- .../cluster/SparkDeploySchedulerBackend.scala | 11 +- .../cluster/StandaloneClusterMessage.scala | 8 +- .../cluster/StandaloneSchedulerBackend.scala | 6 +- .../scheduler/cluster/TaskDescription.scala | 4 +- .../spark/scheduler/cluster/TaskInfo.scala | 4 +- .../scheduler/cluster/TaskLocality.scala | 2 +- .../scheduler/cluster/TaskSetManager.scala | 6 +- .../spark/scheduler/cluster/WorkerOffer.scala | 2 +- .../scheduler/local/LocalScheduler.scala | 14 +- .../scheduler/local/LocalTaskSetManager.scala | 10 +- .../mesos/CoarseMesosSchedulerBackend.scala | 18 +- .../mesos/MesosSchedulerBackend.scala | 8 +- .../apache}/spark/serializer/Serializer.scala | 8 +- .../spark/serializer/SerializerManager.scala | 2 +- .../spark/storage/BlockException.scala | 2 +- .../spark/storage/BlockFetchTracker.scala | 2 +- .../spark/storage/BlockFetcherIterator.scala | 16 +- .../apache}/spark/storage/BlockManager.scala | 12 +- .../spark/storage/BlockManagerId.scala | 8 +- .../spark/storage/BlockManagerMaster.scala | 6 +- .../storage/BlockManagerMasterActor.scala | 6 +- .../spark/storage/BlockManagerMessages.scala | 2 +- .../storage/BlockManagerSlaveActor.scala | 4 +- .../spark/storage/BlockManagerSource.scala | 4 +- .../spark/storage/BlockManagerWorker.scala | 6 +- .../apache}/spark/storage/BlockMessage.scala | 4 +- .../spark/storage/BlockMessageArray.scala | 6 +- .../spark/storage/BlockObjectWriter.scala | 2 +- .../apache}/spark/storage/BlockStore.scala | 4 +- .../apache}/spark/storage/DiskStore.scala | 14 +- .../apache}/spark/storage/MemoryStore.scala | 4 +- .../apache}/spark/storage/PutResult.scala | 2 +- .../spark/storage/ShuffleBlockManager.scala | 4 +- .../apache}/spark/storage/StorageLevel.scala | 4 +- .../apache}/spark/storage/StorageUtils.scala | 4 +- .../apache}/spark/storage/ThreadingTest.scala | 4 +- .../apache}/spark/ui/JettyUtils.scala | 4 +- .../{ => org/apache}/spark/ui/Page.scala | 2 +- .../{ => org/apache}/spark/ui/SparkUI.scala | 16 +- .../{ => org/apache}/spark/ui/UIUtils.scala | 4 +- .../spark/ui/UIWorkloadGenerator.scala | 8 +- .../apache}/spark/ui/env/EnvironmentUI.scala | 10 +- .../apache}/spark/ui/exec/ExecutorsUI.scala | 16 +- .../apache}/spark/ui/jobs/IndexPage.scala | 8 +- .../spark/ui/jobs/JobProgressListener.scala | 10 +- .../apache}/spark/ui/jobs/JobProgressUI.scala | 12 +- .../apache}/spark/ui/jobs/PoolPage.scala | 8 +- .../apache}/spark/ui/jobs/PoolTable.scala | 6 +- .../apache}/spark/ui/jobs/StagePage.scala | 14 +- .../apache}/spark/ui/jobs/StageTable.scala | 8 +- .../spark/ui/storage/BlockManagerUI.scala | 6 +- .../apache}/spark/ui/storage/IndexPage.scala | 10 +- .../apache}/spark/ui/storage/RDDPage.scala | 12 +- .../apache}/spark/util/AkkaUtils.scala | 2 +- .../spark/util/BoundedPriorityQueue.scala | 2 +- .../spark/util/ByteBufferInputStream.scala | 4 +- .../{ => org/apache}/spark/util/Clock.scala | 2 +- .../spark/util/CompletionIterator.scala | 2 +- .../apache}/spark/util/Distribution.scala | 2 +- .../apache}/spark/util/IdGenerator.scala | 2 +- .../apache}/spark/util/IntParam.scala | 2 +- .../apache}/spark/util/MemoryParam.scala | 4 +- .../apache}/spark/util/MetadataCleaner.scala | 4 +- .../apache}/spark/util/MutablePair.scala | 2 +- .../apache}/spark/util/NextIterator.scala | 2 +- .../spark/util/RateLimitedOutputStream.scala | 2 +- .../spark/util/SerializableBuffer.scala | 2 +- .../apache}/spark/util/StatCounter.scala | 2 +- .../spark/util/TimeStampedHashMap.scala | 7 +- .../spark/util/TimeStampedHashSet.scala | 2 +- .../{ => org/apache}/spark/util/Vector.scala | 4 +- .../resources/test_metrics_config.properties | 2 +- .../resources/test_metrics_system.properties | 6 +- .../apache}/spark/AccumulatorSuite.scala | 4 +- .../apache}/spark/BroadcastSuite.scala | 2 +- .../apache}/spark/CheckpointSuite.scala | 6 +- .../apache}/spark/ClosureCleanerSuite.scala | 4 +- .../apache}/spark/DistributedSuite.scala | 2 +- .../{ => org/apache}/spark/DriverSuite.scala | 4 +- .../{ => org/apache}/spark/FailureSuite.scala | 2 +- .../apache}/spark/FileServerSuite.scala | 2 +- .../{ => org/apache}/spark/FileSuite.scala | 2 +- .../{ => org/apache}/spark/JavaAPISuite.java | 20 +- .../apache}/spark/KryoSerializerSuite.scala | 4 +- .../apache}/spark/LocalSparkContext.scala | 2 +- .../apache}/spark/MapOutputTrackerSuite.scala | 8 +- .../apache}/spark/PairRDDFunctionsSuite.scala | 4 +- .../spark/PartitionPruningRDDSuite.scala | 6 +- .../apache}/spark/PartitioningSuite.scala | 4 +- .../apache}/spark/PipedRDDSuite.scala | 2 +- .../{ => org/apache}/spark/RDDSuite.scala | 6 +- .../apache}/spark/SharedSparkContext.scala | 2 +- .../apache}/spark/ShuffleNettySuite.scala | 2 +- .../{ => org/apache}/spark/ShuffleSuite.scala | 16 +- .../apache}/spark/SizeEstimatorSuite.scala | 2 +- .../{ => org/apache}/spark/SortingSuite.scala | 2 +- .../apache}/spark/SparkContextInfoSuite.scala | 6 +- .../apache}/spark/ThreadingSuite.scala | 2 +- .../apache}/spark/UnpersistSuite.scala | 4 +- .../{ => org/apache}/spark/UtilsSuite.scala | 2 +- .../apache}/spark/ZippedPartitionsSuite.scala | 2 +- .../spark/io/CompressionCodecSuite.scala | 2 +- .../spark/metrics/MetricsConfigSuite.scala | 14 +- .../spark/metrics/MetricsSystemSuite.scala | 5 +- .../apache}/spark/rdd/JdbcRDDSuite.scala | 6 +- .../rdd/ParallelCollectionSplitSuite.scala | 2 +- .../spark/scheduler/DAGSchedulerSuite.scala | 28 +-- .../spark/scheduler/JobLoggerSuite.scala | 6 +- .../spark/scheduler/SparkListenerSuite.scala | 6 +- .../spark/scheduler/TaskContextSuite.scala | 12 +- .../cluster/ClusterSchedulerSuite.scala | 8 +- .../cluster/ClusterTaskSetManagerSuite.scala | 10 +- .../spark/scheduler/cluster/FakeTask.scala | 4 +- .../scheduler/local/LocalSchedulerSuite.scala | 8 +- .../spark/storage/BlockManagerSuite.scala | 33 ++-- .../{ => org/apache}/spark/ui/UISuite.scala | 2 +- .../spark/util/DistributionSuite.scala | 2 +- .../apache}/spark/util/FakeClock.scala | 2 +- .../spark/util/NextIteratorSuite.scala | 2 +- .../util/RateLimitedOutputStreamSuite.scala | 2 +- docs/_layouts/global.html | 2 +- examples/pom.xml | 14 +- .../apache}/spark/examples/JavaHdfsLR.java | 10 +- .../apache}/spark/examples/JavaKMeans.java | 14 +- .../apache}/spark/examples/JavaLogQuery.java | 12 +- .../apache}/spark/examples/JavaPageRank.java | 18 +- .../apache}/spark/examples/JavaSparkPi.java | 10 +- .../apache}/spark/examples/JavaTC.java | 8 +- .../apache}/spark/examples/JavaWordCount.java | 14 +- .../apache}/spark/mllib/examples/JavaALS.java | 14 +- .../spark/mllib/examples/JavaKMeans.java | 12 +- .../apache}/spark/mllib/examples/JavaLR.java | 14 +- .../examples/JavaFlumeEventCount.java | 10 +- .../examples/JavaNetworkWordCount.java | 16 +- .../streaming/examples/JavaQueueStream.java | 16 +- .../spark/examples/BroadcastTest.scala | 4 +- .../spark/examples/CassandraTest.scala | 6 +- .../examples/ExceptionHandlingTest.scala | 4 +- .../apache}/spark/examples/GroupByTest.scala | 6 +- .../apache}/spark/examples/HBaseTest.scala | 6 +- .../apache}/spark/examples/HdfsTest.scala | 4 +- .../apache}/spark/examples/LocalALS.scala | 2 +- .../apache}/spark/examples/LocalFileLR.scala | 4 +- .../apache}/spark/examples/LocalKMeans.scala | 6 +- .../apache}/spark/examples/LocalLR.scala | 4 +- .../apache}/spark/examples/LocalPi.scala | 4 +- .../apache}/spark/examples/LogQuery.scala | 6 +- .../spark/examples/MultiBroadcastTest.scala | 4 +- .../examples/SimpleSkewedGroupByTest.scala | 6 +- .../spark/examples/SkewedGroupByTest.scala | 6 +- .../apache}/spark/examples/SparkALS.scala | 4 +- .../apache}/spark/examples/SparkHdfsLR.scala | 8 +- .../apache}/spark/examples/SparkKMeans.scala | 8 +- .../apache}/spark/examples/SparkLR.scala | 6 +- .../spark/examples/SparkPageRank.scala | 6 +- .../apache}/spark/examples/SparkPi.scala | 4 +- .../apache}/spark/examples/SparkTC.scala | 4 +- .../spark/examples/bagel/PageRankUtils.scala | 10 +- .../examples/bagel/WikipediaPageRank.scala | 12 +- .../bagel/WikipediaPageRankStandalone.scala | 12 +- .../streaming/examples/ActorWordCount.scala | 14 +- .../streaming/examples/FlumeEventCount.scala | 8 +- .../streaming/examples/HdfsWordCount.scala | 6 +- .../streaming/examples/KafkaWordCount.scala | 12 +- .../streaming/examples/NetworkWordCount.scala | 6 +- .../streaming/examples/QueueStream.scala | 8 +- .../streaming/examples/RawNetworkGrep.scala | 12 +- .../examples/StatefulNetworkWordCount.scala | 6 +- .../examples/TwitterAlgebirdCMS.scala | 10 +- .../examples/TwitterAlgebirdHLL.scala | 8 +- .../examples/TwitterPopularTags.scala | 6 +- .../streaming/examples/ZeroMQWordCount.scala | 6 +- .../clickstream/PageViewGenerator.scala | 2 +- .../examples/clickstream/PageViewStream.scala | 8 +- mllib/pom.xml | 6 +- .../classification/ClassificationModel.scala | 4 +- .../classification/LogisticRegression.scala | 12 +- .../spark/mllib/classification/SVM.scala | 12 +- .../spark/mllib/clustering/KMeans.scala | 10 +- .../spark/mllib/clustering/KMeansModel.scala | 8 +- .../spark/mllib/clustering/LocalKMeans.scala | 2 +- .../spark/mllib/optimization/Gradient.scala | 2 +- .../mllib/optimization/GradientDescent.scala | 6 +- .../spark/mllib/optimization/Optimizer.scala | 4 +- .../spark/mllib/optimization/Updater.scala | 2 +- .../spark/mllib/recommendation/ALS.scala | 12 +- .../MatrixFactorizationModel.scala | 6 +- .../GeneralizedLinearAlgorithm.scala | 8 +- .../spark/mllib/regression/LabeledPoint.scala | 2 +- .../spark/mllib/regression/Lasso.scala | 8 +- .../mllib/regression/LinearRegression.scala | 8 +- .../mllib/regression/RegressionModel.scala | 4 +- .../mllib/regression/RidgeRegression.scala | 8 +- .../spark/mllib/util/DataValidators.scala | 6 +- .../mllib/util/KMeansDataGenerator.scala | 4 +- .../mllib/util/LinearDataGenerator.scala | 8 +- .../LogisticRegressionDataGenerator.scala | 6 +- .../spark/mllib/util/MFDataGenerator.scala | 8 +- .../apache}/spark/mllib/util/MLUtils.scala | 8 +- .../spark/mllib/util/SVMDataGenerator.scala | 6 +- .../JavaLogisticRegressionSuite.java | 8 +- .../mllib/classification/JavaSVMSuite.java | 8 +- .../mllib/clustering/JavaKMeansSuite.java | 6 +- .../mllib/recommendation/JavaALSSuite.java | 6 +- .../mllib/regression/JavaLassoSuite.java | 8 +- .../regression/JavaLinearRegressionSuite.java | 8 +- .../regression/JavaRidgeRegressionSuite.java | 8 +- .../LogisticRegressionSuite.scala | 6 +- .../spark/mllib/classification/SVMSuite.scala | 10 +- .../spark/mllib/clustering/KMeansSuite.scala | 6 +- .../spark/mllib/recommendation/ALSSuite.scala | 6 +- .../spark/mllib/regression/LassoSuite.scala | 6 +- .../regression/LinearRegressionSuite.scala | 8 +- .../regression/RidgeRegressionSuite.scala | 8 +- pom.xml | 14 +- project/SparkBuild.scala | 12 +- python/pyspark/context.py | 4 +- python/pyspark/files.py | 2 +- python/pyspark/java_gateway.py | 4 +- repl-bin/pom.xml | 12 +- repl/pom.xml | 12 +- .../spark/repl/ExecutorClassLoader.scala | 2 +- .../{ => org/apache}/spark/repl/Main.scala | 2 +- .../apache}/spark/repl/SparkHelper.scala | 0 .../apache}/spark/repl/SparkILoop.scala | 18 +- .../apache}/spark/repl/SparkIMain.scala | 10 +- .../apache}/spark/repl/SparkISettings.scala | 2 +- .../apache}/spark/repl/SparkImports.scala | 2 +- .../spark/repl/SparkJLineCompletion.scala | 2 +- .../apache}/spark/repl/SparkJLineReader.scala | 2 +- .../spark/repl/SparkMemberHandlers.scala | 2 +- .../apache}/spark/repl/ReplSuite.scala | 6 +- spark-executor | 2 +- spark-shell | 2 +- spark-shell.cmd | 2 +- streaming/pom.xml | 6 +- .../apache}/spark/streaming/Checkpoint.scala | 6 +- .../apache}/spark/streaming/DStream.scala | 16 +- .../streaming/DStreamCheckpointData.scala | 4 +- .../spark/streaming/DStreamGraph.scala | 4 +- .../apache}/spark/streaming/Duration.scala | 10 +- .../apache}/spark/streaming/Interval.scala | 2 +- .../apache}/spark/streaming/Job.scala | 2 +- .../apache}/spark/streaming/JobManager.scala | 6 +- .../spark/streaming/NetworkInputTracker.scala | 12 +- .../streaming/PairDStreamFunctions.scala | 26 +-- .../apache}/spark/streaming/Scheduler.scala | 9 +- .../spark/streaming/StreamingContext.scala | 18 +- .../apache}/spark/streaming/Time.scala | 2 +- .../streaming/api/java/JavaDStream.scala | 16 +- .../streaming/api/java/JavaDStreamLike.scala | 10 +- .../streaming/api/java/JavaPairDStream.scala | 24 +-- .../api/java/JavaStreamingContext.scala | 19 +- .../streaming/dstream/CoGroupedDStream.scala | 8 +- .../dstream/ConstantInputDStream.scala | 6 +- .../streaming/dstream/FileInputDStream.scala | 8 +- .../streaming/dstream/FilteredDStream.scala | 6 +- .../dstream/FlatMapValuedDStream.scala | 8 +- .../streaming/dstream/FlatMappedDStream.scala | 6 +- .../streaming/dstream/FlumeInputDStream.scala | 8 +- .../streaming/dstream/ForEachDStream.scala | 6 +- .../streaming/dstream/GlommedDStream.scala | 6 +- .../streaming/dstream/InputDStream.scala | 4 +- .../streaming/dstream/KafkaInputDStream.scala | 8 +- .../dstream/MapPartitionedDStream.scala | 6 +- .../streaming/dstream/MapValuedDStream.scala | 8 +- .../streaming/dstream/MappedDStream.scala | 6 +- .../dstream/NetworkInputDStream.scala | 18 +- .../dstream/PluggableInputDStream.scala | 4 +- .../streaming/dstream/QueueInputDStream.scala | 8 +- .../streaming/dstream/RawInputDStream.scala | 8 +- .../dstream/ReducedWindowedDStream.scala | 16 +- .../streaming/dstream/ShuffledDStream.scala | 8 +- .../dstream/SocketInputDStream.scala | 8 +- .../streaming/dstream/StateDStream.scala | 12 +- .../dstream/TransformedDStream.scala | 6 +- .../dstream/TwitterInputDStream.scala | 6 +- .../streaming/dstream/UnionDStream.scala | 8 +- .../streaming/dstream/WindowedDStream.scala | 10 +- .../streaming/receivers/ActorReceiver.scala | 6 +- .../streaming/receivers/ZeroMQReceiver.scala | 4 +- .../apache}/spark/streaming/util/Clock.scala | 2 +- .../streaming/util/MasterFailureTest.scala | 8 +- .../spark/streaming/util/RawTextHelper.scala | 6 +- .../spark/streaming/util/RawTextSender.scala | 6 +- .../spark/streaming/util/RecurringTimer.scala | 2 +- .../apache}/spark/streaming/JavaAPISuite.java | 32 +-- .../spark/streaming/JavaTestUtils.scala | 17 +- .../streaming/BasicOperationsSuite.scala | 6 +- .../spark/streaming/CheckpointSuite.scala | 8 +- .../spark/streaming/FailureSuite.scala | 6 +- .../spark/streaming/InputStreamsSuite.scala | 12 +- .../spark/streaming/TestSuiteBase.scala | 8 +- .../streaming/WindowOperationsSuite.scala | 6 +- tools/pom.xml | 8 +- .../tools/JavaAPICompletenessChecker.scala | 184 +++++++++--------- yarn/pom.xml | 6 +- .../spark/deploy/yarn/ApplicationMaster.scala | 4 +- .../yarn/ApplicationMasterArguments.scala | 4 +- .../apache}/spark/deploy/yarn/Client.scala | 6 +- .../spark/deploy/yarn/ClientArguments.scala | 8 +- .../spark/deploy/yarn/WorkerRunnable.scala | 6 +- .../deploy/yarn/YarnAllocationHandler.scala | 8 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 4 +- .../cluster/YarnClusterScheduler.scala | 6 +- 521 files changed, 1788 insertions(+), 1777 deletions(-) rename bagel/src/main/scala/{ => org/apache}/spark/bagel/Bagel.scala (86%) rename bagel/src/test/scala/{ => org/apache/spark}/bagel/BagelSuite.scala (96%) rename core/src/main/java/{ => org/apache}/spark/network/netty/FileClient.java (98%) rename core/src/main/java/{ => org/apache}/spark/network/netty/FileClientChannelInitializer.java (97%) rename core/src/main/java/{ => org/apache}/spark/network/netty/FileClientHandler.java (97%) rename core/src/main/java/{ => org/apache}/spark/network/netty/FileServer.java (98%) rename core/src/main/java/{ => org/apache}/spark/network/netty/FileServerChannelInitializer.java (97%) rename core/src/main/java/{ => org/apache}/spark/network/netty/FileServerHandler.java (98%) rename core/src/main/java/{ => org/apache}/spark/network/netty/PathResolver.java (93%) rename core/src/main/resources/{ => org/apache}/spark/ui/static/bootstrap.min.css (100%) rename core/src/main/resources/{ => org/apache}/spark/ui/static/sorttable.js (100%) rename core/src/main/resources/{ => org/apache}/spark/ui/static/spark-logo-77x50px-hd.png (100%) rename core/src/main/resources/{ => org/apache}/spark/ui/static/spark_logo.png (100%) rename core/src/main/resources/{ => org/apache}/spark/ui/static/webui.css (100%) rename core/src/main/scala/{ => org/apache}/spark/Accumulators.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/Aggregator.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/BlockStoreShuffleFetcher.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/CacheManager.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/ClosureCleaner.scala (99%) rename core/src/main/scala/{ => org/apache}/spark/Dependency.scala (99%) rename core/src/main/scala/{ => org/apache}/spark/DoubleRDDFunctions.scala (89%) rename core/src/main/scala/{ => org/apache}/spark/FetchFailedException.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/HttpFileServer.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/HttpServer.scala (99%) rename core/src/main/scala/{ => org/apache}/spark/JavaSerializer.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/KryoSerializer.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/Logging.scala (99%) rename core/src/main/scala/{ => org/apache}/spark/MapOutputTracker.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/PairRDDFunctions.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/Partition.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/Partitioner.scala (94%) rename core/src/main/scala/{ => org/apache}/spark/RDD.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/RDDCheckpointData.scala (99%) rename core/src/main/scala/{ => org/apache}/spark/SequenceFileRDDFunctions.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/SerializableWritable.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/ShuffleFetcher.scala (91%) rename core/src/main/scala/{ => org/apache}/spark/SizeEstimator.scala (99%) rename core/src/main/scala/{ => org/apache}/spark/SparkContext.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/SparkEnv.scala (91%) rename core/src/main/scala/{ => org/apache}/spark/SparkException.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/SparkFiles.java (98%) rename core/src/main/scala/{ => org/apache}/spark/SparkHadoopWriter.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/TaskContext.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/TaskEndReason.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/TaskState.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/Utils.scala (99%) rename core/src/main/scala/{ => org/apache}/spark/api/java/JavaDoubleRDD.scala (92%) rename core/src/main/scala/{ => org/apache}/spark/api/java/JavaPairRDD.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/api/java/JavaRDD.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/api/java/JavaRDDLike.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/api/java/JavaSparkContext.scala (94%) rename core/src/main/scala/{ => org/apache}/spark/api/java/JavaSparkContextVarargsWorkaround.java (98%) rename core/src/main/scala/{ => org/apache}/spark/api/java/JavaUtils.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/api/java/StorageLevels.java (96%) rename core/src/main/scala/{ => org/apache}/spark/api/java/function/DoubleFlatMapFunction.java (96%) rename core/src/main/scala/{ => org/apache}/spark/api/java/function/DoubleFunction.java (96%) rename core/src/main/scala/{ => org/apache}/spark/api/java/function/FlatMapFunction.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/api/java/function/FlatMapFunction2.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/api/java/function/Function.java (97%) rename core/src/main/scala/{ => org/apache}/spark/api/java/function/Function2.java (96%) rename core/src/main/scala/{ => org/apache}/spark/api/java/function/PairFlatMapFunction.java (97%) rename core/src/main/scala/{ => org/apache}/spark/api/java/function/PairFunction.java (97%) rename core/src/main/scala/{ => org/apache}/spark/api/java/function/VoidFunction.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/api/java/function/WrappedFunction1.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/api/java/function/WrappedFunction2.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/api/python/PythonPartitioner.scala (90%) rename core/src/main/scala/{ => org/apache}/spark/api/python/PythonRDD.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/api/python/PythonWorkerFactory.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/broadcast/BitTorrentBroadcast.scala (99%) rename core/src/main/scala/{ => org/apache}/spark/broadcast/Broadcast.scala (91%) rename core/src/main/scala/{ => org/apache}/spark/broadcast/BroadcastFactory.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/broadcast/HttpBroadcast.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/broadcast/MultiTracker.scala (99%) rename core/src/main/scala/{ => org/apache}/spark/broadcast/SourceInfo.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/broadcast/TreeBroadcast.scala (99%) rename core/src/main/scala/{ => org/apache}/spark/deploy/ApplicationDescription.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/deploy/Command.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/deploy/DeployMessage.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/deploy/ExecutorState.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/deploy/JsonProtocol.scala (92%) rename core/src/main/scala/{ => org/apache}/spark/deploy/LocalSparkCluster.scala (92%) rename core/src/main/scala/{ => org/apache}/spark/deploy/SparkHadoopUtil.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/deploy/WebUI.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/deploy/client/Client.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/deploy/client/ClientListener.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/deploy/client/TestClient.scala (90%) rename core/src/main/scala/{ => org/apache}/spark/deploy/client/TestExecutor.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/deploy/master/ApplicationInfo.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/deploy/master/ApplicationSource.scala (89%) rename core/src/main/scala/{ => org/apache}/spark/deploy/master/ApplicationState.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/deploy/master/ExecutorInfo.scala (92%) rename core/src/main/scala/{ => org/apache}/spark/deploy/master/Master.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/deploy/master/MasterArguments.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/deploy/master/MasterSource.scala (90%) rename core/src/main/scala/{ => org/apache}/spark/deploy/master/WorkerInfo.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/deploy/master/WorkerState.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/deploy/master/ui/ApplicationPage.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/deploy/master/ui/IndexPage.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/deploy/master/ui/MasterWebUI.scala (90%) rename core/src/main/scala/{ => org/apache}/spark/deploy/worker/ExecutorRunner.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/deploy/worker/Worker.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/deploy/worker/WorkerArguments.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/deploy/worker/WorkerSource.scala (92%) rename core/src/main/scala/{ => org/apache}/spark/deploy/worker/ui/IndexPage.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/deploy/worker/ui/WorkerWebUI.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/executor/Executor.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/executor/ExecutorBackend.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/executor/ExecutorExitCode.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/executor/ExecutorSource.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/executor/ExecutorURLClassLoader.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/executor/MesosExecutorBackend.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/executor/StandaloneExecutorBackend.scala (94%) rename core/src/main/scala/{ => org/apache}/spark/executor/TaskMetrics.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/io/CompressionCodec.scala (94%) rename core/src/main/scala/{ => org/apache}/spark/metrics/MetricsConfig.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/metrics/MetricsSystem.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/metrics/sink/ConsoleSink.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/metrics/sink/CsvSink.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/metrics/sink/JmxSink.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/metrics/sink/MetricsServlet.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/metrics/sink/Sink.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/metrics/source/JvmSource.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/metrics/source/Source.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/network/BufferMessage.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/network/Connection.scala (99%) rename core/src/main/scala/{ => org/apache}/spark/network/ConnectionManager.scala (99%) rename core/src/main/scala/{ => org/apache}/spark/network/ConnectionManagerId.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/network/ConnectionManagerTest.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/network/Message.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/network/MessageChunk.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/network/MessageChunkHeader.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/network/ReceiverTest.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/network/SenderTest.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/network/netty/FileHeader.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/network/netty/ShuffleCopier.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/network/netty/ShuffleSender.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/package.scala (65%) rename core/src/main/scala/{ => org/apache}/spark/partial/ApproximateActionListener.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/partial/ApproximateEvaluator.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/partial/BoundedDouble.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/partial/CountEvaluator.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/partial/GroupedCountEvaluator.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/partial/GroupedMeanEvaluator.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/partial/GroupedSumEvaluator.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/partial/MeanEvaluator.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/partial/PartialResult.scala (99%) rename core/src/main/scala/{ => org/apache}/spark/partial/StudentTCacher.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/partial/SumEvaluator.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/rdd/BlockRDD.scala (92%) rename core/src/main/scala/{ => org/apache}/spark/rdd/CartesianRDD.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/rdd/CheckpointRDD.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/rdd/CoGroupedRDD.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/rdd/CoalescedRDD.scala (99%) rename core/src/main/scala/{ => org/apache}/spark/rdd/EmptyRDD.scala (91%) rename core/src/main/scala/{ => org/apache}/spark/rdd/FilteredRDD.scala (92%) rename core/src/main/scala/{ => org/apache}/spark/rdd/FlatMappedRDD.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/rdd/FlatMappedValuesRDD.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/rdd/GlommedRDD.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/rdd/HadoopRDD.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/rdd/JdbcRDD.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/rdd/MapPartitionsRDD.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/rdd/MapPartitionsWithIndexRDD.scala (94%) rename core/src/main/scala/{ => org/apache}/spark/rdd/MappedRDD.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/rdd/MappedValuesRDD.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/rdd/NewHadoopRDD.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/rdd/OrderedRDDFunctions.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/rdd/ParallelCollectionRDD.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/rdd/PartitionPruningRDD.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/rdd/PipedRDD.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/rdd/SampledRDD.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/rdd/ShuffledRDD.scala (94%) rename core/src/main/scala/{ => org/apache}/spark/rdd/SubtractedRDD.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/rdd/UnionRDD.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/rdd/ZippedPartitionsRDD.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/rdd/ZippedRDD.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/ActiveJob.scala (94%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/DAGScheduler.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/DAGSchedulerEvent.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/DAGSchedulerSource.scala (92%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/InputFormatInfo.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/JobListener.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/JobLogger.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/JobResult.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/JobWaiter.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/MapStatus.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/ResultTask.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/ShuffleMapTask.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/SparkListener.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/SparkListenerBus.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/SplitInfo.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/Stage.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/StageInfo.scala (88%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/Task.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/TaskLocation.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/TaskResult.scala (94%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/TaskScheduler.scala (92%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/TaskSchedulerListener.scala (90%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/TaskSet.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/cluster/ClusterScheduler.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/cluster/ClusterTaskSetManager.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/cluster/ExecutorLossReason.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/cluster/Pool.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/cluster/Schedulable.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/cluster/SchedulableBuilder.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/cluster/SchedulerBackend.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/cluster/SchedulingAlgorithm.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/cluster/SchedulingMode.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala (89%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/cluster/StandaloneClusterMessage.scala (92%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/cluster/StandaloneSchedulerBackend.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/cluster/TaskDescription.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/cluster/TaskInfo.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/cluster/TaskLocality.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/cluster/TaskSetManager.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/cluster/WorkerOffer.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/local/LocalScheduler.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/local/LocalTaskSetManager.scala (94%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala (94%) rename core/src/main/scala/{ => org/apache}/spark/scheduler/mesos/MesosSchedulerBackend.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/serializer/Serializer.scala (91%) rename core/src/main/scala/{ => org/apache}/spark/serializer/SerializerManager.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/storage/BlockException.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/storage/BlockFetchTracker.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/storage/BlockFetcherIterator.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/storage/BlockManager.scala (99%) rename core/src/main/scala/{ => org/apache}/spark/storage/BlockManagerId.scala (94%) rename core/src/main/scala/{ => org/apache}/spark/storage/BlockManagerMaster.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/storage/BlockManagerMasterActor.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/storage/BlockManagerMessages.scala (99%) rename core/src/main/scala/{ => org/apache}/spark/storage/BlockManagerSlaveActor.scala (93%) rename core/src/main/scala/{ => org/apache}/spark/storage/BlockManagerSource.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/storage/BlockManagerWorker.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/storage/BlockMessage.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/storage/BlockMessageArray.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/storage/BlockObjectWriter.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/storage/BlockStore.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/storage/DiskStore.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/storage/MemoryStore.scala (99%) rename core/src/main/scala/{ => org/apache}/spark/storage/PutResult.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/storage/ShuffleBlockManager.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/storage/StorageLevel.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/storage/StorageUtils.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/storage/ThreadingTest.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/ui/JettyUtils.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/ui/Page.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/ui/SparkUI.scala (88%) rename core/src/main/scala/{ => org/apache}/spark/ui/UIUtils.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/ui/UIWorkloadGenerator.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/ui/env/EnvironmentUI.scala (94%) rename core/src/main/scala/{ => org/apache}/spark/ui/exec/ExecutorsUI.scala (91%) rename core/src/main/scala/{ => org/apache}/spark/ui/jobs/IndexPage.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/ui/jobs/JobProgressListener.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/ui/jobs/JobProgressUI.scala (86%) rename core/src/main/scala/{ => org/apache}/spark/ui/jobs/PoolPage.scala (87%) rename core/src/main/scala/{ => org/apache}/spark/ui/jobs/PoolTable.scala (92%) rename core/src/main/scala/{ => org/apache}/spark/ui/jobs/StagePage.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/ui/jobs/StageTable.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/ui/storage/BlockManagerUI.scala (92%) rename core/src/main/scala/{ => org/apache}/spark/ui/storage/IndexPage.scala (91%) rename core/src/main/scala/{ => org/apache}/spark/ui/storage/RDDPage.scala (94%) rename core/src/main/scala/{ => org/apache}/spark/util/AkkaUtils.scala (99%) rename core/src/main/scala/{ => org/apache}/spark/util/BoundedPriorityQueue.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/util/ByteBufferInputStream.scala (96%) rename core/src/main/scala/{ => org/apache}/spark/util/Clock.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/util/CompletionIterator.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/util/Distribution.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/util/IdGenerator.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/util/IntParam.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/util/MemoryParam.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/util/MetadataCleaner.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/util/MutablePair.scala (97%) rename core/src/main/scala/{ => org/apache}/spark/util/NextIterator.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/util/RateLimitedOutputStream.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/util/SerializableBuffer.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/util/StatCounter.scala (99%) rename core/src/main/scala/{ => org/apache}/spark/util/TimeStampedHashMap.scala (95%) rename core/src/main/scala/{ => org/apache}/spark/util/TimeStampedHashSet.scala (98%) rename core/src/main/scala/{ => org/apache}/spark/util/Vector.scala (97%) rename core/src/test/scala/{ => org/apache}/spark/AccumulatorSuite.scala (98%) rename core/src/test/scala/{ => org/apache}/spark/BroadcastSuite.scala (98%) rename core/src/test/scala/{ => org/apache}/spark/CheckpointSuite.scala (99%) rename core/src/test/scala/{ => org/apache}/spark/ClosureCleanerSuite.scala (98%) rename core/src/test/scala/{ => org/apache}/spark/DistributedSuite.scala (99%) rename core/src/test/scala/{ => org/apache}/spark/DriverSuite.scala (93%) rename core/src/test/scala/{ => org/apache}/spark/FailureSuite.scala (99%) rename core/src/test/scala/{ => org/apache}/spark/FileServerSuite.scala (99%) rename core/src/test/scala/{ => org/apache}/spark/FileSuite.scala (99%) rename core/src/test/scala/{ => org/apache}/spark/JavaAPISuite.java (98%) rename core/src/test/scala/{ => org/apache}/spark/KryoSerializerSuite.scala (98%) rename core/src/test/scala/{ => org/apache}/spark/LocalSparkContext.scala (98%) rename core/src/test/scala/{ => org/apache}/spark/MapOutputTrackerSuite.scala (97%) rename core/src/test/scala/{ => org/apache}/spark/PairRDDFunctionsSuite.scala (99%) rename core/src/test/scala/{ => org/apache}/spark/PartitionPruningRDDSuite.scala (86%) rename core/src/test/scala/{ => org/apache}/spark/PartitioningSuite.scala (99%) rename core/src/test/scala/{ => org/apache}/spark/PipedRDDSuite.scala (99%) rename core/src/test/scala/{ => org/apache}/spark/RDDSuite.scala (99%) rename core/src/test/scala/{ => org/apache}/spark/SharedSparkContext.scala (98%) rename core/src/test/scala/{ => org/apache}/spark/ShuffleNettySuite.scala (98%) rename core/src/test/scala/{ => org/apache}/spark/ShuffleSuite.scala (95%) rename core/src/test/scala/{ => org/apache}/spark/SizeEstimatorSuite.scala (99%) rename core/src/test/scala/{ => org/apache}/spark/SortingSuite.scala (99%) rename core/src/test/scala/{ => org/apache}/spark/SparkContextInfoSuite.scala (96%) rename core/src/test/scala/{ => org/apache}/spark/ThreadingSuite.scala (99%) rename core/src/test/scala/{ => org/apache}/spark/UnpersistSuite.scala (96%) rename core/src/test/scala/{ => org/apache}/spark/UtilsSuite.scala (99%) rename core/src/test/scala/{ => org/apache}/spark/ZippedPartitionsSuite.scala (98%) rename core/src/test/scala/{ => org/apache}/spark/io/CompressionCodecSuite.scala (98%) rename core/src/test/scala/{ => org/apache}/spark/metrics/MetricsConfigSuite.scala (82%) rename core/src/test/scala/{ => org/apache}/spark/metrics/MetricsSystemSuite.scala (93%) rename core/src/test/scala/{ => org/apache}/spark/rdd/JdbcRDDSuite.scala (95%) rename core/src/test/scala/{ => org/apache}/spark/rdd/ParallelCollectionSplitSuite.scala (99%) rename core/src/test/scala/{ => org/apache}/spark/scheduler/DAGSchedulerSuite.scala (96%) rename core/src/test/scala/{ => org/apache}/spark/scheduler/JobLoggerSuite.scala (98%) rename core/src/test/scala/{ => org/apache}/spark/scheduler/SparkListenerSuite.scala (96%) rename core/src/test/scala/{ => org/apache}/spark/scheduler/TaskContextSuite.scala (88%) rename core/src/test/scala/{ => org/apache}/spark/scheduler/cluster/ClusterSchedulerSuite.scala (98%) rename core/src/test/scala/{ => org/apache}/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala (98%) rename core/src/test/scala/{ => org/apache}/spark/scheduler/cluster/FakeTask.scala (91%) rename core/src/test/scala/{ => org/apache}/spark/scheduler/local/LocalSchedulerSuite.scala (98%) rename core/src/test/scala/{ => org/apache}/spark/storage/BlockManagerSuite.scala (97%) rename core/src/test/scala/{ => org/apache}/spark/ui/UISuite.scala (98%) rename core/src/test/scala/{ => org/apache}/spark/util/DistributionSuite.scala (97%) rename core/src/test/scala/{ => org/apache}/spark/util/FakeClock.scala (96%) rename core/src/test/scala/{ => org/apache}/spark/util/NextIteratorSuite.scala (98%) rename core/src/test/scala/{ => org/apache}/spark/util/RateLimitedOutputStreamSuite.scala (97%) rename examples/src/main/java/{ => org/apache}/spark/examples/JavaHdfsLR.java (94%) rename examples/src/main/java/{ => org/apache}/spark/examples/JavaKMeans.java (92%) rename examples/src/main/java/{ => org/apache}/spark/examples/JavaLogQuery.java (94%) rename examples/src/main/java/{ => org/apache}/spark/examples/JavaPageRank.java (89%) rename examples/src/main/java/{ => org/apache}/spark/examples/JavaSparkPi.java (89%) rename examples/src/main/java/{ => org/apache}/spark/examples/JavaTC.java (94%) rename examples/src/main/java/{ => org/apache}/spark/examples/JavaWordCount.java (85%) rename examples/src/main/java/{ => org/apache}/spark/mllib/examples/JavaALS.java (88%) rename examples/src/main/java/{ => org/apache}/spark/mllib/examples/JavaKMeans.java (89%) rename examples/src/main/java/{ => org/apache}/spark/mllib/examples/JavaLR.java (87%) rename examples/src/main/java/{ => org/apache}/spark/streaming/examples/JavaFlumeEventCount.java (90%) rename examples/src/main/java/{ => org/apache}/spark/streaming/examples/JavaNetworkWordCount.java (86%) rename examples/src/main/java/{ => org/apache}/spark/streaming/examples/JavaQueueStream.java (85%) rename examples/src/main/scala/{ => org/apache}/spark/examples/BroadcastTest.scala (95%) rename examples/src/main/scala/{ => org/apache}/spark/examples/CassandraTest.scala (98%) rename examples/src/main/scala/{ => org/apache}/spark/examples/ExceptionHandlingTest.scala (94%) rename examples/src/main/scala/{ => org/apache}/spark/examples/GroupByTest.scala (94%) rename examples/src/main/scala/{ => org/apache}/spark/examples/HBaseTest.scala (94%) rename examples/src/main/scala/{ => org/apache}/spark/examples/HdfsTest.scala (95%) rename examples/src/main/scala/{ => org/apache}/spark/examples/LocalALS.scala (99%) rename examples/src/main/scala/{ => org/apache}/spark/examples/LocalFileLR.scala (96%) rename examples/src/main/scala/{ => org/apache}/spark/examples/LocalKMeans.scala (96%) rename examples/src/main/scala/{ => org/apache}/spark/examples/LocalLR.scala (96%) rename examples/src/main/scala/{ => org/apache}/spark/examples/LocalPi.scala (94%) rename examples/src/main/scala/{ => org/apache}/spark/examples/LogQuery.scala (97%) rename examples/src/main/scala/{ => org/apache}/spark/examples/MultiBroadcastTest.scala (95%) rename examples/src/main/scala/{ => org/apache}/spark/examples/SimpleSkewedGroupByTest.scala (96%) rename examples/src/main/scala/{ => org/apache}/spark/examples/SkewedGroupByTest.scala (95%) rename examples/src/main/scala/{ => org/apache}/spark/examples/SparkALS.scala (98%) rename examples/src/main/scala/{ => org/apache}/spark/examples/SparkHdfsLR.scala (94%) rename examples/src/main/scala/{ => org/apache}/spark/examples/SparkKMeans.scala (94%) rename examples/src/main/scala/{ => org/apache}/spark/examples/SparkLR.scala (95%) rename examples/src/main/scala/{ => org/apache}/spark/examples/SparkPageRank.scala (91%) rename examples/src/main/scala/{ => org/apache}/spark/examples/SparkPi.scala (96%) rename examples/src/main/scala/{ => org/apache}/spark/examples/SparkTC.scala (97%) rename examples/src/main/scala/{ => org/apache}/spark/examples/bagel/PageRankUtils.scala (95%) rename examples/src/main/scala/{ => org/apache}/spark/examples/bagel/WikipediaPageRank.scala (93%) rename examples/src/main/scala/{ => org/apache}/spark/examples/bagel/WikipediaPageRankStandalone.scala (96%) rename examples/src/main/scala/{ => org/apache}/spark/streaming/examples/ActorWordCount.scala (93%) rename examples/src/main/scala/{ => org/apache}/spark/streaming/examples/FlumeEventCount.scala (92%) rename examples/src/main/scala/{ => org/apache}/spark/streaming/examples/HdfsWordCount.scala (92%) rename examples/src/main/scala/{ => org/apache}/spark/streaming/examples/KafkaWordCount.scala (92%) rename examples/src/main/scala/{ => org/apache}/spark/streaming/examples/NetworkWordCount.scala (93%) rename examples/src/main/scala/{ => org/apache}/spark/streaming/examples/QueueStream.scala (90%) rename examples/src/main/scala/{ => org/apache}/spark/streaming/examples/RawNetworkGrep.scala (90%) rename examples/src/main/scala/{ => org/apache}/spark/streaming/examples/StatefulNetworkWordCount.scala (95%) rename examples/src/main/scala/{ => org/apache}/spark/streaming/examples/TwitterAlgebirdCMS.scala (94%) rename examples/src/main/scala/{ => org/apache}/spark/streaming/examples/TwitterAlgebirdHLL.scala (94%) rename examples/src/main/scala/{ => org/apache}/spark/streaming/examples/TwitterPopularTags.scala (94%) rename examples/src/main/scala/{ => org/apache}/spark/streaming/examples/ZeroMQWordCount.scala (95%) rename examples/src/main/scala/{ => org/apache}/spark/streaming/examples/clickstream/PageViewGenerator.scala (98%) rename examples/src/main/scala/{ => org/apache}/spark/streaming/examples/clickstream/PageViewStream.scala (95%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/classification/ClassificationModel.scala (88%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/classification/LogisticRegression.scala (96%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/classification/SVM.scala (95%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/clustering/KMeans.scala (98%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/clustering/KMeansModel.scala (90%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/clustering/LocalKMeans.scala (98%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/optimization/Gradient.scala (98%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/optimization/GradientDescent.scala (97%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/optimization/Optimizer.scala (93%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/optimization/Updater.scala (98%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/recommendation/ALS.scala (98%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/recommendation/MatrixFactorizationModel.scala (94%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/regression/GeneralizedLinearAlgorithm.scala (96%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/regression/LabeledPoint.scala (96%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/regression/Lasso.scala (97%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/regression/LinearRegression.scala (97%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/regression/RegressionModel.scala (95%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/regression/RidgeRegression.scala (97%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/util/DataValidators.scala (91%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/util/KMeansDataGenerator.scala (97%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/util/LinearDataGenerator.scala (95%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/util/LogisticRegressionDataGenerator.scala (95%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/util/MFDataGenerator.scala (97%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/util/MLUtils.scala (96%) rename mllib/src/main/scala/{ => org/apache}/spark/mllib/util/SVMDataGenerator.scala (91%) rename mllib/src/test/java/{ => org/apache}/spark/mllib/classification/JavaLogisticRegressionSuite.java (93%) rename mllib/src/test/java/{ => org/apache}/spark/mllib/classification/JavaSVMSuite.java (93%) rename mllib/src/test/java/{ => org/apache}/spark/mllib/clustering/JavaKMeansSuite.java (96%) rename mllib/src/test/java/{ => org/apache}/spark/mllib/recommendation/JavaALSSuite.java (96%) rename mllib/src/test/java/{ => org/apache}/spark/mllib/regression/JavaLassoSuite.java (94%) rename mllib/src/test/java/{ => org/apache}/spark/mllib/regression/JavaLinearRegressionSuite.java (94%) rename mllib/src/test/java/{ => org/apache}/spark/mllib/regression/JavaRidgeRegressionSuite.java (95%) rename mllib/src/test/scala/{ => org/apache}/spark/mllib/classification/LogisticRegressionSuite.scala (97%) rename mllib/src/test/scala/{ => org/apache}/spark/mllib/classification/SVMSuite.scala (96%) rename mllib/src/test/scala/{ => org/apache}/spark/mllib/clustering/KMeansSuite.scala (98%) rename mllib/src/test/scala/{ => org/apache}/spark/mllib/recommendation/ALSSuite.scala (97%) rename mllib/src/test/scala/{ => org/apache}/spark/mllib/regression/LassoSuite.scala (96%) rename mllib/src/test/scala/{ => org/apache}/spark/mllib/regression/LinearRegressionSuite.scala (93%) rename mllib/src/test/scala/{ => org/apache}/spark/mllib/regression/RidgeRegressionSuite.scala (94%) rename repl/src/main/scala/{ => org/apache}/spark/repl/ExecutorClassLoader.scala (99%) rename repl/src/main/scala/{ => org/apache}/spark/repl/Main.scala (97%) rename repl/src/main/scala/{ => org/apache}/spark/repl/SparkHelper.scala (100%) rename repl/src/main/scala/{ => org/apache}/spark/repl/SparkILoop.scala (98%) rename repl/src/main/scala/{ => org/apache}/spark/repl/SparkIMain.scala (99%) rename repl/src/main/scala/{ => org/apache}/spark/repl/SparkISettings.scala (98%) rename repl/src/main/scala/{ => org/apache}/spark/repl/SparkImports.scala (99%) rename repl/src/main/scala/{ => org/apache}/spark/repl/SparkJLineCompletion.scala (99%) rename repl/src/main/scala/{ => org/apache}/spark/repl/SparkJLineReader.scala (98%) rename repl/src/main/scala/{ => org/apache}/spark/repl/SparkMemberHandlers.scala (99%) rename repl/src/test/scala/{ => org/apache}/spark/repl/ReplSuite.scala (98%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/Checkpoint.scala (98%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/DStream.scala (98%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/DStreamCheckpointData.scala (98%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/DStreamGraph.scala (98%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/Duration.scala (86%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/Interval.scala (98%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/Job.scala (97%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/JobManager.scala (96%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/NetworkInputTracker.scala (95%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/PairDStreamFunctions.scala (96%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/Scheduler.scala (95%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/StreamingContext.scala (97%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/Time.scala (98%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/api/java/JavaDStream.scala (90%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/api/java/JavaDStreamLike.scala (97%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/api/java/JavaPairDStream.scala (97%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/api/java/JavaStreamingContext.scala (97%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/CoGroupedDStream.scala (90%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/ConstantInputDStream.scala (89%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/FileInputDStream.scala (97%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/FilteredDStream.scala (90%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/FlatMapValuedDStream.scala (88%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/FlatMappedDStream.scala (90%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/FlumeInputDStream.scala (96%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/ForEachDStream.scala (91%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/GlommedDStream.scala (89%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/InputDStream.scala (95%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/KafkaInputDStream.scala (96%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/MapPartitionedDStream.scala (90%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/MapValuedDStream.scala (87%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/MappedDStream.scala (90%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/NetworkInputDStream.scala (93%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/PluggableInputDStream.scala (91%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/QueueInputDStream.scala (90%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/RawInputDStream.scala (95%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/ReducedWindowedDStream.scala (94%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/ShuffledDStream.scala (88%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/SocketInputDStream.scala (93%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/StateDStream.scala (94%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/TransformedDStream.scala (90%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/TwitterInputDStream.scala (97%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/UnionDStream.scala (91%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/dstream/WindowedDStream.scala (89%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/receivers/ActorReceiver.scala (97%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/receivers/ZeroMQReceiver.scala (95%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/util/Clock.scala (98%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/util/MasterFailureTest.scala (98%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/util/RawTextHelper.scala (96%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/util/RawTextSender.scala (94%) rename streaming/src/main/scala/{ => org/apache}/spark/streaming/util/RecurringTimer.scala (98%) rename streaming/src/test/java/{ => org/apache}/spark/streaming/JavaAPISuite.java (98%) rename streaming/src/test/java/{ => org/apache}/spark/streaming/JavaTestUtils.scala (84%) rename streaming/src/test/scala/{ => org/apache}/spark/streaming/BasicOperationsSuite.scala (98%) rename streaming/src/test/scala/{ => org/apache}/spark/streaming/CheckpointSuite.scala (98%) rename streaming/src/test/scala/{ => org/apache}/spark/streaming/FailureSuite.scala (93%) rename streaming/src/test/scala/{ => org/apache}/spark/streaming/InputStreamsSuite.scala (97%) rename streaming/src/test/scala/{ => org/apache}/spark/streaming/TestSuiteBase.scala (98%) rename streaming/src/test/scala/{ => org/apache}/spark/streaming/WindowOperationsSuite.scala (98%) rename tools/src/main/scala/{ => org/apache}/spark/tools/JavaAPICompletenessChecker.scala (64%) rename yarn/src/main/scala/{ => org/apache}/spark/deploy/yarn/ApplicationMaster.scala (99%) rename yarn/src/main/scala/{ => org/apache}/spark/deploy/yarn/ApplicationMasterArguments.scala (97%) rename yarn/src/main/scala/{ => org/apache}/spark/deploy/yarn/Client.scala (99%) rename yarn/src/main/scala/{ => org/apache}/spark/deploy/yarn/ClientArguments.scala (95%) rename yarn/src/main/scala/{ => org/apache}/spark/deploy/yarn/WorkerRunnable.scala (98%) rename yarn/src/main/scala/{ => org/apache}/spark/deploy/yarn/YarnAllocationHandler.scala (99%) rename yarn/src/main/scala/{ => org/apache}/spark/deploy/yarn/YarnSparkHadoopUtil.scala (95%) rename yarn/src/main/scala/{ => org/apache}/spark/scheduler/cluster/YarnClusterScheduler.scala (93%) diff --git a/README.md b/README.md index 2ddfe862a23..c4170650f79 100644 --- a/README.md +++ b/README.md @@ -27,7 +27,7 @@ Or, for the Python API, the Python shell (`./pyspark`). Spark also comes with several sample programs in the `examples` directory. To run one of them, use `./run-example `. For example: - ./run-example spark.examples.SparkLR local[2] + ./run-example org.apache.spark.examples.SparkLR local[2] will run the Logistic Regression example locally on 2 CPUs. diff --git a/assembly/pom.xml b/assembly/pom.xml index 74990b63612..dc63811b76a 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -19,13 +19,13 @@ 4.0.0 - org.spark-project + org.apache.spark spark-parent 0.8.0-SNAPSHOT ../pom.xml - org.spark-project + org.apache.spark spark-assembly Spark Project Assembly http://spark-project.org/ @@ -40,27 +40,27 @@ - org.spark-project + org.apache.spark spark-core ${project.version} - org.spark-project + org.apache.spark spark-bagel ${project.version} - org.spark-project + org.apache.spark spark-mllib ${project.version} - org.spark-project + org.apache.spark spark-repl ${project.version} - org.spark-project + org.apache.spark spark-streaming ${project.version} @@ -121,7 +121,7 @@ hadoop2-yarn - org.spark-project + org.apache.spark spark-yarn ${project.version} diff --git a/assembly/src/main/assembly/assembly.xml b/assembly/src/main/assembly/assembly.xml index 4543b52c931..47d3fa93d07 100644 --- a/assembly/src/main/assembly/assembly.xml +++ b/assembly/src/main/assembly/assembly.xml @@ -30,9 +30,9 @@ - ${project.parent.basedir}/core/src/main/resources/spark/ui/static/ + ${project.parent.basedir}/core/src/main/resources/org/apache/spark/ui/static/ - /ui-resources/spark/ui/static + /ui-resources/org/apache/spark/ui/static **/* @@ -63,10 +63,10 @@ - org.spark-project:*:jar + org.apache.spark:*:jar - org.spark-project:spark-assembly:jar + org.apache.spark:spark-assembly:jar @@ -77,7 +77,7 @@ false org.apache.hadoop:*:jar - org.spark-project:*:jar + org.apache.spark:*:jar diff --git a/bagel/pom.xml b/bagel/pom.xml index cbcf8d12393..9340991377a 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -19,13 +19,13 @@ 4.0.0 - org.spark-project + org.apache.spark spark-parent 0.8.0-SNAPSHOT ../pom.xml - org.spark-project + org.apache.spark spark-bagel jar Spark Project Bagel @@ -33,7 +33,7 @@ - org.spark-project + org.apache.spark spark-core ${project.version} diff --git a/bagel/src/main/scala/spark/bagel/Bagel.scala b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala similarity index 86% rename from bagel/src/main/scala/spark/bagel/Bagel.scala rename to bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala index 80c8d53d2bb..fec8737fcdf 100644 --- a/bagel/src/main/scala/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala @@ -15,32 +15,31 @@ * limitations under the License. */ -package spark.bagel +package org.apache.spark.bagel -import spark._ -import spark.SparkContext._ +import org.apache.spark._ +import org.apache.spark.SparkContext._ -import scala.collection.mutable.ArrayBuffer -import storage.StorageLevel +import org.apache.spark.storage.StorageLevel object Bagel extends Logging { val DEFAULT_STORAGE_LEVEL = StorageLevel.MEMORY_AND_DISK /** * Runs a Bagel program. - * @param sc [[spark.SparkContext]] to use for the program. + * @param sc [[org.apache.spark.SparkContext]] to use for the program. * @param vertices vertices of the graph represented as an RDD of (Key, Vertex) pairs. Often the Key will be * the vertex id. * @param messages initial set of messages represented as an RDD of (Key, Message) pairs. Often this will be an * empty array, i.e. sc.parallelize(Array[K, Message]()). - * @param combiner [[spark.bagel.Combiner]] combines multiple individual messages to a given vertex into one + * @param combiner [[org.apache.spark.bagel.Combiner]] combines multiple individual messages to a given vertex into one * message before sending (which often involves network I/O). - * @param aggregator [[spark.bagel.Aggregator]] performs a reduce across all vertices after each superstep, + * @param aggregator [[org.apache.spark.bagel.Aggregator]] performs a reduce across all vertices after each superstep, * and provides the result to each vertex in the next superstep. - * @param partitioner [[spark.Partitioner]] partitions values by key + * @param partitioner [[org.apache.spark.Partitioner]] partitions values by key * @param numPartitions number of partitions across which to split the graph. * Default is the default parallelism of the SparkContext - * @param storageLevel [[spark.storage.StorageLevel]] to use for caching of intermediate RDDs in each superstep. + * @param storageLevel [[org.apache.spark.storage.StorageLevel]] to use for caching of intermediate RDDs in each superstep. * Defaults to caching in memory. * @param compute function that takes a Vertex, optional set of (possibly combined) messages to the Vertex, * optional Aggregator and the current superstep, @@ -98,7 +97,7 @@ object Bagel extends Logging { verts } - /** Runs a Bagel program with no [[spark.bagel.Aggregator]] and the default storage level */ + /** Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] and the default storage level */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( sc: SparkContext, vertices: RDD[(K, V)], @@ -110,7 +109,7 @@ object Bagel extends Logging { compute: (V, Option[C], Int) => (V, Array[M]) ): RDD[(K, V)] = run(sc, vertices, messages, combiner, numPartitions, DEFAULT_STORAGE_LEVEL)(compute) - /** Runs a Bagel program with no [[spark.bagel.Aggregator]] */ + /** Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( sc: SparkContext, vertices: RDD[(K, V)], @@ -128,7 +127,7 @@ object Bagel extends Logging { } /** - * Runs a Bagel program with no [[spark.bagel.Aggregator]], default [[spark.HashPartitioner]] + * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], default [[org.apache.spark.HashPartitioner]] * and default storage level */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( @@ -141,7 +140,7 @@ object Bagel extends Logging { compute: (V, Option[C], Int) => (V, Array[M]) ): RDD[(K, V)] = run(sc, vertices, messages, combiner, numPartitions, DEFAULT_STORAGE_LEVEL)(compute) - /** Runs a Bagel program with no [[spark.bagel.Aggregator]] and the default [[spark.HashPartitioner]]*/ + /** Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] and the default [[org.apache.spark.HashPartitioner]]*/ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( sc: SparkContext, vertices: RDD[(K, V)], @@ -159,8 +158,8 @@ object Bagel extends Logging { } /** - * Runs a Bagel program with no [[spark.bagel.Aggregator]], default [[spark.HashPartitioner]], - * [[spark.bagel.DefaultCombiner]] and the default storage level + * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], default [[org.apache.spark.HashPartitioner]], + * [[org.apache.spark.bagel.DefaultCombiner]] and the default storage level */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest]( sc: SparkContext, @@ -172,8 +171,8 @@ object Bagel extends Logging { ): RDD[(K, V)] = run(sc, vertices, messages, numPartitions, DEFAULT_STORAGE_LEVEL)(compute) /** - * Runs a Bagel program with no [[spark.bagel.Aggregator]], the default [[spark.HashPartitioner]] - * and [[spark.bagel.DefaultCombiner]] + * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], the default [[org.apache.spark.HashPartitioner]] + * and [[org.apache.spark.bagel.DefaultCombiner]] */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest]( sc: SparkContext, diff --git a/bagel/src/test/scala/bagel/BagelSuite.scala b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala similarity index 96% rename from bagel/src/test/scala/bagel/BagelSuite.scala rename to bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala index ef2d57fbd0a..7b954a47757 100644 --- a/bagel/src/test/scala/bagel/BagelSuite.scala +++ b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala @@ -15,16 +15,14 @@ * limitations under the License. */ -package spark.bagel +package org.apache.spark.bagel -import org.scalatest.{FunSuite, Assertions, BeforeAndAfter} +import org.scalatest.{BeforeAndAfter, FunSuite, Assertions} import org.scalatest.concurrent.Timeouts import org.scalatest.time.SpanSugar._ -import scala.collection.mutable.ArrayBuffer - -import spark._ -import storage.StorageLevel +import org.apache.spark._ +import org.apache.spark.storage.StorageLevel class TestVertex(val active: Boolean, val age: Int) extends Vertex with Serializable class TestMessage(val targetId: String) extends Message[String] with Serializable diff --git a/bin/start-master.sh b/bin/start-master.sh index 2288fb19d7b..648c7ae75fe 100755 --- a/bin/start-master.sh +++ b/bin/start-master.sh @@ -49,4 +49,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then fi fi -"$bin"/spark-daemon.sh start spark.deploy.master.Master 1 --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT +"$bin"/spark-daemon.sh start org.apache.spark.deploy.master.Master 1 --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT diff --git a/bin/start-slave.sh b/bin/start-slave.sh index d6db16882d0..4eefa209443 100755 --- a/bin/start-slave.sh +++ b/bin/start-slave.sh @@ -32,4 +32,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then fi fi -"$bin"/spark-daemon.sh start spark.deploy.worker.Worker "$@" +"$bin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker "$@" diff --git a/bin/stop-master.sh b/bin/stop-master.sh index 31a610bf9df..310e33bedc0 100755 --- a/bin/stop-master.sh +++ b/bin/stop-master.sh @@ -24,4 +24,4 @@ bin=`cd "$bin"; pwd` . "$bin/spark-config.sh" -"$bin"/spark-daemon.sh stop spark.deploy.master.Master 1 +"$bin"/spark-daemon.sh stop org.apache.spark.deploy.master.Master 1 diff --git a/bin/stop-slaves.sh b/bin/stop-slaves.sh index 8e056f23d4e..03e416a1327 100755 --- a/bin/stop-slaves.sh +++ b/bin/stop-slaves.sh @@ -29,9 +29,9 @@ if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then fi if [ "$SPARK_WORKER_INSTANCES" = "" ]; then - "$bin"/spark-daemons.sh stop spark.deploy.worker.Worker 1 + "$bin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker 1 else for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do - "$bin"/spark-daemons.sh stop spark.deploy.worker.Worker $(( $i + 1 )) + "$bin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 )) done fi diff --git a/core/pom.xml b/core/pom.xml index 53696367e98..c803217f964 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -19,13 +19,13 @@ 4.0.0 - org.spark-project + org.apache.spark spark-parent 0.8.0-SNAPSHOT ../pom.xml - org.spark-project + org.apache.spark spark-core jar Spark Project Core diff --git a/core/src/main/java/spark/network/netty/FileClient.java b/core/src/main/java/org/apache/spark/network/netty/FileClient.java similarity index 98% rename from core/src/main/java/spark/network/netty/FileClient.java rename to core/src/main/java/org/apache/spark/network/netty/FileClient.java index 0625a6d5029..20a7a3aa8c1 100644 --- a/core/src/main/java/spark/network/netty/FileClient.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileClient.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.network.netty; +package org.apache.spark.network.netty; import io.netty.bootstrap.Bootstrap; import io.netty.channel.Channel; diff --git a/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java b/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java similarity index 97% rename from core/src/main/java/spark/network/netty/FileClientChannelInitializer.java rename to core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java index 05ad4b61d72..65ee15d63b8 100644 --- a/core/src/main/java/spark/network/netty/FileClientChannelInitializer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.network.netty; +package org.apache.spark.network.netty; import io.netty.buffer.BufType; import io.netty.channel.ChannelInitializer; diff --git a/core/src/main/java/spark/network/netty/FileClientHandler.java b/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java similarity index 97% rename from core/src/main/java/spark/network/netty/FileClientHandler.java rename to core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java index e8cd9801f63..c4aa2669e05 100644 --- a/core/src/main/java/spark/network/netty/FileClientHandler.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.network.netty; +package org.apache.spark.network.netty; import io.netty.buffer.ByteBuf; import io.netty.channel.ChannelHandlerContext; diff --git a/core/src/main/java/spark/network/netty/FileServer.java b/core/src/main/java/org/apache/spark/network/netty/FileServer.java similarity index 98% rename from core/src/main/java/spark/network/netty/FileServer.java rename to core/src/main/java/org/apache/spark/network/netty/FileServer.java index 9f009a61d5d..666432474dc 100644 --- a/core/src/main/java/spark/network/netty/FileServer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServer.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.network.netty; +package org.apache.spark.network.netty; import java.net.InetSocketAddress; diff --git a/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java b/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java similarity index 97% rename from core/src/main/java/spark/network/netty/FileServerChannelInitializer.java rename to core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java index 50c57a81a35..833af1632de 100644 --- a/core/src/main/java/spark/network/netty/FileServerChannelInitializer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.network.netty; +package org.apache.spark.network.netty; import io.netty.channel.ChannelInitializer; import io.netty.channel.socket.SocketChannel; diff --git a/core/src/main/java/spark/network/netty/FileServerHandler.java b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java similarity index 98% rename from core/src/main/java/spark/network/netty/FileServerHandler.java rename to core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java index 176ba8da491..d3d57a02555 100644 --- a/core/src/main/java/spark/network/netty/FileServerHandler.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.network.netty; +package org.apache.spark.network.netty; import java.io.File; import java.io.FileInputStream; diff --git a/core/src/main/java/spark/network/netty/PathResolver.java b/core/src/main/java/org/apache/spark/network/netty/PathResolver.java similarity index 93% rename from core/src/main/java/spark/network/netty/PathResolver.java rename to core/src/main/java/org/apache/spark/network/netty/PathResolver.java index f446c55b19b..94c034cad01 100755 --- a/core/src/main/java/spark/network/netty/PathResolver.java +++ b/core/src/main/java/org/apache/spark/network/netty/PathResolver.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.network.netty; +package org.apache.spark.network.netty; public interface PathResolver { diff --git a/core/src/main/resources/spark/ui/static/bootstrap.min.css b/core/src/main/resources/org/apache/spark/ui/static/bootstrap.min.css similarity index 100% rename from core/src/main/resources/spark/ui/static/bootstrap.min.css rename to core/src/main/resources/org/apache/spark/ui/static/bootstrap.min.css diff --git a/core/src/main/resources/spark/ui/static/sorttable.js b/core/src/main/resources/org/apache/spark/ui/static/sorttable.js similarity index 100% rename from core/src/main/resources/spark/ui/static/sorttable.js rename to core/src/main/resources/org/apache/spark/ui/static/sorttable.js diff --git a/core/src/main/resources/spark/ui/static/spark-logo-77x50px-hd.png b/core/src/main/resources/org/apache/spark/ui/static/spark-logo-77x50px-hd.png similarity index 100% rename from core/src/main/resources/spark/ui/static/spark-logo-77x50px-hd.png rename to core/src/main/resources/org/apache/spark/ui/static/spark-logo-77x50px-hd.png diff --git a/core/src/main/resources/spark/ui/static/spark_logo.png b/core/src/main/resources/org/apache/spark/ui/static/spark_logo.png similarity index 100% rename from core/src/main/resources/spark/ui/static/spark_logo.png rename to core/src/main/resources/org/apache/spark/ui/static/spark_logo.png diff --git a/core/src/main/resources/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css similarity index 100% rename from core/src/main/resources/spark/ui/static/webui.css rename to core/src/main/resources/org/apache/spark/ui/static/webui.css diff --git a/core/src/main/scala/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala similarity index 95% rename from core/src/main/scala/spark/Accumulators.scala rename to core/src/main/scala/org/apache/spark/Accumulators.scala index 6ff92ce833b..5177ee58faa 100644 --- a/core/src/main/scala/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import java.io._ @@ -28,7 +28,7 @@ import scala.collection.generic.Growable * * You must define how to add data, and how to merge two of these together. For some datatypes, * such as a counter, these might be the same operation. In that case, you can use the simpler - * [[spark.Accumulator]]. They won't always be the same, though -- e.g., imagine you are + * [[org.apache.spark.Accumulator]]. They won't always be the same, though -- e.g., imagine you are * accumulating a set. You will add items to the set, and you will union two sets together. * * @param initialValue initial value of accumulator @@ -176,7 +176,7 @@ class GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Ser def zero(initialValue: R): R = { // We need to clone initialValue, but it's hard to specify that R should also be Cloneable. // Instead we'll serialize it to a buffer and load it back. - val ser = (new spark.JavaSerializer).newInstance() + val ser = new JavaSerializer().newInstance() val copy = ser.deserialize[R](ser.serialize(initialValue)) copy.clear() // In case it contained stuff copy @@ -184,7 +184,7 @@ class GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Ser } /** - * A simpler value of [[spark.Accumulable]] where the result type being accumulated is the same + * A simpler value of [[org.apache.spark.Accumulable]] where the result type being accumulated is the same * as the types of elements being merged. * * @param initialValue initial value of accumulator @@ -195,7 +195,7 @@ class Accumulator[T](@transient initialValue: T, param: AccumulatorParam[T]) extends Accumulable[T,T](initialValue, param) /** - * A simpler version of [[spark.AccumulableParam]] where the only datatype you can add in is the same type + * A simpler version of [[org.apache.spark.AccumulableParam]] where the only datatype you can add in is the same type * as the accumulated value. An implicit AccumulatorParam object needs to be available when you create * Accumulators of a specific type. * diff --git a/core/src/main/scala/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala similarity index 98% rename from core/src/main/scala/spark/Aggregator.scala rename to core/src/main/scala/org/apache/spark/Aggregator.scala index 9af401986d3..3ef402926e8 100644 --- a/core/src/main/scala/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import java.util.{HashMap => JHashMap} diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala similarity index 93% rename from core/src/main/scala/spark/BlockStoreShuffleFetcher.scala rename to core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala index 1ec95ed9b88..908ff56a6bb 100644 --- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala @@ -15,15 +15,15 @@ * limitations under the License. */ -package spark +package org.apache.spark import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap -import spark.executor.{ShuffleReadMetrics, TaskMetrics} -import spark.serializer.Serializer -import spark.storage.BlockManagerId -import spark.util.CompletionIterator +import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} +import org.apache.spark.serializer.Serializer +import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.CompletionIterator private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging { diff --git a/core/src/main/scala/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala similarity index 97% rename from core/src/main/scala/spark/CacheManager.scala rename to core/src/main/scala/org/apache/spark/CacheManager.scala index 81314805a93..42e465b9d86 100644 --- a/core/src/main/scala/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark +package org.apache.spark import scala.collection.mutable.{ArrayBuffer, HashSet} -import spark.storage.{BlockManager, StorageLevel} +import org.apache.spark.storage.{BlockManager, StorageLevel} /** Spark class responsible for passing RDDs split contents to the BlockManager and making diff --git a/core/src/main/scala/spark/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/ClosureCleaner.scala similarity index 99% rename from core/src/main/scala/spark/ClosureCleaner.scala rename to core/src/main/scala/org/apache/spark/ClosureCleaner.scala index 8b39241095b..71d9e62d4f0 100644 --- a/core/src/main/scala/spark/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ClosureCleaner.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import java.lang.reflect.Field diff --git a/core/src/main/scala/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala similarity index 99% rename from core/src/main/scala/spark/Dependency.scala rename to core/src/main/scala/org/apache/spark/Dependency.scala index d5a96065709..cc3c2474a6d 100644 --- a/core/src/main/scala/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark /** * Base class for dependencies. diff --git a/core/src/main/scala/spark/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/DoubleRDDFunctions.scala similarity index 89% rename from core/src/main/scala/spark/DoubleRDDFunctions.scala rename to core/src/main/scala/org/apache/spark/DoubleRDDFunctions.scala index 104168e61cc..dd344491b83 100644 --- a/core/src/main/scala/spark/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/DoubleRDDFunctions.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark +package org.apache.spark -import spark.partial.BoundedDouble -import spark.partial.MeanEvaluator -import spark.partial.PartialResult -import spark.partial.SumEvaluator -import spark.util.StatCounter +import org.apache.spark.partial.BoundedDouble +import org.apache.spark.partial.MeanEvaluator +import org.apache.spark.partial.PartialResult +import org.apache.spark.partial.SumEvaluator +import org.apache.spark.util.StatCounter /** * Extra functions available on RDDs of Doubles through an implicit conversion. @@ -34,7 +34,7 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { } /** - * Return a [[spark.util.StatCounter]] object that captures the mean, variance and count + * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and count * of the RDD's elements in one operation. */ def stats(): StatCounter = { diff --git a/core/src/main/scala/spark/FetchFailedException.scala b/core/src/main/scala/org/apache/spark/FetchFailedException.scala similarity index 95% rename from core/src/main/scala/spark/FetchFailedException.scala rename to core/src/main/scala/org/apache/spark/FetchFailedException.scala index a2dae6cae9a..d242047502f 100644 --- a/core/src/main/scala/spark/FetchFailedException.scala +++ b/core/src/main/scala/org/apache/spark/FetchFailedException.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark +package org.apache.spark -import spark.storage.BlockManagerId +import org.apache.spark.storage.BlockManagerId private[spark] class FetchFailedException( taskEndReason: TaskEndReason, diff --git a/core/src/main/scala/spark/HttpFileServer.scala b/core/src/main/scala/org/apache/spark/HttpFileServer.scala similarity index 98% rename from core/src/main/scala/spark/HttpFileServer.scala rename to core/src/main/scala/org/apache/spark/HttpFileServer.scala index a13a7a28590..9b3a8966489 100644 --- a/core/src/main/scala/spark/HttpFileServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpFileServer.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import java.io.{File} import com.google.common.io.Files diff --git a/core/src/main/scala/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala similarity index 99% rename from core/src/main/scala/spark/HttpServer.scala rename to core/src/main/scala/org/apache/spark/HttpServer.scala index c9dffbc6311..db36c7c9dd1 100644 --- a/core/src/main/scala/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import java.io.File import java.net.InetAddress diff --git a/core/src/main/scala/spark/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/JavaSerializer.scala similarity index 97% rename from core/src/main/scala/spark/JavaSerializer.scala rename to core/src/main/scala/org/apache/spark/JavaSerializer.scala index 04c5f44e6be..f43396cb6b0 100644 --- a/core/src/main/scala/spark/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/JavaSerializer.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark +package org.apache.spark import java.io._ import java.nio.ByteBuffer import serializer.{Serializer, SerializerInstance, DeserializationStream, SerializationStream} -import spark.util.ByteBufferInputStream +import org.apache.spark.util.ByteBufferInputStream private[spark] class JavaSerializationStream(out: OutputStream) extends SerializationStream { val objOut = new ObjectOutputStream(out) diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/KryoSerializer.scala similarity index 96% rename from core/src/main/scala/spark/KryoSerializer.scala rename to core/src/main/scala/org/apache/spark/KryoSerializer.scala index eeb2993d8a5..db86e6db439 100644 --- a/core/src/main/scala/spark/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/KryoSerializer.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import java.io._ import java.nio.ByteBuffer @@ -24,8 +24,8 @@ import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} import com.twitter.chill.ScalaKryoInstantiator import serializer.{SerializerInstance, DeserializationStream, SerializationStream} -import spark.broadcast._ -import spark.storage._ +import org.apache.spark.broadcast._ +import org.apache.spark.storage._ private[spark] class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { @@ -104,7 +104,7 @@ trait KryoRegistrator { /** * 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 { +class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging { private val bufferSize = System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 def newKryoOutput() = new KryoOutput(bufferSize) @@ -153,4 +153,4 @@ class KryoSerializer extends spark.serializer.Serializer with Logging { def newInstance(): SerializerInstance = { new KryoSerializerInstance(this) } -} \ No newline at end of file +} diff --git a/core/src/main/scala/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala similarity index 99% rename from core/src/main/scala/spark/Logging.scala rename to core/src/main/scala/org/apache/spark/Logging.scala index 79b0362830d..6a973ea4951 100644 --- a/core/src/main/scala/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import org.slf4j.Logger import org.slf4j.LoggerFactory diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala similarity index 98% rename from core/src/main/scala/spark/MapOutputTracker.scala rename to core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 0cd0341a724..0f422d910a1 100644 --- a/core/src/main/scala/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} @@ -30,9 +30,9 @@ import akka.remote._ import akka.util.Duration -import spark.scheduler.MapStatus -import spark.storage.BlockManagerId -import spark.util.{MetadataCleaner, TimeStampedHashMap} +import org.apache.spark.scheduler.MapStatus +import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap} private[spark] sealed trait MapOutputTrackerMessage diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/PairRDDFunctions.scala similarity index 98% rename from core/src/main/scala/spark/PairRDDFunctions.scala rename to core/src/main/scala/org/apache/spark/PairRDDFunctions.scala index cc1285dd95f..d046e7c1a4f 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/PairRDDFunctions.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import java.nio.ByteBuffer import java.util.{Date, HashMap => JHashMap} @@ -40,11 +40,11 @@ import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, Job => NewAPIHadoopJob, SparkHadoopMapReduceUtil} import org.apache.hadoop.security.UserGroupInformation -import spark.partial.BoundedDouble -import spark.partial.PartialResult -import spark.rdd._ -import spark.SparkContext._ -import spark.Partitioner._ +import org.apache.spark.partial.BoundedDouble +import org.apache.spark.partial.PartialResult +import org.apache.spark.rdd._ +import org.apache.spark.SparkContext._ +import org.apache.spark.Partitioner._ /** * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. @@ -559,7 +559,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) val formatter = new SimpleDateFormat("yyyyMMddHHmm") val jobtrackerID = formatter.format(new Date()) val stageId = self.id - def writeShard(context: spark.TaskContext, iter: Iterator[(K,V)]): Int = { + def writeShard(context: TaskContext, iter: Iterator[(K,V)]): Int = { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt @@ -571,7 +571,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) committer.setupTask(hadoopContext) val writer = format.getRecordWriter(hadoopContext).asInstanceOf[NewRecordWriter[K,V]] while (iter.hasNext) { - val (k, v) = iter.next + val (k, v) = iter.next() writer.write(k, v) } writer.close(hadoopContext) diff --git a/core/src/main/scala/spark/Partition.scala b/core/src/main/scala/org/apache/spark/Partition.scala similarity index 97% rename from core/src/main/scala/spark/Partition.scala rename to core/src/main/scala/org/apache/spark/Partition.scala index 2a4edcec984..87914a061f5 100644 --- a/core/src/main/scala/spark/Partition.scala +++ b/core/src/main/scala/org/apache/spark/Partition.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark /** * A partition of an RDD. diff --git a/core/src/main/scala/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala similarity index 94% rename from core/src/main/scala/spark/Partitioner.scala rename to core/src/main/scala/org/apache/spark/Partitioner.scala index 65da8235d75..4dce2607b0a 100644 --- a/core/src/main/scala/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark /** * An object that defines how the elements in a key-value pair RDD are partitioned by key. @@ -56,7 +56,7 @@ object Partitioner { } /** - * A [[spark.Partitioner]] that implements hash-based partitioning using Java's `Object.hashCode`. + * A [[org.apache.spark.Partitioner]] that implements hash-based partitioning using Java's `Object.hashCode`. * * Java arrays have hashCodes that are based on the arrays' identities rather than their contents, * so attempting to partition an RDD[Array[_]] or RDD[(Array[_], _)] using a HashPartitioner will @@ -79,7 +79,7 @@ class HashPartitioner(partitions: Int) extends Partitioner { } /** - * A [[spark.Partitioner]] that partitions sortable records by range into roughly equal ranges. + * A [[org.apache.spark.Partitioner]] that partitions sortable records by range into roughly equal ranges. * Determines the ranges by sampling the RDD passed in. */ class RangePartitioner[K <% Ordered[K]: ClassManifest, V]( diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/org/apache/spark/RDD.scala similarity index 95% rename from core/src/main/scala/spark/RDD.scala rename to core/src/main/scala/org/apache/spark/RDD.scala index 25a6951732b..0d1f07f76cb 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/org/apache/spark/RDD.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import java.util.Random @@ -31,30 +31,30 @@ import org.apache.hadoop.mapred.TextOutputFormat import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} -import spark.Partitioner._ -import spark.api.java.JavaRDD -import spark.partial.BoundedDouble -import spark.partial.CountEvaluator -import spark.partial.GroupedCountEvaluator -import spark.partial.PartialResult -import spark.rdd.CoalescedRDD -import spark.rdd.CartesianRDD -import spark.rdd.FilteredRDD -import spark.rdd.FlatMappedRDD -import spark.rdd.GlommedRDD -import spark.rdd.MappedRDD -import spark.rdd.MapPartitionsRDD -import spark.rdd.MapPartitionsWithIndexRDD -import spark.rdd.PipedRDD -import spark.rdd.SampledRDD -import spark.rdd.ShuffledRDD -import spark.rdd.UnionRDD -import spark.rdd.ZippedRDD -import spark.rdd.ZippedPartitionsRDD2 -import spark.rdd.ZippedPartitionsRDD3 -import spark.rdd.ZippedPartitionsRDD4 -import spark.storage.StorageLevel -import spark.util.BoundedPriorityQueue +import org.apache.spark.Partitioner._ +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.partial.BoundedDouble +import org.apache.spark.partial.CountEvaluator +import org.apache.spark.partial.GroupedCountEvaluator +import org.apache.spark.partial.PartialResult +import org.apache.spark.rdd.CoalescedRDD +import org.apache.spark.rdd.CartesianRDD +import org.apache.spark.rdd.FilteredRDD +import org.apache.spark.rdd.FlatMappedRDD +import org.apache.spark.rdd.GlommedRDD +import org.apache.spark.rdd.MappedRDD +import org.apache.spark.rdd.MapPartitionsRDD +import org.apache.spark.rdd.MapPartitionsWithIndexRDD +import org.apache.spark.rdd.PipedRDD +import org.apache.spark.rdd.SampledRDD +import org.apache.spark.rdd.ShuffledRDD +import org.apache.spark.rdd.UnionRDD +import org.apache.spark.rdd.ZippedRDD +import org.apache.spark.rdd.ZippedPartitionsRDD2 +import org.apache.spark.rdd.ZippedPartitionsRDD3 +import org.apache.spark.rdd.ZippedPartitionsRDD4 +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.BoundedPriorityQueue import SparkContext._ @@ -62,12 +62,12 @@ import SparkContext._ * A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. Represents an immutable, * partitioned collection of elements that can be operated on in parallel. This class contains the * basic operations available on all RDDs, such as `map`, `filter`, and `persist`. In addition, - * [[spark.PairRDDFunctions]] contains operations available only on RDDs of key-value pairs, such - * as `groupByKey` and `join`; [[spark.DoubleRDDFunctions]] contains operations available only on - * RDDs of Doubles; and [[spark.SequenceFileRDDFunctions]] contains operations available on RDDs + * [[org.apache.spark.PairRDDFunctions]] contains operations available only on RDDs of key-value pairs, such + * as `groupByKey` and `join`; [[org.apache.spark.DoubleRDDFunctions]] contains operations available only on + * RDDs of Doubles; and [[org.apache.spark.SequenceFileRDDFunctions]] contains operations available on RDDs * that can be saved as SequenceFiles. These operations are automatically available on any RDD of * the right type (e.g. RDD[(Int, Int)] through implicit conversions when you - * `import spark.SparkContext._`. + * `import org.apache.spark.SparkContext._`. * * Internally, each RDD is characterized by five main properties: * @@ -893,7 +893,7 @@ abstract class RDD[T: ClassManifest]( dependencies.head.rdd.asInstanceOf[RDD[U]] } - /** The [[spark.SparkContext]] that this RDD was created on. */ + /** The [[org.apache.spark.SparkContext]] that this RDD was created on. */ def context = sc // Avoid handling doCheckpoint multiple times to prevent excessive recursion @@ -929,7 +929,7 @@ abstract class RDD[T: ClassManifest]( * Clears the dependencies of this RDD. This method must ensure that all references * to the original parent RDDs is removed to enable the parent RDDs to be garbage * collected. Subclasses of RDD may override this method for implementing their own cleaning - * logic. See [[spark.rdd.UnionRDD]] for an example. + * logic. See [[org.apache.spark.rdd.UnionRDD]] for an example. */ protected def clearDependencies() { dependencies_ = null diff --git a/core/src/main/scala/spark/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/RDDCheckpointData.scala similarity index 99% rename from core/src/main/scala/spark/RDDCheckpointData.scala rename to core/src/main/scala/org/apache/spark/RDDCheckpointData.scala index b615f820eb4..0334de6924c 100644 --- a/core/src/main/scala/spark/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/RDDCheckpointData.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration diff --git a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/SequenceFileRDDFunctions.scala similarity index 98% rename from core/src/main/scala/spark/SequenceFileRDDFunctions.scala rename to core/src/main/scala/org/apache/spark/SequenceFileRDDFunctions.scala index 9f30b7f22f2..d58fb4e4bcb 100644 --- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/SequenceFileRDDFunctions.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import java.io.EOFException import java.net.URL @@ -41,7 +41,7 @@ import org.apache.hadoop.io.NullWritable import org.apache.hadoop.io.BytesWritable import org.apache.hadoop.io.Text -import spark.SparkContext._ +import org.apache.spark.SparkContext._ /** * Extra functions available on RDDs of (key, value) pairs to create a Hadoop SequenceFile, diff --git a/core/src/main/scala/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala similarity index 98% rename from core/src/main/scala/spark/SerializableWritable.scala rename to core/src/main/scala/org/apache/spark/SerializableWritable.scala index 936d8e62412..fdd4c24e234 100644 --- a/core/src/main/scala/spark/SerializableWritable.scala +++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import java.io._ diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala similarity index 91% rename from core/src/main/scala/spark/ShuffleFetcher.scala rename to core/src/main/scala/org/apache/spark/ShuffleFetcher.scala index a6839cf7a40..307c383a89c 100644 --- a/core/src/main/scala/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark +package org.apache.spark -import spark.executor.TaskMetrics -import spark.serializer.Serializer +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.serializer.Serializer private[spark] abstract class ShuffleFetcher { diff --git a/core/src/main/scala/spark/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/SizeEstimator.scala similarity index 99% rename from core/src/main/scala/spark/SizeEstimator.scala rename to core/src/main/scala/org/apache/spark/SizeEstimator.scala index 6cc57566d79..4bfc8377103 100644 --- a/core/src/main/scala/spark/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/SizeEstimator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import java.lang.reflect.Field import java.lang.reflect.Modifier diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala similarity index 97% rename from core/src/main/scala/spark/SparkContext.scala rename to core/src/main/scala/org/apache/spark/SparkContext.scala index 7ce9505b9c2..1207b242bc0 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import java.io._ import java.net.URI @@ -52,22 +52,22 @@ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFor import org.apache.mesos.MesosNativeLibrary -import spark.deploy.LocalSparkCluster -import spark.partial.{ApproximateEvaluator, PartialResult} -import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD, +import org.apache.spark.deploy.LocalSparkCluster +import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} +import org.apache.spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD, OrderedRDDFunctions} -import spark.scheduler._ -import spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, +import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler, Schedulable, SchedulingMode} -import spark.scheduler.local.LocalScheduler -import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import spark.storage.{StorageStatus, StorageUtils, RDDInfo, BlockManagerSource} -import spark.ui.SparkUI -import spark.util.{MetadataCleaner, TimeStampedHashMap} +import org.apache.spark.scheduler.local.LocalScheduler +import org.apache.spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} +import org.apache.spark.storage.{StorageStatus, StorageUtils, RDDInfo, BlockManagerSource} +import org.apache.spark.ui.SparkUI +import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap} import scala.Some -import spark.scheduler.StageInfo -import spark.storage.RDDInfo -import spark.storage.StorageStatus +import org.apache.spark.scheduler.StageInfo +import org.apache.spark.storage.RDDInfo +import org.apache.spark.storage.StorageStatus /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -494,14 +494,14 @@ class SparkContext( // Methods for creating shared variables /** - * Create an [[spark.Accumulator]] variable of a given type, which tasks can "add" values + * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" values * to using the `+=` method. Only the driver can access the accumulator's `value`. */ def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]) = new Accumulator(initialValue, param) /** - * Create an [[spark.Accumulable]] shared variable, to which tasks can add values with `+=`. + * Create an [[org.apache.spark.Accumulable]] shared variable, to which tasks can add values with `+=`. * Only the driver can access the accumuable's `value`. * @tparam T accumulator type * @tparam R type that can be added to the accumulator @@ -521,7 +521,7 @@ class SparkContext( } /** - * Broadcast a read-only variable to the cluster, returning a [[spark.broadcast.Broadcast]] object for + * Broadcast a read-only variable to the cluster, returning a [[org.apache.spark.broadcast.Broadcast]] object for * reading it in distributed functions. The variable will be sent to each cluster only once. */ def broadcast[T](value: T) = env.broadcastManager.newBroadcast[T](value, isLocal) diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala similarity index 91% rename from core/src/main/scala/spark/SparkEnv.scala rename to core/src/main/scala/org/apache/spark/SparkEnv.scala index 1f66e9cc7f3..6e6fe5df6b6 100644 --- a/core/src/main/scala/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import collection.mutable import serializer.Serializer @@ -23,15 +23,14 @@ import serializer.Serializer import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem} import akka.remote.RemoteActorRefProvider -import spark.broadcast.BroadcastManager -import spark.metrics.MetricsSystem -import spark.deploy.SparkHadoopUtil -import spark.storage.BlockManager -import spark.storage.BlockManagerMaster -import spark.network.ConnectionManager -import spark.serializer.{Serializer, SerializerManager} -import spark.util.AkkaUtils -import spark.api.python.PythonWorkerFactory +import org.apache.spark.broadcast.BroadcastManager +import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.storage.{BlockManagerMasterActor, BlockManager, BlockManagerMaster} +import org.apache.spark.network.ConnectionManager +import org.apache.spark.serializer.{Serializer, SerializerManager} +import org.apache.spark.util.AkkaUtils +import org.apache.spark.api.python.PythonWorkerFactory /** @@ -156,10 +155,10 @@ object SparkEnv extends Logging { val serializerManager = new SerializerManager val serializer = serializerManager.setDefault( - System.getProperty("spark.serializer", "spark.JavaSerializer")) + System.getProperty("spark.serializer", "org.apache.spark.JavaSerializer")) val closureSerializer = serializerManager.get( - System.getProperty("spark.closure.serializer", "spark.JavaSerializer")) + System.getProperty("spark.closure.serializer", "org.apache.spark.JavaSerializer")) def registerOrLookup(name: String, newActor: => Actor): ActorRef = { if (isDriver) { @@ -177,7 +176,7 @@ object SparkEnv extends Logging { val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", - new spark.storage.BlockManagerMasterActor(isLocal))) + new BlockManagerMasterActor(isLocal))) val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer) val connectionManager = blockManager.connectionManager @@ -194,7 +193,7 @@ object SparkEnv extends Logging { new MapOutputTrackerActor(mapOutputTracker)) val shuffleFetcher = instantiateClass[ShuffleFetcher]( - "spark.shuffle.fetcher", "spark.BlockStoreShuffleFetcher") + "spark.shuffle.fetcher", "org.apache.spark.BlockStoreShuffleFetcher") val httpFileServer = new HttpFileServer() httpFileServer.initialize() diff --git a/core/src/main/scala/spark/SparkException.scala b/core/src/main/scala/org/apache/spark/SparkException.scala similarity index 97% rename from core/src/main/scala/spark/SparkException.scala rename to core/src/main/scala/org/apache/spark/SparkException.scala index b7045eea63e..d34e47e8cac 100644 --- a/core/src/main/scala/spark/SparkException.scala +++ b/core/src/main/scala/org/apache/spark/SparkException.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark class SparkException(message: String, cause: Throwable) extends Exception(message, cause) { diff --git a/core/src/main/scala/spark/SparkFiles.java b/core/src/main/scala/org/apache/spark/SparkFiles.java similarity index 98% rename from core/src/main/scala/spark/SparkFiles.java rename to core/src/main/scala/org/apache/spark/SparkFiles.java index f9b3f7965ea..af9cf85e372 100644 --- a/core/src/main/scala/spark/SparkFiles.java +++ b/core/src/main/scala/org/apache/spark/SparkFiles.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark; +package org.apache.spark; import java.io.File; diff --git a/core/src/main/scala/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala similarity index 98% rename from core/src/main/scala/spark/SparkHadoopWriter.scala rename to core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 6b330ef5727..2bab9d6e3d9 100644 --- a/core/src/main/scala/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -25,8 +25,8 @@ import java.text.NumberFormat import java.io.IOException import java.util.Date -import spark.Logging -import spark.SerializableWritable +import org.apache.spark.Logging +import org.apache.spark.SerializableWritable /** * Internal helper class that saves an RDD using a Hadoop OutputFormat. This is only public diff --git a/core/src/main/scala/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala similarity index 98% rename from core/src/main/scala/spark/TaskContext.scala rename to core/src/main/scala/org/apache/spark/TaskContext.scala index b79f4ca8130..b2dd668330a 100644 --- a/core/src/main/scala/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import executor.TaskMetrics import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala similarity index 93% rename from core/src/main/scala/spark/TaskEndReason.scala rename to core/src/main/scala/org/apache/spark/TaskEndReason.scala index 3ad665da344..03bf268863c 100644 --- a/core/src/main/scala/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark +package org.apache.spark -import spark.executor.TaskMetrics -import spark.storage.BlockManagerId +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.storage.BlockManagerId /** * Various possible reasons why a task ended. The low-level TaskScheduler is supposed to retry diff --git a/core/src/main/scala/spark/TaskState.scala b/core/src/main/scala/org/apache/spark/TaskState.scala similarity index 98% rename from core/src/main/scala/spark/TaskState.scala rename to core/src/main/scala/org/apache/spark/TaskState.scala index bf757530565..19ce8369d90 100644 --- a/core/src/main/scala/spark/TaskState.scala +++ b/core/src/main/scala/org/apache/spark/TaskState.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import org.apache.mesos.Protos.{TaskState => MesosTaskState} diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/org/apache/spark/Utils.scala similarity index 99% rename from core/src/main/scala/spark/Utils.scala rename to core/src/main/scala/org/apache/spark/Utils.scala index bb8aad3f4ce..1e17deb010d 100644 --- a/core/src/main/scala/spark/Utils.scala +++ b/core/src/main/scala/org/apache/spark/Utils.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import java.io._ import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address, ServerSocket} @@ -33,8 +33,8 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} -import spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} -import spark.deploy.SparkHadoopUtil +import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} +import org.apache.spark.deploy.SparkHadoopUtil import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala similarity index 92% rename from core/src/main/scala/spark/api/java/JavaDoubleRDD.scala rename to core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index 8ce7df62133..cb25ff728ee 100644 --- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -15,16 +15,16 @@ * limitations under the License. */ -package spark.api.java - -import spark.RDD -import spark.SparkContext.doubleRDDToDoubleRDDFunctions -import spark.api.java.function.{Function => JFunction} -import spark.util.StatCounter -import spark.partial.{BoundedDouble, PartialResult} -import spark.storage.StorageLevel +package org.apache.spark.api.java + +import org.apache.spark.RDD +import org.apache.spark.SparkContext.doubleRDDToDoubleRDDFunctions +import org.apache.spark.api.java.function.{Function => JFunction} +import org.apache.spark.util.StatCounter +import org.apache.spark.partial.{BoundedDouble, PartialResult} +import org.apache.spark.storage.StorageLevel import java.lang.Double -import spark.Partitioner +import org.apache.spark.Partitioner class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, JavaDoubleRDD] { @@ -119,7 +119,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav def sum(): Double = srdd.sum() /** - * Return a [[spark.util.StatCounter]] object that captures the mean, variance and count + * Return a [[org.apache.spark.util.StatCounter]] object that captures the mean, variance and count * of the RDD's elements in one operation. */ def stats(): StatCounter = srdd.stats() diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala similarity index 97% rename from core/src/main/scala/spark/api/java/JavaPairRDD.scala rename to core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index effe6e5e0d6..09da35aee66 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.api.java +package org.apache.spark.api.java import java.util.{List => JList} import java.util.Comparator @@ -30,17 +30,17 @@ import org.apache.hadoop.mapred.OutputFormat import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.hadoop.conf.Configuration -import spark.HashPartitioner -import spark.Partitioner -import spark.Partitioner._ -import spark.RDD -import spark.SparkContext.rddToPairRDDFunctions -import spark.api.java.function.{Function2 => JFunction2} -import spark.api.java.function.{Function => JFunction} -import spark.partial.BoundedDouble -import spark.partial.PartialResult -import spark.rdd.OrderedRDDFunctions -import spark.storage.StorageLevel +import org.apache.spark.HashPartitioner +import org.apache.spark.Partitioner +import org.apache.spark.Partitioner._ +import org.apache.spark.RDD +import org.apache.spark.SparkContext.rddToPairRDDFunctions +import org.apache.spark.api.java.function.{Function2 => JFunction2} +import org.apache.spark.api.java.function.{Function => JFunction} +import org.apache.spark.partial.BoundedDouble +import org.apache.spark.partial.PartialResult +import org.apache.spark.rdd.OrderedRDDFunctions +import org.apache.spark.storage.StorageLevel class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManifest[K], diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala similarity index 95% rename from core/src/main/scala/spark/api/java/JavaRDD.scala rename to core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index c0bf2cf5686..68cfcf59994 100644 --- a/core/src/main/scala/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.api.java +package org.apache.spark.api.java -import spark._ -import spark.api.java.function.{Function => JFunction} -import spark.storage.StorageLevel +import org.apache.spark._ +import org.apache.spark.api.java.function.{Function => JFunction} +import org.apache.spark.storage.StorageLevel class JavaRDD[T](val rdd: RDD[T])(implicit val classManifest: ClassManifest[T]) extends JavaRDDLike[T, JavaRDD[T]] { diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala similarity index 97% rename from core/src/main/scala/spark/api/java/JavaRDDLike.scala rename to core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 2c2b138f16c..1ad8514980f 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -15,18 +15,18 @@ * limitations under the License. */ -package spark.api.java +package org.apache.spark.api.java import java.util.{List => JList, Comparator} 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, _} -import spark.partial.{PartialResult, BoundedDouble} -import spark.storage.StorageLevel +import org.apache.spark.{SparkContext, Partition, RDD, TaskContext} +import org.apache.spark.api.java.JavaPairRDD._ +import org.apache.spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _} +import org.apache.spark.partial.{PartialResult, BoundedDouble} +import org.apache.spark.storage.StorageLevel import com.google.common.base.Optional @@ -40,7 +40,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** Set of partitions in this RDD. */ def splits: JList[Partition] = new java.util.ArrayList(rdd.partitions.toSeq) - /** The [[spark.SparkContext]] that this RDD was created on. */ + /** The [[org.apache.spark.SparkContext]] that this RDD was created on. */ def context: SparkContext = rdd.context /** A unique ID for this RDD (within its SparkContext). */ diff --git a/core/src/main/scala/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala similarity index 94% rename from core/src/main/scala/spark/api/java/JavaSparkContext.scala rename to core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 29d57004b56..618a7b3bf7f 100644 --- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.api.java +package org.apache.spark.api.java import java.util.{Map => JMap} @@ -27,15 +27,15 @@ import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} -import spark.{Accumulable, AccumulableParam, Accumulator, AccumulatorParam, RDD, SparkContext} -import spark.SparkContext.IntAccumulatorParam -import spark.SparkContext.DoubleAccumulatorParam -import spark.broadcast.Broadcast +import org.apache.spark.{Accumulable, AccumulableParam, Accumulator, AccumulatorParam, RDD, SparkContext} +import org.apache.spark.SparkContext.IntAccumulatorParam +import org.apache.spark.SparkContext.DoubleAccumulatorParam +import org.apache.spark.broadcast.Broadcast import com.google.common.base.Optional /** - * A Java-friendly version of [[spark.SparkContext]] that returns [[spark.api.java.JavaRDD]]s and + * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns [[org.apache.spark.api.java.JavaRDD]]s and * works with Java collections instead of Scala ones. */ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWorkaround { @@ -283,48 +283,48 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork } /** - * Create an [[spark.Accumulator]] integer variable, which tasks can "add" values + * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values * to using the `add` method. Only the master can access the accumulator's `value`. */ def intAccumulator(initialValue: Int): Accumulator[java.lang.Integer] = sc.accumulator(initialValue)(IntAccumulatorParam).asInstanceOf[Accumulator[java.lang.Integer]] /** - * Create an [[spark.Accumulator]] double variable, which tasks can "add" values + * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values * to using the `add` method. Only the master can access the accumulator's `value`. */ def doubleAccumulator(initialValue: Double): Accumulator[java.lang.Double] = sc.accumulator(initialValue)(DoubleAccumulatorParam).asInstanceOf[Accumulator[java.lang.Double]] /** - * Create an [[spark.Accumulator]] integer variable, which tasks can "add" values + * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values * to using the `add` method. Only the master can access the accumulator's `value`. */ def accumulator(initialValue: Int): Accumulator[java.lang.Integer] = intAccumulator(initialValue) /** - * Create an [[spark.Accumulator]] double variable, which tasks can "add" values + * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values * to using the `add` method. Only the master can access the accumulator's `value`. */ def accumulator(initialValue: Double): Accumulator[java.lang.Double] = doubleAccumulator(initialValue) /** - * Create an [[spark.Accumulator]] variable of a given type, which tasks can "add" values + * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" values * to using the `add` method. Only the master can access the accumulator's `value`. */ def accumulator[T](initialValue: T, accumulatorParam: AccumulatorParam[T]): Accumulator[T] = sc.accumulator(initialValue)(accumulatorParam) /** - * Create an [[spark.Accumulable]] shared variable of the given type, to which tasks can + * Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks can * "add" values with `add`. Only the master can access the accumuable's `value`. */ def accumulable[T, R](initialValue: T, param: AccumulableParam[T, R]): Accumulable[T, R] = sc.accumulable(initialValue)(param) /** - * Broadcast a read-only variable to the cluster, returning a [[spark.Broadcast]] object for + * Broadcast a read-only variable to the cluster, returning a [[org.apache.spark.Broadcast]] object for * reading it in distributed functions. The variable will be sent to each cluster only once. */ def broadcast[T](value: T): Broadcast[T] = sc.broadcast(value) diff --git a/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java similarity index 98% rename from core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java rename to core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java index 42b1de01b10..c9cbce5624a 100644 --- a/core/src/main/scala/spark/api/java/JavaSparkContextVarargsWorkaround.java +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.api.java; +package org.apache.spark.api.java; import java.util.Arrays; import java.util.ArrayList; diff --git a/core/src/main/scala/spark/api/java/JavaUtils.scala b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala similarity index 96% rename from core/src/main/scala/spark/api/java/JavaUtils.scala rename to core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala index ffc131ac83c..ecbf18849ad 100644 --- a/core/src/main/scala/spark/api/java/JavaUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.api.java +package org.apache.spark.api.java import com.google.common.base.Optional diff --git a/core/src/main/scala/spark/api/java/StorageLevels.java b/core/src/main/scala/org/apache/spark/api/java/StorageLevels.java similarity index 96% rename from core/src/main/scala/spark/api/java/StorageLevels.java rename to core/src/main/scala/org/apache/spark/api/java/StorageLevels.java index f385636e832..0744269773f 100644 --- a/core/src/main/scala/spark/api/java/StorageLevels.java +++ b/core/src/main/scala/org/apache/spark/api/java/StorageLevels.java @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.api.java; +package org.apache.spark.api.java; -import spark.storage.StorageLevel; +import org.apache.spark.storage.StorageLevel; /** * Expose some commonly useful storage level constants. diff --git a/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.java similarity index 96% rename from core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java rename to core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.java index 8bc88d757fd..4830067f7a0 100644 --- a/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.api.java.function; +package org.apache.spark.api.java.function; import scala.runtime.AbstractFunction1; diff --git a/core/src/main/scala/spark/api/java/function/DoubleFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java similarity index 96% rename from core/src/main/scala/spark/api/java/function/DoubleFunction.java rename to core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java index 1aa1e5dae03..db34cd190ad 100644 --- a/core/src/main/scala/spark/api/java/function/DoubleFunction.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.api.java.function; +package org.apache.spark.api.java.function; import scala.runtime.AbstractFunction1; diff --git a/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala similarity index 96% rename from core/src/main/scala/spark/api/java/function/FlatMapFunction.scala rename to core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala index 9eb0cfe3f9a..158539a8461 100644 --- a/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala +++ b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.api.java.function +package org.apache.spark.api.java.function /** * A function that returns zero or more output records from each input record. diff --git a/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala similarity index 96% rename from core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala rename to core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala index dda98710c2d..5ef6a814f5a 100644 --- a/core/src/main/scala/spark/api/java/function/FlatMapFunction2.scala +++ b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.api.java.function +package org.apache.spark.api.java.function /** * A function that takes two inputs and returns zero or more output records. diff --git a/core/src/main/scala/spark/api/java/function/Function.java b/core/src/main/scala/org/apache/spark/api/java/function/Function.java similarity index 97% rename from core/src/main/scala/spark/api/java/function/Function.java rename to core/src/main/scala/org/apache/spark/api/java/function/Function.java index 2a2ea0aacf0..b9070cfd837 100644 --- a/core/src/main/scala/spark/api/java/function/Function.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/Function.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.api.java.function; +package org.apache.spark.api.java.function; import scala.reflect.ClassManifest; import scala.reflect.ClassManifest$; diff --git a/core/src/main/scala/spark/api/java/function/Function2.java b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java similarity index 96% rename from core/src/main/scala/spark/api/java/function/Function2.java rename to core/src/main/scala/org/apache/spark/api/java/function/Function2.java index 952d31ece49..d4c9154869f 100644 --- a/core/src/main/scala/spark/api/java/function/Function2.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.api.java.function; +package org.apache.spark.api.java.function; import scala.reflect.ClassManifest; import scala.reflect.ClassManifest$; diff --git a/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java similarity index 97% rename from core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java rename to core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java index 4aad602da3a..c0e5544b7df 100644 --- a/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.api.java.function; +package org.apache.spark.api.java.function; import scala.Tuple2; import scala.reflect.ClassManifest; diff --git a/core/src/main/scala/spark/api/java/function/PairFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java similarity index 97% rename from core/src/main/scala/spark/api/java/function/PairFunction.java rename to core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java index ccfe64ecf14..40480fe8e81 100644 --- a/core/src/main/scala/spark/api/java/function/PairFunction.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.api.java.function; +package org.apache.spark.api.java.function; import scala.Tuple2; import scala.reflect.ClassManifest; diff --git a/core/src/main/scala/spark/api/java/function/VoidFunction.scala b/core/src/main/scala/org/apache/spark/api/java/function/VoidFunction.scala similarity index 96% rename from core/src/main/scala/spark/api/java/function/VoidFunction.scala rename to core/src/main/scala/org/apache/spark/api/java/function/VoidFunction.scala index f6fc0b0f7db..ea94313a4ab 100644 --- a/core/src/main/scala/spark/api/java/function/VoidFunction.scala +++ b/core/src/main/scala/org/apache/spark/api/java/function/VoidFunction.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.api.java.function +package org.apache.spark.api.java.function /** * A function with no return value. diff --git a/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala b/core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction1.scala similarity index 96% rename from core/src/main/scala/spark/api/java/function/WrappedFunction1.scala rename to core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction1.scala index 1758a38c4e2..cfe694f65d5 100644 --- a/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala +++ b/core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction1.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.api.java.function +package org.apache.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/org/apache/spark/api/java/function/WrappedFunction2.scala similarity index 96% rename from core/src/main/scala/spark/api/java/function/WrappedFunction2.scala rename to core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction2.scala index b093567d2c1..eb9277c6fb4 100644 --- a/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala +++ b/core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction2.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.api.java.function +package org.apache.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/org/apache/spark/api/python/PythonPartitioner.scala similarity index 90% rename from core/src/main/scala/spark/api/python/PythonPartitioner.scala rename to core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala index ac112b8c2c7..eea63d5a4e4 100644 --- a/core/src/main/scala/spark/api/python/PythonPartitioner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.api.python +package org.apache.spark.api.python -import spark.Partitioner -import spark.Utils +import org.apache.spark.Partitioner +import org.apache.spark.Utils import java.util.Arrays /** - * A [[spark.Partitioner]] that performs handling of byte arrays, for use by the Python API. + * A [[org.apache.spark.Partitioner]] that performs handling of byte arrays, for use by the Python API. * * Stores the unique id() of the Python-side partitioning function so that it is incorporated into * equality comparisons. Correctness requires that the id is a unique identifier for the diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala similarity index 97% rename from core/src/main/scala/spark/api/python/PythonRDD.scala rename to core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 49671437d04..621f0fe8ee6 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.api.python +package org.apache.spark.api.python import java.io._ import java.net._ @@ -23,10 +23,10 @@ import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collectio import scala.collection.JavaConversions._ -import spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} -import spark.broadcast.Broadcast -import spark._ -import spark.rdd.PipedRDD +import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark._ +import org.apache.spark.rdd.PipedRDD private[spark] class PythonRDD[T: ClassManifest]( @@ -298,7 +298,7 @@ private object Pickle { val APPENDS: Byte = 'e' } -private class BytesToString extends spark.api.java.function.Function[Array[Byte], String] { +private class BytesToString extends org.apache.spark.api.java.function.Function[Array[Byte], String] { override def call(arr: Array[Byte]) : String = new String(arr, "UTF-8") } diff --git a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala similarity index 98% rename from core/src/main/scala/spark/api/python/PythonWorkerFactory.scala rename to core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 14f83206782..08e3f670f57 100644 --- a/core/src/main/scala/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.api.python +package org.apache.spark.api.python import java.io.{File, DataInputStream, IOException} import java.net.{Socket, SocketException, InetAddress} import scala.collection.JavaConversions._ -import spark._ +import org.apache.spark._ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String, String]) extends Logging { diff --git a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala similarity index 99% rename from core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala rename to core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala index 6f7d3853798..99e86237fc0 100644 --- a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.broadcast +package org.apache.spark.broadcast import java.io._ import java.net._ @@ -25,8 +25,8 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ListBuffer, Map, Set} import scala.math -import spark._ -import spark.storage.StorageLevel +import org.apache.spark._ +import org.apache.spark.storage.StorageLevel private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) extends Broadcast[T](id) diff --git a/core/src/main/scala/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala similarity index 91% rename from core/src/main/scala/spark/broadcast/Broadcast.scala rename to core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala index aba56a60cad..43c18294c55 100644 --- a/core/src/main/scala/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.broadcast +package org.apache.spark.broadcast import java.io._ import java.util.concurrent.atomic.AtomicLong -import spark._ +import org.apache.spark._ abstract class Broadcast[T](private[spark] val id: Long) extends Serializable { def value: T @@ -28,7 +28,7 @@ abstract class Broadcast[T](private[spark] val id: Long) extends Serializable { // We cannot have an abstract readObject here due to some weird issues with // readObject having to be 'private' in sub-classes. - override def toString = "spark.Broadcast(" + id + ")" + override def toString = "Broadcast(" + id + ")" } private[spark] @@ -44,7 +44,7 @@ class BroadcastManager(val _isDriver: Boolean) extends Logging with Serializable synchronized { if (!initialized) { val broadcastFactoryClass = System.getProperty( - "spark.broadcast.factory", "spark.broadcast.HttpBroadcastFactory") + "spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") broadcastFactory = Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory] diff --git a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala similarity index 97% rename from core/src/main/scala/spark/broadcast/BroadcastFactory.scala rename to core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala index d33d95c7d95..68bff75b908 100644 --- a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.broadcast +package org.apache.spark.broadcast /** * An interface for all the broadcast implementations in Spark (to allow diff --git a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala similarity index 95% rename from core/src/main/scala/spark/broadcast/HttpBroadcast.scala rename to core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 138a8c21bcf..7a52ff0769c 100644 --- a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.broadcast +package org.apache.spark.broadcast import java.io.{File, FileOutputStream, ObjectInputStream, OutputStream} import java.net.URL @@ -23,10 +23,10 @@ import java.net.URL import it.unimi.dsi.fastutil.io.FastBufferedInputStream import it.unimi.dsi.fastutil.io.FastBufferedOutputStream -import spark.{HttpServer, Logging, SparkEnv, Utils} -import spark.io.CompressionCodec -import spark.storage.StorageLevel -import spark.util.{MetadataCleaner, TimeStampedHashSet} +import org.apache.spark.{HttpServer, Logging, SparkEnv, Utils} +import org.apache.spark.io.CompressionCodec +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.{MetadataCleaner, TimeStampedHashSet} private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) diff --git a/core/src/main/scala/spark/broadcast/MultiTracker.scala b/core/src/main/scala/org/apache/spark/broadcast/MultiTracker.scala similarity index 99% rename from core/src/main/scala/spark/broadcast/MultiTracker.scala rename to core/src/main/scala/org/apache/spark/broadcast/MultiTracker.scala index 7855d44e9b0..10b910df87e 100644 --- a/core/src/main/scala/spark/broadcast/MultiTracker.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/MultiTracker.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.broadcast +package org.apache.spark.broadcast import java.io._ import java.net._ @@ -23,7 +23,7 @@ import java.util.Random import scala.collection.mutable.Map -import spark._ +import org.apache.spark._ private object MultiTracker extends Logging { diff --git a/core/src/main/scala/spark/broadcast/SourceInfo.scala b/core/src/main/scala/org/apache/spark/broadcast/SourceInfo.scala similarity index 96% rename from core/src/main/scala/spark/broadcast/SourceInfo.scala rename to core/src/main/scala/org/apache/spark/broadcast/SourceInfo.scala index b17ae63b5c4..baa1fd6da46 100644 --- a/core/src/main/scala/spark/broadcast/SourceInfo.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/SourceInfo.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.broadcast +package org.apache.spark.broadcast import java.util.BitSet -import spark._ +import org.apache.spark._ /** * Used to keep and pass around information of peers involved in a broadcast diff --git a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala similarity index 99% rename from core/src/main/scala/spark/broadcast/TreeBroadcast.scala rename to core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala index ea1e9a12c1b..b5a4ccc0eee 100644 --- a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.broadcast +package org.apache.spark.broadcast import java.io._ import java.net._ @@ -24,8 +24,8 @@ import java.util.{Comparator, Random, UUID} import scala.collection.mutable.{ListBuffer, Map, Set} import scala.math -import spark._ -import spark.storage.StorageLevel +import org.apache.spark._ +import org.apache.spark.storage.StorageLevel private[spark] class TreeBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) extends Broadcast[T](id) with Logging with Serializable { diff --git a/core/src/main/scala/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala similarity index 97% rename from core/src/main/scala/spark/deploy/ApplicationDescription.scala rename to core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index a8b22fbef87..19d393a0dbc 100644 --- a/core/src/main/scala/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy +package org.apache.spark.deploy private[spark] class ApplicationDescription( val name: String, diff --git a/core/src/main/scala/spark/deploy/Command.scala b/core/src/main/scala/org/apache/spark/deploy/Command.scala similarity index 96% rename from core/src/main/scala/spark/deploy/Command.scala rename to core/src/main/scala/org/apache/spark/deploy/Command.scala index bad629e9652..fa8af9a6467 100644 --- a/core/src/main/scala/spark/deploy/Command.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Command.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy +package org.apache.spark.deploy import scala.collection.Map diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala similarity index 93% rename from core/src/main/scala/spark/deploy/DeployMessage.scala rename to core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index 0db13ffc987..4dc6ada2d16 100644 --- a/core/src/main/scala/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.deploy +package org.apache.spark.deploy import scala.collection.immutable.List -import spark.Utils -import spark.deploy.ExecutorState.ExecutorState -import spark.deploy.master.{WorkerInfo, ApplicationInfo} -import spark.deploy.worker.ExecutorRunner +import org.apache.spark.Utils +import org.apache.spark.deploy.ExecutorState.ExecutorState +import org.apache.spark.deploy.master.{WorkerInfo, ApplicationInfo} +import org.apache.spark.deploy.worker.ExecutorRunner private[deploy] sealed trait DeployMessage extends Serializable diff --git a/core/src/main/scala/spark/deploy/ExecutorState.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala similarity index 97% rename from core/src/main/scala/spark/deploy/ExecutorState.scala rename to core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala index 08c9a3b7252..fcfea96ad60 100644 --- a/core/src/main/scala/spark/deploy/ExecutorState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy +package org.apache.spark.deploy private[spark] object ExecutorState extends Enumeration("LAUNCHING", "LOADING", "RUNNING", "KILLED", "FAILED", "LOST") { diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala similarity index 92% rename from core/src/main/scala/spark/deploy/JsonProtocol.scala rename to core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index f8dcf025b44..a6be8efef18 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.deploy +package org.apache.spark.deploy import net.liftweb.json.JsonDSL._ -import spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} -import spark.deploy.master.{ApplicationInfo, WorkerInfo} -import spark.deploy.worker.ExecutorRunner +import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} +import org.apache.spark.deploy.master.{ApplicationInfo, WorkerInfo} +import org.apache.spark.deploy.worker.ExecutorRunner private[spark] object JsonProtocol { diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala similarity index 92% rename from core/src/main/scala/spark/deploy/LocalSparkCluster.scala rename to core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index 6b8e9f27af7..af5a4110b07 100644 --- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.deploy +package org.apache.spark.deploy import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} -import spark.deploy.worker.Worker -import spark.deploy.master.Master -import spark.util.AkkaUtils -import spark.{Logging, Utils} +import org.apache.spark.deploy.worker.Worker +import org.apache.spark.deploy.master.Master +import org.apache.spark.util.AkkaUtils +import org.apache.spark.{Logging, Utils} import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala similarity index 97% rename from core/src/main/scala/spark/deploy/SparkHadoopUtil.scala rename to core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 882161e6699..0a5f4c368f3 100644 --- a/core/src/main/scala/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy +package org.apache.spark.deploy import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.JobConf diff --git a/core/src/main/scala/spark/deploy/WebUI.scala b/core/src/main/scala/org/apache/spark/deploy/WebUI.scala similarity index 98% rename from core/src/main/scala/spark/deploy/WebUI.scala rename to core/src/main/scala/org/apache/spark/deploy/WebUI.scala index 8ea7792ef4e..ae258b58b9c 100644 --- a/core/src/main/scala/spark/deploy/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/WebUI.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy +package org.apache.spark.deploy import java.text.SimpleDateFormat import java.util.Date diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala similarity index 95% rename from core/src/main/scala/spark/deploy/client/Client.scala rename to core/src/main/scala/org/apache/spark/deploy/client/Client.scala index 9d5ba8a7960..a342dd724a8 100644 --- a/core/src/main/scala/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy.client +package org.apache.spark.deploy.client import java.util.concurrent.TimeoutException @@ -28,10 +28,10 @@ import akka.remote.RemoteClientLifeCycleEvent import akka.remote.RemoteClientShutdown import akka.dispatch.Await -import spark.Logging -import spark.deploy.{ApplicationDescription, ExecutorState} -import spark.deploy.DeployMessages._ -import spark.deploy.master.Master +import org.apache.spark.Logging +import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} +import org.apache.spark.deploy.DeployMessages._ +import org.apache.spark.deploy.master.Master /** diff --git a/core/src/main/scala/spark/deploy/client/ClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala similarity index 97% rename from core/src/main/scala/spark/deploy/client/ClientListener.scala rename to core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala index 064024455ee..4605368c117 100644 --- a/core/src/main/scala/spark/deploy/client/ClientListener.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy.client +package org.apache.spark.deploy.client /** * Callbacks invoked by deploy client when various events happen. There are currently four events: diff --git a/core/src/main/scala/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala similarity index 90% rename from core/src/main/scala/spark/deploy/client/TestClient.scala rename to core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index 4f4daa141a4..0322029fbd0 100644 --- a/core/src/main/scala/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.deploy.client +package org.apache.spark.deploy.client -import spark.util.AkkaUtils -import spark.{Logging, Utils} -import spark.deploy.{Command, ApplicationDescription} +import org.apache.spark.util.AkkaUtils +import org.apache.spark.{Logging, Utils} +import org.apache.spark.deploy.{Command, ApplicationDescription} private[spark] object TestClient { diff --git a/core/src/main/scala/spark/deploy/client/TestExecutor.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestExecutor.scala similarity index 96% rename from core/src/main/scala/spark/deploy/client/TestExecutor.scala rename to core/src/main/scala/org/apache/spark/deploy/client/TestExecutor.scala index 8a22b6b89fa..c5ac45c6730 100644 --- a/core/src/main/scala/spark/deploy/client/TestExecutor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestExecutor.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy.client +package org.apache.spark.deploy.client private[spark] object TestExecutor { def main(args: Array[String]) { diff --git a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala similarity index 96% rename from core/src/main/scala/spark/deploy/master/ApplicationInfo.scala rename to core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index 6dd2f061265..bd5327627a8 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.deploy.master +package org.apache.spark.deploy.master -import spark.deploy.ApplicationDescription +import org.apache.spark.deploy.ApplicationDescription import java.util.Date import akka.actor.ActorRef import scala.collection.mutable diff --git a/core/src/main/scala/spark/deploy/master/ApplicationSource.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala similarity index 89% rename from core/src/main/scala/spark/deploy/master/ApplicationSource.scala rename to core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala index 4df2b6bfddd..2d75ad5a2c9 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala @@ -1,8 +1,8 @@ -package spark.deploy.master +package org.apache.spark.deploy.master import com.codahale.metrics.{Gauge, MetricRegistry} -import spark.metrics.source.Source +import org.apache.spark.metrics.source.Source class ApplicationSource(val application: ApplicationInfo) extends Source { val metricRegistry = new MetricRegistry() diff --git a/core/src/main/scala/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala similarity index 96% rename from core/src/main/scala/spark/deploy/master/ApplicationState.scala rename to core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala index 94f0ad8baec..7e804223cf4 100644 --- a/core/src/main/scala/spark/deploy/master/ApplicationState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy.master +package org.apache.spark.deploy.master private[spark] object ApplicationState extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED") { diff --git a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala similarity index 92% rename from core/src/main/scala/spark/deploy/master/ExecutorInfo.scala rename to core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala index 99b60f7d092..cf384a985e9 100644 --- a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.deploy.master +package org.apache.spark.deploy.master -import spark.deploy.ExecutorState +import org.apache.spark.deploy.ExecutorState private[spark] class ExecutorInfo( val id: Int, diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala similarity index 97% rename from core/src/main/scala/spark/deploy/master/Master.scala rename to core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 04af5e149c6..869b2b2646d 100644 --- a/core/src/main/scala/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy.master +package org.apache.spark.deploy.master import java.text.SimpleDateFormat import java.util.Date @@ -27,12 +27,12 @@ import akka.actor.Terminated import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown} import akka.util.duration._ -import spark.{Logging, SparkException, Utils} -import spark.deploy.{ApplicationDescription, ExecutorState} -import spark.deploy.DeployMessages._ -import spark.deploy.master.ui.MasterWebUI -import spark.metrics.MetricsSystem -import spark.util.AkkaUtils +import org.apache.spark.{Logging, SparkException, Utils} +import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} +import org.apache.spark.deploy.DeployMessages._ +import org.apache.spark.deploy.master.ui.MasterWebUI +import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.util.AkkaUtils private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { diff --git a/core/src/main/scala/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala similarity index 96% rename from core/src/main/scala/spark/deploy/master/MasterArguments.scala rename to core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index 0ae01607673..c86cca278d0 100644 --- a/core/src/main/scala/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.deploy.master +package org.apache.spark.deploy.master -import spark.util.IntParam -import spark.Utils +import org.apache.spark.util.IntParam +import org.apache.spark.Utils /** * Command-line parser for the master. diff --git a/core/src/main/scala/spark/deploy/master/MasterSource.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala similarity index 90% rename from core/src/main/scala/spark/deploy/master/MasterSource.scala rename to core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala index b8cfa6a7736..8dd0a42f717 100644 --- a/core/src/main/scala/spark/deploy/master/MasterSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala @@ -1,8 +1,8 @@ -package spark.deploy.master +package org.apache.spark.deploy.master import com.codahale.metrics.{Gauge, MetricRegistry} -import spark.metrics.source.Source +import org.apache.spark.metrics.source.Source private[spark] class MasterSource(val master: Master) extends Source { val metricRegistry = new MetricRegistry() diff --git a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala similarity index 96% rename from core/src/main/scala/spark/deploy/master/WorkerInfo.scala rename to core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala index 4135cfeb284..285e07a8232 100644 --- a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.deploy.master +package org.apache.spark.deploy.master import akka.actor.ActorRef import scala.collection.mutable -import spark.Utils +import org.apache.spark.Utils private[spark] class WorkerInfo( val id: String, diff --git a/core/src/main/scala/spark/deploy/master/WorkerState.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala similarity index 96% rename from core/src/main/scala/spark/deploy/master/WorkerState.scala rename to core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala index 3e50b7748d6..b5ee6dca79f 100644 --- a/core/src/main/scala/spark/deploy/master/WorkerState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy.master +package org.apache.spark.deploy.master private[spark] object WorkerState extends Enumeration("ALIVE", "DEAD", "DECOMMISSIONED") { type WorkerState = Value diff --git a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala similarity index 93% rename from core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala rename to core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 2ad98f759cb..6435c7f917c 100644 --- a/core/src/main/scala/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy.master.ui +package org.apache.spark.deploy.master.ui import scala.xml.Node @@ -27,11 +27,11 @@ import javax.servlet.http.HttpServletRequest import net.liftweb.json.JsonAST.JValue -import spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} -import spark.deploy.JsonProtocol -import spark.deploy.master.ExecutorInfo -import spark.ui.UIUtils -import spark.Utils +import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} +import org.apache.spark.deploy.JsonProtocol +import org.apache.spark.deploy.master.ExecutorInfo +import org.apache.spark.ui.UIUtils +import org.apache.spark.Utils private[spark] class ApplicationPage(parent: MasterWebUI) { val master = parent.masterActorRef diff --git a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala similarity index 93% rename from core/src/main/scala/spark/deploy/master/ui/IndexPage.scala rename to core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala index 093e523e239..58d3863009e 100644 --- a/core/src/main/scala/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy.master.ui +package org.apache.spark.deploy.master.ui import javax.servlet.http.HttpServletRequest @@ -27,12 +27,12 @@ import akka.util.duration._ import net.liftweb.json.JsonAST.JValue -import spark.Utils -import spark.deploy.DeployWebUI -import spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} -import spark.deploy.JsonProtocol -import spark.deploy.master.{ApplicationInfo, WorkerInfo} -import spark.ui.UIUtils +import org.apache.spark.Utils +import org.apache.spark.deploy.DeployWebUI +import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} +import org.apache.spark.deploy.JsonProtocol +import org.apache.spark.deploy.master.{ApplicationInfo, WorkerInfo} +import org.apache.spark.ui.UIUtils private[spark] class IndexPage(parent: MasterWebUI) { val master = parent.masterActorRef diff --git a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala similarity index 90% rename from core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala rename to core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index c91e1db9f24..47b1e521f5c 100644 --- a/core/src/main/scala/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy.master.ui +package org.apache.spark.deploy.master.ui import akka.util.Duration @@ -23,10 +23,10 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} -import spark.{Logging, Utils} -import spark.deploy.master.Master -import spark.ui.JettyUtils -import spark.ui.JettyUtils._ +import org.apache.spark.{Logging, Utils} +import org.apache.spark.deploy.master.Master +import org.apache.spark.ui.JettyUtils +import org.apache.spark.ui.JettyUtils._ /** * Web UI server for the standalone master. @@ -76,5 +76,5 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { } private[spark] object MasterWebUI { - val STATIC_RESOURCE_DIR = "spark/ui/static" + val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" } diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala similarity index 96% rename from core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala rename to core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 34665ce451a..01ce4a6deab 100644 --- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy.worker +package org.apache.spark.deploy.worker import java.io._ import java.lang.System.getenv @@ -25,9 +25,9 @@ import akka.actor.ActorRef import com.google.common.base.Charsets import com.google.common.io.Files -import spark.{Utils, Logging} -import spark.deploy.{ExecutorState, ApplicationDescription} -import spark.deploy.DeployMessages.ExecutorStateChanged +import org.apache.spark.{Utils, Logging} +import org.apache.spark.deploy.{ExecutorState, ApplicationDescription} +import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged /** * Manages the execution of one executor process. diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala similarity index 95% rename from core/src/main/scala/spark/deploy/worker/Worker.scala rename to core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 053ac552260..86e8e7543ba 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy.worker +package org.apache.spark.deploy.worker import java.text.SimpleDateFormat import java.util.Date @@ -27,13 +27,13 @@ import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import akka.util.duration._ -import spark.{Logging, Utils} -import spark.deploy.ExecutorState -import spark.deploy.DeployMessages._ -import spark.deploy.master.Master -import spark.deploy.worker.ui.WorkerWebUI -import spark.metrics.MetricsSystem -import spark.util.AkkaUtils +import org.apache.spark.{Logging, Utils} +import org.apache.spark.deploy.ExecutorState +import org.apache.spark.deploy.DeployMessages._ +import org.apache.spark.deploy.master.Master +import org.apache.spark.deploy.worker.ui.WorkerWebUI +import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.util.AkkaUtils private[spark] class Worker( diff --git a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala similarity index 97% rename from core/src/main/scala/spark/deploy/worker/WorkerArguments.scala rename to core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index 9fcd3260ca9..6d91223413e 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.deploy.worker +package org.apache.spark.deploy.worker -import spark.util.IntParam -import spark.util.MemoryParam -import spark.Utils +import org.apache.spark.util.IntParam +import org.apache.spark.util.MemoryParam +import org.apache.spark.Utils import java.lang.management.ManagementFactory /** diff --git a/core/src/main/scala/spark/deploy/worker/WorkerSource.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala similarity index 92% rename from core/src/main/scala/spark/deploy/worker/WorkerSource.scala rename to core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala index 39cb8e56901..6427c0178fa 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala @@ -1,8 +1,8 @@ -package spark.deploy.worker +package org.apache.spark.deploy.worker import com.codahale.metrics.{Gauge, MetricRegistry} -import spark.metrics.source.Source +import org.apache.spark.metrics.source.Source private[spark] class WorkerSource(val worker: Worker) extends Source { val sourceName = "worker" diff --git a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala similarity index 93% rename from core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala rename to core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala index 243e0765cb7..6192c2324bd 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy.worker.ui +package org.apache.spark.deploy.worker.ui import javax.servlet.http.HttpServletRequest @@ -27,11 +27,11 @@ import akka.util.duration._ import net.liftweb.json.JsonAST.JValue -import spark.Utils -import spark.deploy.JsonProtocol -import spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} -import spark.deploy.worker.ExecutorRunner -import spark.ui.UIUtils +import org.apache.spark.Utils +import org.apache.spark.deploy.JsonProtocol +import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} +import org.apache.spark.deploy.worker.ExecutorRunner +import org.apache.spark.ui.UIUtils private[spark] class IndexPage(parent: WorkerWebUI) { diff --git a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala similarity index 95% rename from core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala rename to core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 0a75ad8cf4a..bb8165ac098 100644 --- a/core/src/main/scala/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy.worker.ui +package org.apache.spark.deploy.worker.ui import akka.util.{Duration, Timeout} @@ -25,11 +25,11 @@ 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._ -import spark.ui.UIUtils +import org.apache.spark.deploy.worker.Worker +import org.apache.spark.{Utils, Logging} +import org.apache.spark.ui.JettyUtils +import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.ui.UIUtils /** * Web UI server for the standalone worker. @@ -185,6 +185,6 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } private[spark] object WorkerWebUI { - val STATIC_RESOURCE_DIR = "spark/ui/static" + val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" val DEFAULT_PORT="8081" } diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala similarity index 97% rename from core/src/main/scala/spark/executor/Executor.scala rename to core/src/main/scala/org/apache/spark/executor/Executor.scala index fa82d2b3245..5446a3fca9f 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.executor +package org.apache.spark.executor import java.io.{File} import java.lang.management.ManagementFactory @@ -25,8 +25,8 @@ import java.util.concurrent._ import scala.collection.JavaConversions._ import scala.collection.mutable.HashMap -import spark.scheduler._ -import spark._ +import org.apache.spark.scheduler._ +import org.apache.spark._ /** @@ -225,13 +225,13 @@ private[spark] class Executor( if (classUri != null) { logInfo("Using REPL class URI: " + classUri) try { - val klass = Class.forName("spark.repl.ExecutorClassLoader") + val klass = Class.forName("org.apache.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!") + logError("Could not find org.apache.spark.repl.ExecutorClassLoader on classpath!") System.exit(1) null } diff --git a/core/src/main/scala/spark/executor/ExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala similarity index 93% rename from core/src/main/scala/spark/executor/ExecutorBackend.scala rename to core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala index 33a6f8a824e..ad7dd34c769 100644 --- a/core/src/main/scala/spark/executor/ExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorBackend.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.executor +package org.apache.spark.executor import java.nio.ByteBuffer -import spark.TaskState.TaskState +import org.apache.spark.TaskState.TaskState /** * A pluggable interface used by the Executor to send updates to the cluster scheduler. diff --git a/core/src/main/scala/spark/executor/ExecutorExitCode.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala similarity index 98% rename from core/src/main/scala/spark/executor/ExecutorExitCode.scala rename to core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala index 64b9fb88f89..e5c9bbbe287 100644 --- a/core/src/main/scala/spark/executor/ExecutorExitCode.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.executor +package org.apache.spark.executor /** * These are exit codes that executors should use to provide the master with information about diff --git a/core/src/main/scala/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala similarity index 96% rename from core/src/main/scala/spark/executor/ExecutorSource.scala rename to core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index d491a3c0c95..17653cd5608 100644 --- a/core/src/main/scala/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -1,4 +1,4 @@ -package spark.executor +package org.apache.spark.executor import com.codahale.metrics.{Gauge, MetricRegistry} @@ -8,7 +8,7 @@ import org.apache.hadoop.fs.LocalFileSystem import scala.collection.JavaConversions._ -import spark.metrics.source.Source +import org.apache.spark.metrics.source.Source class ExecutorSource(val executor: Executor) extends Source { private def fileStats(scheme: String) : Option[FileSystem.Statistics] = diff --git a/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala similarity index 97% rename from core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala rename to core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala index 09d12fb65b9..f9bfe8ed2f5 100644 --- a/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.executor +package org.apache.spark.executor import java.net.{URLClassLoader, URL} diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala similarity index 95% rename from core/src/main/scala/spark/executor/MesosExecutorBackend.scala rename to core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 4961c42fadd..410a94df6bf 100644 --- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -15,15 +15,15 @@ * limitations under the License. */ -package spark.executor +package org.apache.spark.executor import java.nio.ByteBuffer import org.apache.mesos.{Executor => MesosExecutor, MesosExecutorDriver, MesosNativeLibrary, ExecutorDriver} import org.apache.mesos.Protos.{TaskState => MesosTaskState, TaskStatus => MesosTaskStatus, _} -import spark.TaskState.TaskState +import org.apache.spark.TaskState.TaskState import com.google.protobuf.ByteString -import spark.{Utils, Logging} -import spark.TaskState +import org.apache.spark.{Utils, Logging} +import org.apache.spark.TaskState private[spark] class MesosExecutorBackend extends MesosExecutor diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala similarity index 94% rename from core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala rename to core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala index b5fb6dbe29e..65801f75b7d 100644 --- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala @@ -15,17 +15,17 @@ * limitations under the License. */ -package spark.executor +package org.apache.spark.executor import java.nio.ByteBuffer import akka.actor.{ActorRef, Actor, Props, Terminated} import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} -import spark.{Logging, Utils, SparkEnv} -import spark.TaskState.TaskState -import spark.scheduler.cluster.StandaloneClusterMessages._ -import spark.util.AkkaUtils +import org.apache.spark.{Logging, Utils, SparkEnv} +import org.apache.spark.TaskState.TaskState +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages._ +import org.apache.spark.util.AkkaUtils private[spark] class StandaloneExecutorBackend( diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala similarity index 98% rename from core/src/main/scala/spark/executor/TaskMetrics.scala rename to core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 47b8890bee9..f311141148c 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.executor +package org.apache.spark.executor class TaskMetrics extends Serializable { /** diff --git a/core/src/main/scala/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala similarity index 94% rename from core/src/main/scala/spark/io/CompressionCodec.scala rename to core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 0adebecadb3..90a0420cafb 100644 --- a/core/src/main/scala/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.io +package org.apache.spark.io import java.io.{InputStream, OutputStream} @@ -55,7 +55,7 @@ private[spark] object CompressionCodec { /** - * LZF implementation of [[spark.io.CompressionCodec]]. + * LZF implementation of [[org.apache.spark.io.CompressionCodec]]. */ class LZFCompressionCodec extends CompressionCodec { @@ -68,7 +68,7 @@ class LZFCompressionCodec extends CompressionCodec { /** - * Snappy implementation of [[spark.io.CompressionCodec]]. + * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. * Block size can be configured by spark.io.compression.snappy.block.size. */ class SnappyCompressionCodec extends CompressionCodec { diff --git a/core/src/main/scala/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala similarity index 95% rename from core/src/main/scala/spark/metrics/MetricsConfig.scala rename to core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala index d7fb5378a49..0f9c4e00b1f 100644 --- a/core/src/main/scala/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.metrics +package org.apache.spark.metrics import java.util.Properties import java.io.{File, FileInputStream, InputStream, IOException} @@ -23,7 +23,7 @@ import java.io.{File, FileInputStream, InputStream, IOException} import scala.collection.mutable import scala.util.matching.Regex -import spark.Logging +import org.apache.spark.Logging private[spark] class MetricsConfig(val configFile: Option[String]) extends Logging { initLogging() @@ -36,7 +36,7 @@ private[spark] class MetricsConfig(val configFile: Option[String]) extends Loggi var propertyCategories: mutable.HashMap[String, Properties] = null private def setDefaultProperties(prop: Properties) { - prop.setProperty("*.sink.servlet.class", "spark.metrics.sink.MetricsServlet") + prop.setProperty("*.sink.servlet.class", "org.apache.spark.metrics.sink.MetricsServlet") prop.setProperty("*.sink.servlet.uri", "/metrics/json") prop.setProperty("*.sink.servlet.sample", "false") prop.setProperty("master.sink.servlet.uri", "/metrics/master/json") diff --git a/core/src/main/scala/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala similarity index 97% rename from core/src/main/scala/spark/metrics/MetricsSystem.scala rename to core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 4e6c6b26c86..bec0c83be8b 100644 --- a/core/src/main/scala/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.metrics +package org.apache.spark.metrics import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry} @@ -24,9 +24,9 @@ import java.util.concurrent.TimeUnit import scala.collection.mutable -import spark.Logging -import spark.metrics.sink.{MetricsServlet, Sink} -import spark.metrics.source.Source +import org.apache.spark.Logging +import org.apache.spark.metrics.sink.{MetricsServlet, Sink} +import org.apache.spark.metrics.source.Source /** * Spark Metrics System, created by specific "instance", combined by source, diff --git a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala similarity index 95% rename from core/src/main/scala/spark/metrics/sink/ConsoleSink.scala rename to core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala index 966ba37c20e..bce257d6e6f 100644 --- a/core/src/main/scala/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.metrics.sink +package org.apache.spark.metrics.sink import com.codahale.metrics.{ConsoleReporter, MetricRegistry} import java.util.Properties import java.util.concurrent.TimeUnit -import spark.metrics.MetricsSystem +import org.apache.spark.metrics.MetricsSystem class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { val CONSOLE_DEFAULT_PERIOD = 10 diff --git a/core/src/main/scala/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala similarity index 96% rename from core/src/main/scala/spark/metrics/sink/CsvSink.scala rename to core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala index cb990afdefc..3d1a06a395a 100644 --- a/core/src/main/scala/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.metrics.sink +package org.apache.spark.metrics.sink import com.codahale.metrics.{CsvReporter, MetricRegistry} @@ -23,7 +23,7 @@ import java.io.File import java.util.{Locale, Properties} import java.util.concurrent.TimeUnit -import spark.metrics.MetricsSystem +import org.apache.spark.metrics.MetricsSystem class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { val CSV_KEY_PERIOD = "period" diff --git a/core/src/main/scala/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala similarity index 96% rename from core/src/main/scala/spark/metrics/sink/JmxSink.scala rename to core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala index ee04544c0e9..621d086d415 100644 --- a/core/src/main/scala/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.metrics.sink +package org.apache.spark.metrics.sink import com.codahale.metrics.{JmxReporter, MetricRegistry} diff --git a/core/src/main/scala/spark/metrics/sink/MetricsServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala similarity index 96% rename from core/src/main/scala/spark/metrics/sink/MetricsServlet.scala rename to core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala index 17432b1ed1a..4e90dd43239 100644 --- a/core/src/main/scala/spark/metrics/sink/MetricsServlet.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.metrics.sink +package org.apache.spark.metrics.sink import com.codahale.metrics.MetricRegistry import com.codahale.metrics.json.MetricsModule @@ -28,7 +28,7 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler -import spark.ui.JettyUtils +import org.apache.spark.ui.JettyUtils class MetricsServlet(val property: Properties, val registry: MetricRegistry) extends Sink { val SERVLET_KEY_URI = "uri" diff --git a/core/src/main/scala/spark/metrics/sink/Sink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala similarity index 95% rename from core/src/main/scala/spark/metrics/sink/Sink.scala rename to core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala index dad1a7f0fe7..3a739aa563e 100644 --- a/core/src/main/scala/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.metrics.sink +package org.apache.spark.metrics.sink trait Sink { def start: 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/org/apache/spark/metrics/source/JvmSource.scala similarity index 96% rename from core/src/main/scala/spark/metrics/source/JvmSource.scala rename to core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala index e7710085577..75cb2b8973a 100644 --- a/core/src/main/scala/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.metrics.source +package org.apache.spark.metrics.source import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} diff --git a/core/src/main/scala/spark/metrics/source/Source.scala b/core/src/main/scala/org/apache/spark/metrics/source/Source.scala similarity index 95% rename from core/src/main/scala/spark/metrics/source/Source.scala rename to core/src/main/scala/org/apache/spark/metrics/source/Source.scala index 76199a004b5..3fee55cc6dc 100644 --- a/core/src/main/scala/spark/metrics/source/Source.scala +++ b/core/src/main/scala/org/apache/spark/metrics/source/Source.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.metrics.source +package org.apache.spark.metrics.source import com.codahale.metrics.MetricRegistry diff --git a/core/src/main/scala/spark/network/BufferMessage.scala b/core/src/main/scala/org/apache/spark/network/BufferMessage.scala similarity index 97% rename from core/src/main/scala/spark/network/BufferMessage.scala rename to core/src/main/scala/org/apache/spark/network/BufferMessage.scala index e566aeac137..f736bb37130 100644 --- a/core/src/main/scala/spark/network/BufferMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/BufferMessage.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.network +package org.apache.spark.network import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import spark.storage.BlockManager +import org.apache.spark.storage.BlockManager private[spark] diff --git a/core/src/main/scala/spark/network/Connection.scala b/core/src/main/scala/org/apache/spark/network/Connection.scala similarity index 99% rename from core/src/main/scala/spark/network/Connection.scala rename to core/src/main/scala/org/apache/spark/network/Connection.scala index 1e571d39ae5..95cb0206acd 100644 --- a/core/src/main/scala/spark/network/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/Connection.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.network +package org.apache.spark.network -import spark._ +import org.apache.spark._ import scala.collection.mutable.{HashMap, Queue, ArrayBuffer} diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala similarity index 99% rename from core/src/main/scala/spark/network/ConnectionManager.scala rename to core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 8b9f3ae18c5..9e2233c07b5 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.network +package org.apache.spark.network -import spark._ +import org.apache.spark._ import java.nio._ import java.nio.channels._ diff --git a/core/src/main/scala/spark/network/ConnectionManagerId.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala similarity index 95% rename from core/src/main/scala/spark/network/ConnectionManagerId.scala rename to core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala index 9d5c518293d..0839c011b8f 100644 --- a/core/src/main/scala/spark/network/ConnectionManagerId.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.network +package org.apache.spark.network import java.net.InetSocketAddress -import spark.Utils +import org.apache.spark.Utils private[spark] case class ConnectionManagerId(host: String, port: Int) { diff --git a/core/src/main/scala/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala similarity index 97% rename from core/src/main/scala/spark/network/ConnectionManagerTest.scala rename to core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala index 9e3827aaf56..8d9ad9604d6 100644 --- a/core/src/main/scala/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.network +package org.apache.spark.network -import spark._ -import spark.SparkContext._ +import org.apache.spark._ +import org.apache.spark.SparkContext._ import scala.io.Source diff --git a/core/src/main/scala/spark/network/Message.scala b/core/src/main/scala/org/apache/spark/network/Message.scala similarity index 98% rename from core/src/main/scala/spark/network/Message.scala rename to core/src/main/scala/org/apache/spark/network/Message.scala index a25457ea35f..f2ecc6d439a 100644 --- a/core/src/main/scala/spark/network/Message.scala +++ b/core/src/main/scala/org/apache/spark/network/Message.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.network +package org.apache.spark.network import java.nio.ByteBuffer import java.net.InetSocketAddress diff --git a/core/src/main/scala/spark/network/MessageChunk.scala b/core/src/main/scala/org/apache/spark/network/MessageChunk.scala similarity index 97% rename from core/src/main/scala/spark/network/MessageChunk.scala rename to core/src/main/scala/org/apache/spark/network/MessageChunk.scala index 784db5ab62d..e0fe57b80d5 100644 --- a/core/src/main/scala/spark/network/MessageChunk.scala +++ b/core/src/main/scala/org/apache/spark/network/MessageChunk.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.network +package org.apache.spark.network import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/network/MessageChunkHeader.scala b/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala similarity index 98% rename from core/src/main/scala/spark/network/MessageChunkHeader.scala rename to core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala index 18d0cbcc146..235fbc39b3b 100644 --- a/core/src/main/scala/spark/network/MessageChunkHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.network +package org.apache.spark.network import java.net.InetAddress import java.net.InetSocketAddress diff --git a/core/src/main/scala/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala similarity index 97% rename from core/src/main/scala/spark/network/ReceiverTest.scala rename to core/src/main/scala/org/apache/spark/network/ReceiverTest.scala index 2bbc736f405..781715108be 100644 --- a/core/src/main/scala/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.network +package org.apache.spark.network import java.nio.ByteBuffer import java.net.InetAddress diff --git a/core/src/main/scala/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala similarity index 98% rename from core/src/main/scala/spark/network/SenderTest.scala rename to core/src/main/scala/org/apache/spark/network/SenderTest.scala index 542c54c36b0..777574980fb 100644 --- a/core/src/main/scala/spark/network/SenderTest.scala +++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.network +package org.apache.spark.network import java.nio.ByteBuffer import java.net.InetAddress diff --git a/core/src/main/scala/spark/network/netty/FileHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala similarity index 96% rename from core/src/main/scala/spark/network/netty/FileHeader.scala rename to core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala index bf46d32aa3a..3c29700920a 100644 --- a/core/src/main/scala/spark/network/netty/FileHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.network.netty +package org.apache.spark.network.netty import io.netty.buffer._ -import spark.Logging +import org.apache.spark.Logging private[spark] class FileHeader ( val fileLen: Int, diff --git a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala similarity index 96% rename from core/src/main/scala/spark/network/netty/ShuffleCopier.scala rename to core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala index b01f6369f65..9493ccffd99 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleCopier.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.network.netty +package org.apache.spark.network.netty import java.util.concurrent.Executors @@ -23,8 +23,8 @@ import io.netty.buffer.ByteBuf import io.netty.channel.ChannelHandlerContext import io.netty.util.CharsetUtil -import spark.Logging -import spark.network.ConnectionManagerId +import org.apache.spark.Logging +import org.apache.spark.network.ConnectionManagerId import scala.collection.JavaConverters._ diff --git a/core/src/main/scala/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala similarity index 96% rename from core/src/main/scala/spark/network/netty/ShuffleSender.scala rename to core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala index cdf88b03a08..537f225469a 100644 --- a/core/src/main/scala/spark/network/netty/ShuffleSender.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.network.netty +package org.apache.spark.network.netty import java.io.File -import spark.Logging +import org.apache.spark.Logging private[spark] class ShuffleSender(portIn: Int, val pResolver: PathResolver) extends Logging { diff --git a/core/src/main/scala/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala similarity index 65% rename from core/src/main/scala/spark/package.scala rename to core/src/main/scala/org/apache/spark/package.scala index b244bfbf069..11264806890 100644 --- a/core/src/main/scala/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -16,16 +16,16 @@ */ /** - * 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 + * Core Spark functionality. [[org.apache.spark.SparkContext]] serves as the main entry point to Spark, while + * [[org.apache.spark.RDD]] is the data type representing a distributed collection, and provides most * parallel operations. * - * In addition, [[spark.PairRDDFunctions]] contains operations available only on RDDs of key-value - * pairs, such as `groupByKey` and `join`; [[spark.DoubleRDDFunctions]] contains operations - * available only on RDDs of Doubles; and [[spark.SequenceFileRDDFunctions]] contains operations + * In addition, [[org.apache.spark.PairRDDFunctions]] contains operations available only on RDDs of key-value + * pairs, such as `groupByKey` and `join`; [[org.apache.spark.DoubleRDDFunctions]] contains operations + * available only on RDDs of Doubles; and [[org.apache.spark.SequenceFileRDDFunctions]] contains operations * available on RDDs that can be saved as SequenceFiles. These operations are automatically * available on any RDD of the right type (e.g. RDD[(Int, Int)] through implicit conversions when - * you `import spark.SparkContext._`. + * you `import org.apache.spark.SparkContext._`. */ package object spark { // For package docs only diff --git a/core/src/main/scala/spark/partial/ApproximateActionListener.scala b/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala similarity index 96% rename from core/src/main/scala/spark/partial/ApproximateActionListener.scala rename to core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala index 691d939150a..c5d51bee507 100644 --- a/core/src/main/scala/spark/partial/ApproximateActionListener.scala +++ b/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.partial +package org.apache.spark.partial -import spark._ -import spark.scheduler.JobListener +import org.apache.spark._ +import org.apache.spark.scheduler.JobListener /** * A JobListener for an approximate single-result action, such as count() or non-parallel reduce(). diff --git a/core/src/main/scala/spark/partial/ApproximateEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/ApproximateEvaluator.scala similarity index 97% rename from core/src/main/scala/spark/partial/ApproximateEvaluator.scala rename to core/src/main/scala/org/apache/spark/partial/ApproximateEvaluator.scala index 5eae144dfbc..9c2859c8b98 100644 --- a/core/src/main/scala/spark/partial/ApproximateEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/ApproximateEvaluator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.partial +package org.apache.spark.partial /** * An object that computes a function incrementally by merging in results of type U from multiple diff --git a/core/src/main/scala/spark/partial/BoundedDouble.scala b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala similarity index 96% rename from core/src/main/scala/spark/partial/BoundedDouble.scala rename to core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala index 8bdbe6c0123..5f4450859cc 100644 --- a/core/src/main/scala/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.partial +package org.apache.spark.partial /** * A Double with error bars on it. diff --git a/core/src/main/scala/spark/partial/CountEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala similarity index 98% rename from core/src/main/scala/spark/partial/CountEvaluator.scala rename to core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala index 6aa92094eb4..3155dfe1656 100644 --- a/core/src/main/scala/spark/partial/CountEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.partial +package org.apache.spark.partial import cern.jet.stat.Probability diff --git a/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala similarity index 98% rename from core/src/main/scala/spark/partial/GroupedCountEvaluator.scala rename to core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala index ebe2e5a1e3a..e519e3a5484 100644 --- a/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.partial +package org.apache.spark.partial import java.util.{HashMap => JHashMap} import java.util.{Map => JMap} diff --git a/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala similarity index 97% rename from core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala rename to core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala index 2dadbbd5fb4..cf8a5680b66 100644 --- a/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.partial +package org.apache.spark.partial import java.util.{HashMap => JHashMap} import java.util.{Map => JMap} @@ -24,7 +24,7 @@ import scala.collection.mutable.HashMap import scala.collection.Map import scala.collection.JavaConversions.mapAsScalaMap -import spark.util.StatCounter +import org.apache.spark.util.StatCounter /** * An ApproximateEvaluator for means by key. Returns a map of key to confidence interval. diff --git a/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala similarity index 97% rename from core/src/main/scala/spark/partial/GroupedSumEvaluator.scala rename to core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala index ae2b63f7cb7..8225a5d933c 100644 --- a/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedSumEvaluator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.partial +package org.apache.spark.partial import java.util.{HashMap => JHashMap} import java.util.{Map => JMap} @@ -24,7 +24,7 @@ import scala.collection.mutable.HashMap import scala.collection.Map import scala.collection.JavaConversions.mapAsScalaMap -import spark.util.StatCounter +import org.apache.spark.util.StatCounter /** * An ApproximateEvaluator for sums by key. Returns a map of key to confidence interval. diff --git a/core/src/main/scala/spark/partial/MeanEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala similarity index 96% rename from core/src/main/scala/spark/partial/MeanEvaluator.scala rename to core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala index 5ddcad70759..d24959cba87 100644 --- a/core/src/main/scala/spark/partial/MeanEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.partial +package org.apache.spark.partial import cern.jet.stat.Probability -import spark.util.StatCounter +import org.apache.spark.util.StatCounter /** * An ApproximateEvaluator for means. diff --git a/core/src/main/scala/spark/partial/PartialResult.scala b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala similarity index 99% rename from core/src/main/scala/spark/partial/PartialResult.scala rename to core/src/main/scala/org/apache/spark/partial/PartialResult.scala index 922a9f9bc65..5ce49b8100e 100644 --- a/core/src/main/scala/spark/partial/PartialResult.scala +++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.partial +package org.apache.spark.partial class PartialResult[R](initialVal: R, isFinal: Boolean) { private var finalValue: Option[R] = if (isFinal) Some(initialVal) else None diff --git a/core/src/main/scala/spark/partial/StudentTCacher.scala b/core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala similarity index 98% rename from core/src/main/scala/spark/partial/StudentTCacher.scala rename to core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala index f3bb987d46c..92915ee66d2 100644 --- a/core/src/main/scala/spark/partial/StudentTCacher.scala +++ b/core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.partial +package org.apache.spark.partial import cern.jet.stat.Probability diff --git a/core/src/main/scala/spark/partial/SumEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala similarity index 97% rename from core/src/main/scala/spark/partial/SumEvaluator.scala rename to core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala index 4083abef03d..a74f8009443 100644 --- a/core/src/main/scala/spark/partial/SumEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.partial +package org.apache.spark.partial import cern.jet.stat.Probability -import spark.util.StatCounter +import org.apache.spark.util.StatCounter /** * An ApproximateEvaluator for sums. It estimates the mean and the cont and multiplies them diff --git a/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala similarity index 92% rename from core/src/main/scala/spark/rdd/BlockRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala index 03800584ae0..4bb01efa865 100644 --- a/core/src/main/scala/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd -import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} -import spark.storage.BlockManager +import org.apache.spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} +import org.apache.spark.storage.BlockManager private[spark] class BlockRDDPartition(val blockId: String, idx: Int) extends Partition { val index = idx diff --git a/core/src/main/scala/spark/rdd/CartesianRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala similarity index 98% rename from core/src/main/scala/spark/rdd/CartesianRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala index 91b3e69d6fb..9b0c882481c 100644 --- a/core/src/main/scala/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd import java.io.{ObjectOutputStream, IOException} -import spark._ +import org.apache.spark._ private[spark] diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala similarity index 98% rename from core/src/main/scala/spark/rdd/CheckpointRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index 1ad5fe65390..3311757189a 100644 --- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd -import spark._ +import org.apache.spark._ import org.apache.hadoop.mapred.{FileInputFormat, SequenceFileInputFormat, JobConf, Reporter} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.{NullWritable, BytesWritable} @@ -138,7 +138,7 @@ private[spark] object CheckpointRDD extends Logging { // each split file having multiple blocks. This needs to be run on a // cluster (mesos or standalone) using HDFS. def main(args: Array[String]) { - import spark._ + import org.apache.spark._ val Array(cluster, hdfsPath) = args val env = SparkEnv.get diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala similarity index 96% rename from core/src/main/scala/spark/rdd/CoGroupedRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 01b6c23dcc2..dcc35e8d0e9 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd import java.io.{ObjectOutputStream, IOException} import java.util.{HashMap => JHashMap} @@ -23,8 +23,8 @@ import java.util.{HashMap => JHashMap} import scala.collection.JavaConversions import scala.collection.mutable.ArrayBuffer -import spark.{Partition, Partitioner, RDD, SparkEnv, TaskContext} -import spark.{Dependency, OneToOneDependency, ShuffleDependency} +import org.apache.spark.{Partition, Partitioner, RDD, SparkEnv, TaskContext} +import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} private[spark] sealed trait CoGroupSplitDep extends Serializable diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala similarity index 99% rename from core/src/main/scala/spark/rdd/CoalescedRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index e612d026b22..c5de6362a9a 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd -import spark._ +import org.apache.spark._ import java.io.{ObjectOutputStream, IOException} import scala.collection.mutable import scala.Some diff --git a/core/src/main/scala/spark/rdd/EmptyRDD.scala b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala similarity index 91% rename from core/src/main/scala/spark/rdd/EmptyRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala index d7d4db5d309..24ce4abbc47 100644 --- a/core/src/main/scala/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd -import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} +import org.apache.spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} /** diff --git a/core/src/main/scala/spark/rdd/FilteredRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala similarity index 92% rename from core/src/main/scala/spark/rdd/FilteredRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala index 783508cfd17..4df8ceb58be 100644 --- a/core/src/main/scala/spark/rdd/FilteredRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd -import spark.{OneToOneDependency, RDD, Partition, TaskContext} +import org.apache.spark.{OneToOneDependency, RDD, Partition, TaskContext} private[spark] class FilteredRDD[T: ClassManifest]( prev: RDD[T], diff --git a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala similarity index 93% rename from core/src/main/scala/spark/rdd/FlatMappedRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala index ed75eac3ff6..2bf7653af1b 100644 --- a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd -import spark.{RDD, Partition, TaskContext} +import org.apache.spark.{RDD, Partition, TaskContext} private[spark] diff --git a/core/src/main/scala/spark/rdd/FlatMappedValuesRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala similarity index 93% rename from core/src/main/scala/spark/rdd/FlatMappedValuesRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala index a6bdce89d8a..e544720b054 100644 --- a/core/src/main/scala/spark/rdd/FlatMappedValuesRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd -import spark.{TaskContext, Partition, RDD} +import org.apache.spark.{TaskContext, Partition, RDD} private[spark] diff --git a/core/src/main/scala/spark/rdd/GlommedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala similarity index 93% rename from core/src/main/scala/spark/rdd/GlommedRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala index 1573f8a289a..2ce94199f2e 100644 --- a/core/src/main/scala/spark/rdd/GlommedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd -import spark.{RDD, Partition, TaskContext} +import org.apache.spark.{RDD, Partition, TaskContext} private[spark] class GlommedRDD[T: ClassManifest](prev: RDD[T]) extends RDD[Array[T]](prev) { diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala similarity index 95% rename from core/src/main/scala/spark/rdd/HadoopRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index e512423fd63..08e6154bb9f 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd import java.io.EOFException import java.util.NoSuchElementException @@ -32,8 +32,8 @@ import org.apache.hadoop.mapred.RecordReader import org.apache.hadoop.mapred.Reporter import org.apache.hadoop.util.ReflectionUtils -import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, SparkEnv, TaskContext} -import spark.util.NextIterator +import org.apache.spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, SparkEnv, TaskContext} +import org.apache.spark.util.NextIterator import org.apache.hadoop.conf.{Configuration, Configurable} diff --git a/core/src/main/scala/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala similarity index 96% rename from core/src/main/scala/spark/rdd/JdbcRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index 59132437d20..3db460b3ced 100644 --- a/core/src/main/scala/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd import java.sql.{Connection, ResultSet} -import spark.{Logging, Partition, RDD, SparkContext, TaskContext} -import spark.util.NextIterator +import org.apache.spark.{Logging, Partition, RDD, SparkContext, TaskContext} +import org.apache.spark.util.NextIterator private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { override def index = idx diff --git a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala similarity index 93% rename from core/src/main/scala/spark/rdd/MapPartitionsRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala index af8f0a112f2..13009d3e172 100644 --- a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd -import spark.{RDD, Partition, TaskContext} +import org.apache.spark.{RDD, Partition, TaskContext} private[spark] diff --git a/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala similarity index 94% rename from core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala index 3b4e9518fd0..1683050b865 100644 --- a/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd -import spark.{RDD, Partition, TaskContext} +import org.apache.spark.{RDD, Partition, TaskContext} /** diff --git a/core/src/main/scala/spark/rdd/MappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala similarity index 93% rename from core/src/main/scala/spark/rdd/MappedRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala index 8b411dd85d9..26d4806edb0 100644 --- a/core/src/main/scala/spark/rdd/MappedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd -import spark.{RDD, Partition, TaskContext} +import org.apache.spark.{RDD, Partition, TaskContext} private[spark] class MappedRDD[U: ClassManifest, T: ClassManifest](prev: RDD[T], f: T => U) diff --git a/core/src/main/scala/spark/rdd/MappedValuesRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala similarity index 93% rename from core/src/main/scala/spark/rdd/MappedValuesRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala index 8334e3b557d..a405e9acddb 100644 --- a/core/src/main/scala/spark/rdd/MappedValuesRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd -import spark.{TaskContext, Partition, RDD} +import org.apache.spark.{TaskContext, Partition, RDD} private[spark] class MappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => U) diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala similarity index 96% rename from core/src/main/scala/spark/rdd/NewHadoopRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index b1877dc06e5..114b5044861 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd import java.text.SimpleDateFormat import java.util.Date @@ -24,7 +24,7 @@ import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ -import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext} +import org.apache.spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext} private[spark] diff --git a/core/src/main/scala/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala similarity index 96% rename from core/src/main/scala/spark/rdd/OrderedRDDFunctions.scala rename to core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index 9154b760359..4c3df0eaf46 100644 --- a/core/src/main/scala/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd -import spark.{RangePartitioner, Logging, RDD} +import org.apache.spark.{RangePartitioner, Logging, RDD} /** * Extra functions available on RDDs of (key, value) pairs where the key is sortable through diff --git a/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala similarity index 98% rename from core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala index 33079cd5393..8db3611054d 100644 --- a/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd import scala.collection.immutable.NumericRange import scala.collection.mutable.ArrayBuffer import scala.collection.Map -import spark._ +import org.apache.spark._ import java.io._ import scala.Serializable diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala similarity index 96% rename from core/src/main/scala/spark/rdd/PartitionPruningRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala index d8700becb0e..8e79a5c874e 100644 --- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd -import spark.{NarrowDependency, RDD, SparkEnv, Partition, TaskContext} +import org.apache.spark.{NarrowDependency, RDD, SparkEnv, Partition, TaskContext} class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends Partition { diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala similarity index 96% rename from core/src/main/scala/spark/rdd/PipedRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 2cefdc78b01..98498d5ddfe 100644 --- a/core/src/main/scala/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd import java.io.PrintWriter import java.util.StringTokenizer @@ -25,8 +25,8 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer import scala.io.Source -import spark.{RDD, SparkEnv, Partition, TaskContext} -import spark.broadcast.Broadcast +import org.apache.spark.{RDD, SparkEnv, Partition, TaskContext} +import org.apache.spark.broadcast.Broadcast /** diff --git a/core/src/main/scala/spark/rdd/SampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala similarity index 96% rename from core/src/main/scala/spark/rdd/SampledRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala index 574c9b141de..1e8d89e9120 100644 --- a/core/src/main/scala/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd import java.util.Random import cern.jet.random.Poisson import cern.jet.random.engine.DRand -import spark.{RDD, Partition, TaskContext} +import org.apache.spark.{RDD, Partition, TaskContext} private[spark] class SampledRDDPartition(val prev: Partition, val seed: Int) extends Partition with Serializable { diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala similarity index 94% rename from core/src/main/scala/spark/rdd/ShuffledRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index 51c05af064e..f0e9ab8b801 100644 --- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd -import spark.{Dependency, Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext} +import org.apache.spark.{Dependency, Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext} private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala similarity index 93% rename from core/src/main/scala/spark/rdd/SubtractedRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala index dadef5e17de..7369dfaa74b 100644 --- a/core/src/main/scala/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala @@ -15,19 +15,19 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd import java.util.{HashMap => JHashMap} import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer -import spark.RDD -import spark.Partitioner -import spark.Dependency -import spark.TaskContext -import spark.Partition -import spark.SparkEnv -import spark.ShuffleDependency -import spark.OneToOneDependency +import org.apache.spark.RDD +import org.apache.spark.Partitioner +import org.apache.spark.Dependency +import org.apache.spark.TaskContext +import org.apache.spark.Partition +import org.apache.spark.SparkEnv +import org.apache.spark.ShuffleDependency +import org.apache.spark.OneToOneDependency /** diff --git a/core/src/main/scala/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala similarity index 95% rename from core/src/main/scala/spark/rdd/UnionRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index 2776826f187..fd02476b627 100644 --- a/core/src/main/scala/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd import scala.collection.mutable.ArrayBuffer -import spark.{Dependency, RangeDependency, RDD, SparkContext, Partition, TaskContext} +import org.apache.spark.{Dependency, RangeDependency, RDD, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} private[spark] class UnionPartition[T: ClassManifest](idx: Int, rdd: RDD[T], splitIndex: Int) diff --git a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala similarity index 97% rename from core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index 9a0831bd899..5ae1db3e676 100644 --- a/core/src/main/scala/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd -import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} +import org.apache.spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} private[spark] class ZippedPartitionsPartition( diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala similarity index 95% rename from core/src/main/scala/spark/rdd/ZippedRDD.scala rename to core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala index 4074e50e440..3bd00d291b1 100644 --- a/core/src/main/scala/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd -import spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} +import org.apache.spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} diff --git a/core/src/main/scala/spark/scheduler/ActiveJob.scala b/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala similarity index 94% rename from core/src/main/scala/spark/scheduler/ActiveJob.scala rename to core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala index fecc3e9648a..0b04607d019 100644 --- a/core/src/main/scala/spark/scheduler/ActiveJob.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler -import spark.TaskContext +import org.apache.spark.TaskContext import java.util.Properties diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala similarity index 98% rename from core/src/main/scala/spark/scheduler/DAGScheduler.scala rename to core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 7275bd346ad..5ac700bbf44 100644 --- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler import java.io.NotSerializableException import java.util.Properties @@ -24,12 +24,12 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} -import spark._ -import spark.executor.TaskMetrics -import spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} -import spark.scheduler.cluster.TaskInfo -import spark.storage.{BlockManager, BlockManagerMaster} -import spark.util.{MetadataCleaner, TimeStampedHashMap} +import org.apache.spark._ +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} +import org.apache.spark.scheduler.cluster.TaskInfo +import org.apache.spark.storage.{BlockManager, BlockManagerMaster} +import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap} /** * The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala similarity index 93% rename from core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala rename to core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index b8ba0e92394..5b07933eedc 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -15,15 +15,15 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler import java.util.Properties -import spark.scheduler.cluster.TaskInfo +import org.apache.spark.scheduler.cluster.TaskInfo import scala.collection.mutable.Map -import spark._ -import spark.executor.TaskMetrics +import org.apache.spark._ +import org.apache.spark.executor.TaskMetrics /** * Types of events that can be handled by the DAGScheduler. The DAGScheduler uses an event queue diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala similarity index 92% rename from core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala rename to core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala index 98c4fb7e59a..ce0dc9093d8 100644 --- a/core/src/main/scala/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala @@ -1,8 +1,8 @@ -package spark.scheduler +package org.apache.spark.scheduler import com.codahale.metrics.{Gauge,MetricRegistry} -import spark.metrics.source.Source +import org.apache.spark.metrics.source.Source private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends Source { val metricRegistry = new MetricRegistry() diff --git a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala similarity index 98% rename from core/src/main/scala/spark/scheduler/InputFormatInfo.scala rename to core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index 8f1b9b29b54..370ccd183c3 100644 --- a/core/src/main/scala/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler -import spark.{Logging, SparkEnv} +import org.apache.spark.{Logging, SparkEnv} import scala.collection.immutable.Set import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.apache.hadoop.security.UserGroupInformation diff --git a/core/src/main/scala/spark/scheduler/JobListener.scala b/core/src/main/scala/org/apache/spark/scheduler/JobListener.scala similarity index 97% rename from core/src/main/scala/spark/scheduler/JobListener.scala rename to core/src/main/scala/org/apache/spark/scheduler/JobListener.scala index af108b8fec3..50c2b9acd60 100644 --- a/core/src/main/scala/spark/scheduler/JobListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobListener.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler /** * Interface used to listen for job completion or failure events after submitting a job to the diff --git a/core/src/main/scala/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala similarity index 96% rename from core/src/main/scala/spark/scheduler/JobLogger.scala rename to core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 1bc9fabdffd..98ef4d1e634 100644 --- a/core/src/main/scala/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler import java.io.PrintWriter import java.io.File @@ -27,9 +27,9 @@ 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 +import org.apache.spark._ +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler.cluster.TaskInfo // Used to record runtime information for each job, including RDD graph // tasks' start/stop shuffle information and information from outside diff --git a/core/src/main/scala/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala similarity index 96% rename from core/src/main/scala/spark/scheduler/JobResult.scala rename to core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index a61b335152e..c381348a8d4 100644 --- a/core/src/main/scala/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler /** * A result of a job in the DAGScheduler. diff --git a/core/src/main/scala/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala similarity index 98% rename from core/src/main/scala/spark/scheduler/JobWaiter.scala rename to core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala index 69cd161c1f0..200d8817990 100644 --- a/core/src/main/scala/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala similarity index 95% rename from core/src/main/scala/spark/scheduler/MapStatus.scala rename to core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 2f6a68ee85b..1c61687f280 100644 --- a/core/src/main/scala/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler -import spark.storage.BlockManagerId +import org.apache.spark.storage.BlockManagerId import java.io.{ObjectOutput, ObjectInput, Externalizable} /** diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala similarity index 98% rename from core/src/main/scala/spark/scheduler/ResultTask.scala rename to core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index d066df5dc1d..2f157ccdd25 100644 --- a/core/src/main/scala/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler -import spark._ +import org.apache.spark._ import java.io._ import util.{MetadataCleaner, TimeStampedHashMap} import java.util.zip.{GZIPInputStream, GZIPOutputStream} diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala similarity index 96% rename from core/src/main/scala/spark/scheduler/ShuffleMapTask.scala rename to core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index f2a038576b7..ca716b44e88 100644 --- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -15,17 +15,17 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.mutable.HashMap -import spark._ -import spark.executor.ShuffleWriteMetrics -import spark.storage._ -import spark.util.{TimeStampedHashMap, MetadataCleaner} +import org.apache.spark._ +import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.storage._ +import org.apache.spark.util.{TimeStampedHashMap, MetadataCleaner} private[spark] object ShuffleMapTask { diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala similarity index 96% rename from core/src/main/scala/spark/scheduler/SparkListener.scala rename to core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index e5531011c2b..3504424fa94 100644 --- a/core/src/main/scala/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler import java.util.Properties -import spark.scheduler.cluster.TaskInfo -import spark.util.Distribution -import spark.{Logging, SparkContext, TaskEndReason, Utils} -import spark.executor.TaskMetrics +import org.apache.spark.scheduler.cluster.TaskInfo +import org.apache.spark.util.Distribution +import org.apache.spark.{Logging, SparkContext, TaskEndReason, Utils} +import org.apache.spark.executor.TaskMetrics sealed trait SparkListenerEvents @@ -79,7 +79,7 @@ trait SparkListener { */ class StatsReportListener extends SparkListener with Logging { override def onStageCompleted(stageCompleted: StageCompleted) { - import spark.scheduler.StatsReportListener._ + import org.apache.spark.scheduler.StatsReportListener._ implicit val sc = stageCompleted this.logInfo("Finished stage: " + stageCompleted.stageInfo) showMillisDistribution("task runtime:", (info, _) => Some(info.duration)) diff --git a/core/src/main/scala/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala similarity index 95% rename from core/src/main/scala/spark/scheduler/SparkListenerBus.scala rename to core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index f55ed455ed3..a65e1ecd6d4 100644 --- a/core/src/main/scala/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} -import spark.Logging +import org.apache.spark.Logging /** Asynchronously passes SparkListenerEvents to registered SparkListeners. */ private[spark] class SparkListenerBus() extends Logging { diff --git a/core/src/main/scala/spark/scheduler/SplitInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala similarity index 98% rename from core/src/main/scala/spark/scheduler/SplitInfo.scala rename to core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala index 4e3661ec5da..5b40a3eb29b 100644 --- a/core/src/main/scala/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler import collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala similarity index 97% rename from core/src/main/scala/spark/scheduler/Stage.scala rename to core/src/main/scala/org/apache/spark/scheduler/Stage.scala index c599c00ac44..87b1fe4e0c9 100644 --- a/core/src/main/scala/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler import java.net.URI -import spark._ -import spark.storage.BlockManagerId +import org.apache.spark._ +import org.apache.spark.storage.BlockManagerId /** * A stage is a set of independent tasks all computing the same function that need to run as part diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala similarity index 88% rename from core/src/main/scala/spark/scheduler/StageInfo.scala rename to core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index c4026f995a4..72cb1c9ce8a 100644 --- a/core/src/main/scala/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler -import spark.scheduler.cluster.TaskInfo +import org.apache.spark.scheduler.cluster.TaskInfo import scala.collection._ -import spark.executor.TaskMetrics +import org.apache.spark.executor.TaskMetrics case class StageInfo( val stage: Stage, diff --git a/core/src/main/scala/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala similarity index 95% rename from core/src/main/scala/spark/scheduler/Task.scala rename to core/src/main/scala/org/apache/spark/scheduler/Task.scala index 0ab2ae6cfe9..598d91752a3 100644 --- a/core/src/main/scala/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -15,15 +15,15 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler -import spark.serializer.SerializerInstance +import org.apache.spark.serializer.SerializerInstance import java.io.{DataInputStream, DataOutputStream} import java.nio.ByteBuffer import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream -import spark.util.ByteBufferInputStream +import org.apache.spark.util.ByteBufferInputStream import scala.collection.mutable.HashMap -import spark.executor.TaskMetrics +import org.apache.spark.executor.TaskMetrics /** * A task to execute on a worker node. diff --git a/core/src/main/scala/spark/scheduler/TaskLocation.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala similarity index 97% rename from core/src/main/scala/spark/scheduler/TaskLocation.scala rename to core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala index fea117e956d..67c9a6760b1 100644 --- a/core/src/main/scala/spark/scheduler/TaskLocation.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler /** * A location where a task should run. This can either be a host or a (host, executorID) pair. diff --git a/core/src/main/scala/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala similarity index 94% rename from core/src/main/scala/spark/scheduler/TaskResult.scala rename to core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index fc4856756b1..776675d28c9 100644 --- a/core/src/main/scala/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler import java.io._ import scala.collection.mutable.Map -import spark.executor.TaskMetrics -import spark.{Utils, SparkEnv} +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.{Utils, SparkEnv} import java.nio.ByteBuffer // Task result. Also contains updates to accumulator variables. diff --git a/core/src/main/scala/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala similarity index 92% rename from core/src/main/scala/spark/scheduler/TaskScheduler.scala rename to core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 4943d58e254..63be8ba3f58 100644 --- a/core/src/main/scala/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler -import spark.scheduler.cluster.Pool -import spark.scheduler.cluster.SchedulingMode.SchedulingMode +import org.apache.spark.scheduler.cluster.Pool +import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * Low-level task scheduler interface, implemented by both ClusterScheduler and LocalScheduler. * These schedulers get sets of tasks submitted to them from the DAGScheduler for each stage, diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerListener.scala similarity index 90% rename from core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala rename to core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerListener.scala index 64be50b2d02..83be051c1a8 100644 --- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerListener.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler -import spark.scheduler.cluster.TaskInfo +import org.apache.spark.scheduler.cluster.TaskInfo import scala.collection.mutable.Map -import spark.TaskEndReason -import spark.executor.TaskMetrics +import org.apache.spark.TaskEndReason +import org.apache.spark.executor.TaskMetrics /** * Interface for getting events back from the TaskScheduler. diff --git a/core/src/main/scala/spark/scheduler/TaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala similarity index 97% rename from core/src/main/scala/spark/scheduler/TaskSet.scala rename to core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala index dc3550dd0b1..c3ad325156f 100644 --- a/core/src/main/scala/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler import java.util.Properties diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala similarity index 98% rename from core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala index 679d899b472..3196ab5022a 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster import java.lang.{Boolean => JBoolean} @@ -23,10 +23,10 @@ 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.SchedulingMode.SchedulingMode +import org.apache.spark._ +import org.apache.spark.TaskState.TaskState +import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicLong import java.util.{TimerTask, Timer} diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala similarity index 98% rename from core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala index a4d6880abb5..a33307b83ae 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster import java.nio.ByteBuffer import java.util.{Arrays, NoSuchElementException} @@ -26,15 +26,15 @@ import scala.collection.mutable.HashSet import scala.math.max import scala.math.min -import spark.{FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskState, Utils} -import spark.{ExceptionFailure, SparkException, TaskResultTooBigFailure} -import spark.TaskState.TaskState -import spark.scheduler._ +import org.apache.spark.{FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskState, Utils} +import org.apache.spark.{ExceptionFailure, SparkException, TaskResultTooBigFailure} +import org.apache.spark.TaskState.TaskState +import org.apache.spark.scheduler._ import scala.Some -import spark.FetchFailed -import spark.ExceptionFailure -import spark.TaskResultTooBigFailure -import spark.util.{SystemClock, Clock} +import org.apache.spark.FetchFailed +import org.apache.spark.ExceptionFailure +import org.apache.spark.TaskResultTooBigFailure +import org.apache.spark.util.{SystemClock, Clock} /** diff --git a/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorLossReason.scala similarity index 93% rename from core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorLossReason.scala index 8825f2dd242..5077b2b48b5 100644 --- a/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorLossReason.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster -import spark.executor.ExecutorExitCode +import org.apache.spark.executor.ExecutorExitCode /** * Represents an explanation for a executor or whole slave failing or exiting. diff --git a/core/src/main/scala/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/Pool.scala similarity index 96% rename from core/src/main/scala/spark/scheduler/cluster/Pool.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/Pool.scala index 83708f07e1a..35b32600da6 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/Pool.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap -import spark.Logging -import spark.scheduler.cluster.SchedulingMode.SchedulingMode +import org.apache.spark.Logging +import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * An Schedulable entity that represent collection of Pools or TaskSetManagers diff --git a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/Schedulable.scala similarity index 93% rename from core/src/main/scala/spark/scheduler/cluster/Schedulable.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/Schedulable.scala index e77e8e4162e..f4726450ec2 100644 --- a/core/src/main/scala/spark/scheduler/cluster/Schedulable.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/Schedulable.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster -import spark.scheduler.cluster.SchedulingMode.SchedulingMode +import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode import scala.collection.mutable.ArrayBuffer /** diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala similarity index 97% rename from core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala index 2fc8a76a057..d04eeb6b983 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala @@ -15,15 +15,15 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster import java.io.{File, FileInputStream, FileOutputStream, FileNotFoundException} import java.util.Properties import scala.xml.XML -import spark.Logging -import spark.scheduler.cluster.SchedulingMode.SchedulingMode +import org.apache.spark.Logging +import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode /** diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala similarity index 93% rename from core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala index 4431744ec33..bde2f73df41 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster -import spark.{SparkContext, Utils} +import org.apache.spark.{SparkContext, Utils} /** * A backend interface for cluster scheduling systems that allows plugging in different ones under diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingAlgorithm.scala similarity index 98% rename from core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingAlgorithm.scala index 69e0ac2a6bd..cbeed4731a7 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingAlgorithm.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingAlgorithm.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster /** * An interface for sort algorithm diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala similarity index 96% rename from core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala index 55cdf4791f9..34811389a02 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster /** * "FAIR" and "FIFO" determines which policy is used diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala similarity index 89% rename from core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 7ac574bdc8e..ac6dc7d8793 100644 --- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster -import spark.{Utils, Logging, SparkContext} -import spark.deploy.client.{Client, ClientListener} -import spark.deploy.{Command, ApplicationDescription} +import org.apache.spark.{Utils, Logging, SparkContext} +import org.apache.spark.deploy.client.{Client, ClientListener} +import org.apache.spark.deploy.{Command, ApplicationDescription} import scala.collection.mutable.HashMap private[spark] class SparkDeploySchedulerBackend( @@ -45,7 +45,8 @@ private[spark] class SparkDeploySchedulerBackend( System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), StandaloneSchedulerBackend.ACTOR_NAME) val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}") - val command = Command("spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs) + val command = Command( + "org.apache.spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome().getOrElse(null) val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, sc.ui.appUIAddress) diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala similarity index 92% rename from core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala index 05c29eb72f6..1cc5daf6739 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster import java.nio.ByteBuffer -import spark.TaskState.TaskState -import spark.Utils -import spark.util.SerializableBuffer +import org.apache.spark.TaskState.TaskState +import org.apache.spark.Utils +import org.apache.spark.util.SerializableBuffer private[spark] sealed trait StandaloneClusterMessage extends Serializable diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala similarity index 97% rename from core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 3203be10294..3677a827e0c 100644 --- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster import java.util.concurrent.atomic.AtomicInteger @@ -28,8 +28,8 @@ import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClient import akka.util.Duration import akka.util.duration._ -import spark.{Utils, SparkException, Logging, TaskState} -import spark.scheduler.cluster.StandaloneClusterMessages._ +import org.apache.spark.{Utils, SparkException, Logging, TaskState} +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages._ /** * A standalone scheduler backend, which waits for standalone executors to connect to it through diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskDescription.scala similarity index 93% rename from core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/TaskDescription.scala index 187553233fb..309ac2f6c9d 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskDescription.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster import java.nio.ByteBuffer -import spark.util.SerializableBuffer +import org.apache.spark.util.SerializableBuffer private[spark] class TaskDescription( val taskId: Long, diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala similarity index 96% rename from core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala index c2c5522686f..7ce14be7fb9 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster -import spark.Utils +import org.apache.spark.Utils /** * Information about a running task attempt inside a TaskSet. diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala similarity index 96% rename from core/src/main/scala/spark/scheduler/cluster/TaskLocality.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala index 1c33e41f87e..5d4130e14a8 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster private[spark] object TaskLocality diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskSetManager.scala similarity index 93% rename from core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/TaskSetManager.scala index 0248830b7a2..648a3ef922c 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskSetManager.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster import java.nio.ByteBuffer -import spark.TaskState.TaskState -import spark.scheduler.TaskSet +import org.apache.spark.TaskState.TaskState +import org.apache.spark.scheduler.TaskSet /** * Tracks and schedules the tasks within a single TaskSet. This class keeps track of the status of diff --git a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/WorkerOffer.scala similarity index 95% rename from core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala rename to core/src/main/scala/org/apache/spark/scheduler/cluster/WorkerOffer.scala index 1d09bd9b034..938f62883a1 100644 --- a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/WorkerOffer.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster /** * Represents free resources available on an executor. diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala similarity index 97% rename from core/src/main/scala/spark/scheduler/local/LocalScheduler.scala rename to core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala index 5be4dbd9f0b..f0ebe66d823 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler.local +package org.apache.spark.scheduler.local import java.io.File import java.lang.management.ManagementFactory @@ -27,12 +27,12 @@ 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._ -import spark.scheduler.cluster.SchedulingMode.SchedulingMode +import org.apache.spark._ +import org.apache.spark.TaskState.TaskState +import org.apache.spark.executor.ExecutorURLClassLoader +import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster._ +import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode import akka.actor._ /** diff --git a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala similarity index 94% rename from core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala rename to core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala index e237f289e36..e52cb998bdf 100644 --- a/core/src/main/scala/spark/scheduler/local/LocalTaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala @@ -15,16 +15,16 @@ * limitations under the License. */ -package spark.scheduler.local +package org.apache.spark.scheduler.local import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap -import spark.{ExceptionFailure, Logging, SparkEnv, Success, TaskState} -import spark.TaskState.TaskState -import spark.scheduler.{Task, TaskResult, TaskSet} -import spark.scheduler.cluster.{Schedulable, TaskDescription, TaskInfo, TaskLocality, TaskSetManager} +import org.apache.spark.{ExceptionFailure, Logging, SparkEnv, Success, TaskState} +import org.apache.spark.TaskState.TaskState +import org.apache.spark.scheduler.{Task, TaskResult, TaskSet} +import org.apache.spark.scheduler.cluster.{Schedulable, TaskDescription, TaskInfo, TaskLocality, TaskSetManager} private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet) diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala similarity index 94% rename from core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala rename to core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index eef3ee14251..f6a2feab289 100644 --- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler.mesos +package org.apache.spark.scheduler.mesos import com.google.protobuf.ByteString @@ -23,14 +23,14 @@ import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import spark.{SparkException, Utils, Logging, SparkContext} +import org.apache.spark.{SparkException, Utils, Logging, SparkContext} import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.collection.JavaConversions._ import java.io.File -import spark.scheduler.cluster._ +import org.apache.spark.scheduler.cluster._ import java.util.{ArrayList => JArrayList, List => JList} import java.util.Collections -import spark.TaskState +import org.apache.spark.TaskState /** * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds @@ -126,14 +126,16 @@ private[spark] class CoarseMesosSchedulerBackend( val uri = System.getProperty("spark.executor.uri") if (uri == null) { val runScript = new File(sparkHome, "spark-class").getCanonicalPath - command.setValue("\"%s\" spark.executor.StandaloneExecutorBackend %s %s %s %d".format( - runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) + command.setValue( + "\"%s\" org.apache.spark.executor.StandaloneExecutorBackend %s %s %s %d".format( + runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". val basename = uri.split('/').last.split('.').head - command.setValue("cd %s*; ./spark-class spark.executor.StandaloneExecutorBackend %s %s %s %d".format( - basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) + command.setValue( + "cd %s*; ./spark-class org.apache.spark.executor.StandaloneExecutorBackend %s %s %s %d".format( + basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } return command.build() diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala similarity index 98% rename from core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala rename to core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala index f6069a5775e..e002af17422 100644 --- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.scheduler.mesos +package org.apache.spark.scheduler.mesos import com.google.protobuf.ByteString @@ -23,14 +23,14 @@ import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import spark.{SparkException, Utils, Logging, SparkContext} +import org.apache.spark.{SparkException, Utils, Logging, SparkContext} import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.collection.JavaConversions._ import java.io.File -import spark.scheduler.cluster._ +import org.apache.spark.scheduler.cluster._ import java.util.{ArrayList => JArrayList, List => JList} import java.util.Collections -import spark.TaskState +import org.apache.spark.TaskState /** * A SchedulerBackend for running fine-grained tasks on Mesos. Each Spark task is mapped to a diff --git a/core/src/main/scala/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala similarity index 91% rename from core/src/main/scala/spark/serializer/Serializer.scala rename to core/src/main/scala/org/apache/spark/serializer/Serializer.scala index dc94d42bb66..160cca4d6c5 100644 --- a/core/src/main/scala/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -15,19 +15,19 @@ * limitations under the License. */ -package spark.serializer +package org.apache.spark.serializer import java.io.{EOFException, InputStream, OutputStream} import java.nio.ByteBuffer import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream -import spark.util.ByteBufferInputStream +import org.apache.spark.util.{NextIterator, 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 + * create [[org.apache.spark.serializer.SerializerInstance]] objects that do the actual serialization and are * guaranteed to only be called from one thread at a time. */ trait Serializer { @@ -95,7 +95,7 @@ trait DeserializationStream { * Read the elements of this stream through an iterator. This can only be called once, as * reading each element will consume data from the input source. */ - def asIterator: Iterator[Any] = new spark.util.NextIterator[Any] { + def asIterator: Iterator[Any] = new NextIterator[Any] { override protected def getNext() = { try { readObject[Any]() diff --git a/core/src/main/scala/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala similarity index 98% rename from core/src/main/scala/spark/serializer/SerializerManager.scala rename to core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala index b7b24705a2a..2955986fece 100644 --- a/core/src/main/scala/spark/serializer/SerializerManager.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.serializer +package org.apache.spark.serializer import java.util.concurrent.ConcurrentHashMap diff --git a/core/src/main/scala/spark/storage/BlockException.scala b/core/src/main/scala/org/apache/spark/storage/BlockException.scala similarity index 96% rename from core/src/main/scala/spark/storage/BlockException.scala rename to core/src/main/scala/org/apache/spark/storage/BlockException.scala index 8ebfaf3cbf6..290dbce4f54 100644 --- a/core/src/main/scala/spark/storage/BlockException.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockException.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage private[spark] case class BlockException(blockId: String, message: String) extends Exception(message) diff --git a/core/src/main/scala/spark/storage/BlockFetchTracker.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetchTracker.scala similarity index 96% rename from core/src/main/scala/spark/storage/BlockFetchTracker.scala rename to core/src/main/scala/org/apache/spark/storage/BlockFetchTracker.scala index 265e554ad88..2e0b0e6eda7 100644 --- a/core/src/main/scala/spark/storage/BlockFetchTracker.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetchTracker.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage private[spark] trait BlockFetchTracker { def totalBlocks : Int diff --git a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala similarity index 97% rename from core/src/main/scala/spark/storage/BlockFetcherIterator.scala rename to core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 568783d8931..c91f0fc1ad8 100644 --- a/core/src/main/scala/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage import java.nio.ByteBuffer import java.util.concurrent.LinkedBlockingQueue @@ -26,13 +26,13 @@ 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 org.apache.spark.Logging +import org.apache.spark.Utils +import org.apache.spark.SparkException +import org.apache.spark.network.BufferMessage +import org.apache.spark.network.ConnectionManagerId +import org.apache.spark.network.netty.ShuffleCopier +import org.apache.spark.serializer.Serializer /** diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala similarity index 99% rename from core/src/main/scala/spark/storage/BlockManager.scala rename to core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 2a6ec2a55dd..3299ac98d55 100644 --- a/core/src/main/scala/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage import java.io.{InputStream, OutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} @@ -29,11 +29,11 @@ import akka.util.duration._ import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream -import spark.{Logging, SparkEnv, SparkException, Utils} -import spark.io.CompressionCodec -import spark.network._ -import spark.serializer.Serializer -import spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStampedHashMap} +import org.apache.spark.{Logging, SparkEnv, SparkException, Utils} +import org.apache.spark.io.CompressionCodec +import org.apache.spark.network._ +import org.apache.spark.serializer.Serializer +import org.apache.spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStampedHashMap} import sun.nio.ch.DirectBuffer diff --git a/core/src/main/scala/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala similarity index 94% rename from core/src/main/scala/spark/storage/BlockManagerId.scala rename to core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index b36a6176c08..a22a80decc9 100644 --- a/core/src/main/scala/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import java.util.concurrent.ConcurrentHashMap -import spark.Utils +import org.apache.spark.Utils /** * This class represent an unique identifier for a BlockManager. @@ -92,13 +92,13 @@ private[spark] class BlockManagerId private ( private[spark] object BlockManagerId { /** - * Returns a [[spark.storage.BlockManagerId]] for the given configuraiton. + * Returns a [[org.apache.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]]. + * @return A new [[org.apache.spark.storage.BlockManagerId]]. */ def apply(execId: String, host: String, port: Int, nettyPort: Int) = getCachedBlockManagerId(new BlockManagerId(execId, host, port, nettyPort)) diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala similarity index 97% rename from core/src/main/scala/spark/storage/BlockManagerMaster.scala rename to core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 76128e8cfff..cf463d6ffc5 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -15,15 +15,15 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage import akka.actor.ActorRef import akka.dispatch.{Await, Future} import akka.pattern.ask import akka.util.Duration -import spark.{Logging, SparkException} -import spark.storage.BlockManagerMessages._ +import org.apache.spark.{Logging, SparkException} +import org.apache.spark.storage.BlockManagerMessages._ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging { diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala similarity index 98% rename from core/src/main/scala/spark/storage/BlockManagerMasterActor.scala rename to core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index b7a981d1014..baa4a1da502 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage import java.util.{HashMap => JHashMap} @@ -28,8 +28,8 @@ import akka.pattern.ask import akka.util.Duration import akka.util.duration._ -import spark.{Logging, Utils, SparkException} -import spark.storage.BlockManagerMessages._ +import org.apache.spark.{Logging, Utils, SparkException} +import org.apache.spark.storage.BlockManagerMessages._ /** diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala similarity index 99% rename from core/src/main/scala/spark/storage/BlockManagerMessages.scala rename to core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 9375a9ca548..24333a179c9 100644 --- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage import java.io.{Externalizable, ObjectInput, ObjectOutput} diff --git a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala similarity index 93% rename from core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala rename to core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala index 6e5fb437329..951503019f7 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage import akka.actor.Actor -import spark.storage.BlockManagerMessages._ +import org.apache.spark.storage.BlockManagerMessages._ /** diff --git a/core/src/main/scala/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala similarity index 95% rename from core/src/main/scala/spark/storage/BlockManagerSource.scala rename to core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index 2aecd1ea716..24190cdd675 100644 --- a/core/src/main/scala/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -1,8 +1,8 @@ -package spark.storage +package org.apache.spark.storage import com.codahale.metrics.{Gauge,MetricRegistry} -import spark.metrics.source.Source +import org.apache.spark.metrics.source.Source private[spark] class BlockManagerSource(val blockManager: BlockManager) extends Source { diff --git a/core/src/main/scala/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala similarity index 97% rename from core/src/main/scala/spark/storage/BlockManagerWorker.scala rename to core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala index 39064bce92c..f4856020e55 100644 --- a/core/src/main/scala/spark/storage/BlockManagerWorker.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage import java.nio.ByteBuffer -import spark.{Logging, Utils} -import spark.network._ +import org.apache.spark.{Logging, Utils} +import org.apache.spark.network._ /** * A network interface for BlockManager. Each slave should have one diff --git a/core/src/main/scala/spark/storage/BlockMessage.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala similarity index 98% rename from core/src/main/scala/spark/storage/BlockMessage.scala rename to core/src/main/scala/org/apache/spark/storage/BlockMessage.scala index bcce26b7c14..d8fa6a91d15 100644 --- a/core/src/main/scala/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage import java.nio.ByteBuffer import scala.collection.mutable.StringBuilder import scala.collection.mutable.ArrayBuffer -import spark.network._ +import org.apache.spark.network._ private[spark] case class GetBlock(id: String) private[spark] case class GotBlock(id: String, data: ByteBuffer) diff --git a/core/src/main/scala/spark/storage/BlockMessageArray.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala similarity index 98% rename from core/src/main/scala/spark/storage/BlockMessageArray.scala rename to core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala index ee2fc167d5b..0aaf846b5b6 100644 --- a/core/src/main/scala/spark/storage/BlockMessageArray.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import spark._ -import spark.network._ +import org.apache.spark._ +import org.apache.spark.network._ private[spark] class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockMessage] with Logging { diff --git a/core/src/main/scala/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala similarity index 98% rename from core/src/main/scala/spark/storage/BlockObjectWriter.scala rename to core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index 3812009ca16..39f103297f9 100644 --- a/core/src/main/scala/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage /** diff --git a/core/src/main/scala/spark/storage/BlockStore.scala b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala similarity index 96% rename from core/src/main/scala/spark/storage/BlockStore.scala rename to core/src/main/scala/org/apache/spark/storage/BlockStore.scala index c8db0022b02..fa834371f48 100644 --- a/core/src/main/scala/spark/storage/BlockStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import spark.Logging +import org.apache.spark.Logging /** * Abstract class to store blocks diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala similarity index 97% rename from core/src/main/scala/spark/storage/DiskStore.scala rename to core/src/main/scala/org/apache/spark/storage/DiskStore.scala index b14497157e9..fd945e065cc 100644 --- a/core/src/main/scala/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage import java.io.{File, FileOutputStream, OutputStream, RandomAccessFile} import java.nio.ByteBuffer @@ -28,12 +28,12 @@ import scala.collection.mutable.ArrayBuffer import it.unimi.dsi.fastutil.io.FastBufferedOutputStream -import spark.Utils -import spark.executor.ExecutorExitCode -import spark.serializer.{Serializer, SerializationStream} -import spark.Logging -import spark.network.netty.ShuffleSender -import spark.network.netty.PathResolver +import org.apache.spark.Utils +import org.apache.spark.executor.ExecutorExitCode +import org.apache.spark.serializer.{Serializer, SerializationStream} +import org.apache.spark.Logging +import org.apache.spark.network.netty.ShuffleSender +import org.apache.spark.network.netty.PathResolver /** diff --git a/core/src/main/scala/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala similarity index 99% rename from core/src/main/scala/spark/storage/MemoryStore.scala rename to core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 5a51f5cf313..828dc0f22da 100644 --- a/core/src/main/scala/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage import java.util.LinkedHashMap import java.util.concurrent.ArrayBlockingQueue -import spark.{SizeEstimator, Utils} +import org.apache.spark.{SizeEstimator, Utils} import java.nio.ByteBuffer import collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/spark/storage/PutResult.scala b/core/src/main/scala/org/apache/spark/storage/PutResult.scala similarity index 97% rename from core/src/main/scala/spark/storage/PutResult.scala rename to core/src/main/scala/org/apache/spark/storage/PutResult.scala index 3a0974fe15f..2eba2f06b5b 100644 --- a/core/src/main/scala/spark/storage/PutResult.scala +++ b/core/src/main/scala/org/apache/spark/storage/PutResult.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage import java.nio.ByteBuffer diff --git a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala similarity index 96% rename from core/src/main/scala/spark/storage/ShuffleBlockManager.scala rename to core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 8a7a6f9ed3d..9da11efb570 100644 --- a/core/src/main/scala/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage -import spark.serializer.Serializer +import org.apache.spark.serializer.Serializer private[spark] diff --git a/core/src/main/scala/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala similarity index 97% rename from core/src/main/scala/spark/storage/StorageLevel.scala rename to core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index f52650988c9..755f1a760ee 100644 --- a/core/src/main/scala/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} @@ -23,7 +23,7 @@ import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} * Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, * whether to drop the RDD to disk if it falls out of memory, whether to keep the data in memory * in a serialized format, and whether to replicate the RDD partitions on multiple nodes. - * The [[spark.storage.StorageLevel$]] singleton object contains some static constants for + * The [[org.apache.spark.storage.StorageLevel$]] singleton object contains some static constants for * commonly useful storage levels. To create your own storage level object, use the factor method * of the singleton object (`StorageLevel(...)`). */ diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala similarity index 98% rename from core/src/main/scala/spark/storage/StorageUtils.scala rename to core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 123b8f63457..0bba1dac540 100644 --- a/core/src/main/scala/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage -import spark.{Utils, SparkContext} +import org.apache.spark.{Utils, SparkContext} import BlockManagerMasterActor.BlockStatus private[spark] diff --git a/core/src/main/scala/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala similarity index 98% rename from core/src/main/scala/spark/storage/ThreadingTest.scala rename to core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index b3ab1ff4b4d..1d5afe9b085 100644 --- a/core/src/main/scala/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage import akka.actor._ -import spark.KryoSerializer +import org.apache.spark.KryoSerializer import java.util.concurrent.ArrayBlockingQueue import util.Random diff --git a/core/src/main/scala/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala similarity index 98% rename from core/src/main/scala/spark/ui/JettyUtils.scala rename to core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index f66fe399052..cfa18f6ea4c 100644 --- a/core/src/main/scala/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.ui +package org.apache.spark.ui import javax.servlet.http.{HttpServletResponse, HttpServletRequest} @@ -29,7 +29,7 @@ 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 spark.Logging +import org.apache.spark.Logging /** Utilities for launching a web server using Jetty's HTTP Server class */ diff --git a/core/src/main/scala/spark/ui/Page.scala b/core/src/main/scala/org/apache/spark/ui/Page.scala similarity index 97% rename from core/src/main/scala/spark/ui/Page.scala rename to core/src/main/scala/org/apache/spark/ui/Page.scala index 87376a19d86..b2a069a3755 100644 --- a/core/src/main/scala/spark/ui/Page.scala +++ b/core/src/main/scala/org/apache/spark/ui/Page.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.ui +package org.apache.spark.ui private[spark] object Page extends Enumeration { val Stages, Storage, Environment, Executors = Value diff --git a/core/src/main/scala/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala similarity index 88% rename from core/src/main/scala/spark/ui/SparkUI.scala rename to core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 23ded44ba38..4688effe0ad 100644 --- a/core/src/main/scala/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -15,18 +15,18 @@ * limitations under the License. */ -package spark.ui +package org.apache.spark.ui import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} -import spark.{Logging, SparkContext, SparkEnv, 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._ +import org.apache.spark.{Logging, SparkContext, SparkEnv, Utils} +import org.apache.spark.ui.env.EnvironmentUI +import org.apache.spark.ui.exec.ExecutorsUI +import org.apache.spark.ui.storage.BlockManagerUI +import org.apache.spark.ui.jobs.JobProgressUI +import org.apache.spark.ui.JettyUtils._ /** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { @@ -83,5 +83,5 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { private[spark] object SparkUI { val DEFAULT_PORT = "3030" - val STATIC_RESOURCE_DIR = "spark/ui/static" + val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" } diff --git a/core/src/main/scala/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala similarity index 98% rename from core/src/main/scala/spark/ui/UIUtils.scala rename to core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 51bb18d888d..ce1acf564cc 100644 --- a/core/src/main/scala/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.ui +package org.apache.spark.ui import scala.xml.Node -import spark.SparkContext +import org.apache.spark.SparkContext /** Utility functions for generating XML pages with spark content. */ private[spark] object UIUtils { diff --git a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala similarity index 95% rename from core/src/main/scala/spark/ui/UIWorkloadGenerator.scala rename to core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index 5ff0572f0a3..0ecb22d2f96 100644 --- a/core/src/main/scala/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.ui +package org.apache.spark.ui import scala.util.Random -import spark.SparkContext -import spark.SparkContext._ -import spark.scheduler.cluster.SchedulingMode +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.scheduler.cluster.SchedulingMode /** diff --git a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala similarity index 94% rename from core/src/main/scala/spark/ui/env/EnvironmentUI.scala rename to core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index b1be1a27efe..c5bf2acc9ec 100644 --- a/core/src/main/scala/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.ui.env +package org.apache.spark.ui.env import javax.servlet.http.HttpServletRequest @@ -25,10 +25,10 @@ import scala.xml.Node import org.eclipse.jetty.server.Handler -import spark.ui.JettyUtils._ -import spark.ui.UIUtils -import spark.ui.Page.Environment -import spark.SparkContext +import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.Page.Environment +import org.apache.spark.SparkContext private[spark] class EnvironmentUI(sc: SparkContext) { diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala similarity index 91% rename from core/src/main/scala/spark/ui/exec/ExecutorsUI.scala rename to core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 0a7021fbf86..efe6b474e00 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -1,4 +1,4 @@ -package spark.ui.exec +package org.apache.spark.ui.exec import javax.servlet.http.HttpServletRequest @@ -7,13 +7,13 @@ import scala.xml.Node import org.eclipse.jetty.server.Handler -import spark.{ExceptionFailure, Logging, Utils, SparkContext} -import spark.executor.TaskMetrics -import spark.scheduler.cluster.TaskInfo -import spark.scheduler.{SparkListenerTaskStart, SparkListenerTaskEnd, SparkListener} -import spark.ui.JettyUtils._ -import spark.ui.Page.Executors -import spark.ui.UIUtils +import org.apache.spark.{ExceptionFailure, Logging, Utils, SparkContext} +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler.cluster.TaskInfo +import org.apache.spark.scheduler.{SparkListenerTaskStart, SparkListenerTaskEnd, SparkListener} +import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.ui.Page.Executors +import org.apache.spark.ui.UIUtils private[spark] class ExecutorsUI(val sc: SparkContext) { diff --git a/core/src/main/scala/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala similarity index 95% rename from core/src/main/scala/spark/ui/jobs/IndexPage.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 8867a6c90c1..3b428effafa 100644 --- a/core/src/main/scala/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -15,15 +15,15 @@ * limitations under the License. */ -package spark.ui.jobs +package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest import scala.xml.{NodeSeq, Node} -import spark.scheduler.cluster.SchedulingMode -import spark.ui.Page._ -import spark.ui.UIUtils._ +import org.apache.spark.scheduler.cluster.SchedulingMode +import org.apache.spark.ui.Page._ +import org.apache.spark.ui.UIUtils._ /** Page showing list of all ongoing and recently finished stages and pools*/ diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala similarity index 96% rename from core/src/main/scala/spark/ui/jobs/JobProgressListener.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 1d9767a83c8..ae022263000 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -1,12 +1,12 @@ -package spark.ui.jobs +package org.apache.spark.ui.jobs import scala.Seq import scala.collection.mutable.{ListBuffer, HashMap, HashSet} -import spark.{ExceptionFailure, SparkContext, Success, Utils} -import spark.scheduler._ -import spark.scheduler.cluster.TaskInfo -import spark.executor.TaskMetrics +import org.apache.spark.{ExceptionFailure, SparkContext, Success, Utils} +import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster.TaskInfo +import org.apache.spark.executor.TaskMetrics import collection.mutable /** diff --git a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala similarity index 86% rename from core/src/main/scala/spark/ui/jobs/JobProgressUI.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index c83f102ff32..1bb7638bd9d 100644 --- a/core/src/main/scala/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.ui.jobs +package org.apache.spark.ui.jobs import akka.util.Duration @@ -28,12 +28,12 @@ import org.eclipse.jetty.server.Handler import scala.Seq import scala.collection.mutable.{HashSet, ListBuffer, HashMap, ArrayBuffer} -import spark.ui.JettyUtils._ -import spark.{ExceptionFailure, SparkContext, Success, Utils} -import spark.scheduler._ +import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.{ExceptionFailure, SparkContext, Success, Utils} +import org.apache.spark.scheduler._ import collection.mutable -import spark.scheduler.cluster.SchedulingMode -import spark.scheduler.cluster.SchedulingMode.SchedulingMode +import org.apache.spark.scheduler.cluster.SchedulingMode +import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode /** Web UI showing progress status of all jobs in the given SparkContext. */ private[spark] class JobProgressUI(val sc: SparkContext) { diff --git a/core/src/main/scala/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala similarity index 87% rename from core/src/main/scala/spark/ui/jobs/PoolPage.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 7fb74dce406..ce92b6932b3 100644 --- a/core/src/main/scala/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -1,13 +1,13 @@ -package spark.ui.jobs +package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest import scala.xml.{NodeSeq, Node} import scala.collection.mutable.HashSet -import spark.scheduler.Stage -import spark.ui.UIUtils._ -import spark.ui.Page._ +import org.apache.spark.scheduler.Stage +import org.apache.spark.ui.UIUtils._ +import org.apache.spark.ui.Page._ /** Page showing specific pool details */ private[spark] class PoolPage(parent: JobProgressUI) { diff --git a/core/src/main/scala/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala similarity index 92% rename from core/src/main/scala/spark/ui/jobs/PoolTable.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index 621828f9c3e..f31465e59da 100644 --- a/core/src/main/scala/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -1,11 +1,11 @@ -package spark.ui.jobs +package org.apache.spark.ui.jobs import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet import scala.xml.Node -import spark.scheduler.Stage -import spark.scheduler.cluster.Schedulable +import org.apache.spark.scheduler.Stage +import org.apache.spark.scheduler.cluster.Schedulable /** Table showing list of pools */ private[spark] class PoolTable(pools: Seq[Schedulable], listener: JobProgressListener) { diff --git a/core/src/main/scala/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala similarity index 95% rename from core/src/main/scala/spark/ui/jobs/StagePage.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index c2341475c7f..2fe85bc0cf3 100644 --- a/core/src/main/scala/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.ui.jobs +package org.apache.spark.ui.jobs import java.util.Date @@ -23,12 +23,12 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import spark.ui.UIUtils._ -import spark.ui.Page._ -import spark.util.Distribution -import spark.{ExceptionFailure, Utils} -import spark.scheduler.cluster.TaskInfo -import spark.executor.TaskMetrics +import org.apache.spark.ui.UIUtils._ +import org.apache.spark.ui.Page._ +import org.apache.spark.util.Distribution +import org.apache.spark.{ExceptionFailure, Utils} +import org.apache.spark.scheduler.cluster.TaskInfo +import org.apache.spark.executor.TaskMetrics /** Page showing statistics and task list for a given stage */ private[spark] class StagePage(parent: JobProgressUI) { diff --git a/core/src/main/scala/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala similarity index 95% rename from core/src/main/scala/spark/ui/jobs/StageTable.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 2b1bc984fca..beb05745486 100644 --- a/core/src/main/scala/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -1,13 +1,13 @@ -package spark.ui.jobs +package org.apache.spark.ui.jobs import java.util.Date import scala.xml.Node import scala.collection.mutable.HashSet -import spark.Utils -import spark.scheduler.cluster.{SchedulingMode, TaskInfo} -import spark.scheduler.Stage +import org.apache.spark.Utils +import org.apache.spark.scheduler.cluster.{SchedulingMode, TaskInfo} +import org.apache.spark.scheduler.Stage /** Page showing list of all ongoing and recently finished stages */ diff --git a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala similarity index 92% rename from core/src/main/scala/spark/ui/storage/BlockManagerUI.scala rename to core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index 49ed069c759..1d633d374a0 100644 --- a/core/src/main/scala/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.ui.storage +package org.apache.spark.ui.storage import akka.util.Duration @@ -23,8 +23,8 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.Handler -import spark.{Logging, SparkContext} -import spark.ui.JettyUtils._ +import org.apache.spark.{Logging, SparkContext} +import org.apache.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 { diff --git a/core/src/main/scala/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala similarity index 91% rename from core/src/main/scala/spark/ui/storage/IndexPage.scala rename to core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index fc6273c6945..1eb4a7a85ea 100644 --- a/core/src/main/scala/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -15,16 +15,16 @@ * limitations under the License. */ -package spark.ui.storage +package org.apache.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 spark.ui.Page._ +import org.apache.spark.storage.{RDDInfo, StorageUtils} +import org.apache.spark.Utils +import org.apache.spark.ui.UIUtils._ +import org.apache.spark.ui.Page._ /** Page showing list of RDD's currently stored in the cluster */ private[spark] class IndexPage(parent: BlockManagerUI) { diff --git a/core/src/main/scala/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala similarity index 94% rename from core/src/main/scala/spark/ui/storage/RDDPage.scala rename to core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index b128a5614dc..37baf17f7a5 100644 --- a/core/src/main/scala/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -15,17 +15,17 @@ * limitations under the License. */ -package spark.ui.storage +package org.apache.spark.ui.storage import javax.servlet.http.HttpServletRequest import scala.xml.Node -import spark.Utils -import spark.storage.{StorageStatus, StorageUtils} -import spark.storage.BlockManagerMasterActor.BlockStatus -import spark.ui.UIUtils._ -import spark.ui.Page._ +import org.apache.spark.Utils +import org.apache.spark.storage.{StorageStatus, StorageUtils} +import org.apache.spark.storage.BlockManagerMasterActor.BlockStatus +import org.apache.spark.ui.UIUtils._ +import org.apache.spark.ui.Page._ /** Page showing storage details for a given RDD */ diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala similarity index 99% rename from core/src/main/scala/spark/util/AkkaUtils.scala rename to core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 9233277bdb3..d4c5065c3f5 100644 --- a/core/src/main/scala/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util import akka.actor.{ActorSystem, ExtendedActorSystem} import com.typesafe.config.ConfigFactory diff --git a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala similarity index 98% rename from core/src/main/scala/spark/util/BoundedPriorityQueue.scala rename to core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala index 0575497f5d8..0b51c23f7b2 100644 --- a/core/src/main/scala/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util import java.io.Serializable import java.util.{PriorityQueue => JPriorityQueue} diff --git a/core/src/main/scala/spark/util/ByteBufferInputStream.scala b/core/src/main/scala/org/apache/spark/util/ByteBufferInputStream.scala similarity index 96% rename from core/src/main/scala/spark/util/ByteBufferInputStream.scala rename to core/src/main/scala/org/apache/spark/util/ByteBufferInputStream.scala index 47a28e2f76e..e214d2a519a 100644 --- a/core/src/main/scala/spark/util/ByteBufferInputStream.scala +++ b/core/src/main/scala/org/apache/spark/util/ByteBufferInputStream.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util import java.io.InputStream import java.nio.ByteBuffer -import spark.storage.BlockManager +import org.apache.spark.storage.BlockManager /** * Reads data from a ByteBuffer, and optionally cleans it up using BlockManager.dispose() diff --git a/core/src/main/scala/spark/util/Clock.scala b/core/src/main/scala/org/apache/spark/util/Clock.scala similarity index 97% rename from core/src/main/scala/spark/util/Clock.scala rename to core/src/main/scala/org/apache/spark/util/Clock.scala index aa71a5b4424..97c2b45aabf 100644 --- a/core/src/main/scala/spark/util/Clock.scala +++ b/core/src/main/scala/org/apache/spark/util/Clock.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util /** * An interface to represent clocks, so that they can be mocked out in unit tests. diff --git a/core/src/main/scala/spark/util/CompletionIterator.scala b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala similarity index 97% rename from core/src/main/scala/spark/util/CompletionIterator.scala rename to core/src/main/scala/org/apache/spark/util/CompletionIterator.scala index 210450892b0..dc15a38b29d 100644 --- a/core/src/main/scala/spark/util/CompletionIterator.scala +++ b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util /** * Wrapper around an iterator which calls a completion method after it successfully iterates through all the elements diff --git a/core/src/main/scala/spark/util/Distribution.scala b/core/src/main/scala/org/apache/spark/util/Distribution.scala similarity index 98% rename from core/src/main/scala/spark/util/Distribution.scala rename to core/src/main/scala/org/apache/spark/util/Distribution.scala index 5d4d7a6c50a..33bf3562fe3 100644 --- a/core/src/main/scala/spark/util/Distribution.scala +++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util import java.io.PrintStream diff --git a/core/src/main/scala/spark/util/IdGenerator.scala b/core/src/main/scala/org/apache/spark/util/IdGenerator.scala similarity index 97% rename from core/src/main/scala/spark/util/IdGenerator.scala rename to core/src/main/scala/org/apache/spark/util/IdGenerator.scala index 34222805595..17e55f7996b 100644 --- a/core/src/main/scala/spark/util/IdGenerator.scala +++ b/core/src/main/scala/org/apache/spark/util/IdGenerator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util import java.util.concurrent.atomic.AtomicInteger diff --git a/core/src/main/scala/spark/util/IntParam.scala b/core/src/main/scala/org/apache/spark/util/IntParam.scala similarity index 97% rename from core/src/main/scala/spark/util/IntParam.scala rename to core/src/main/scala/org/apache/spark/util/IntParam.scala index daf0d58fa25..626bb49eeae 100644 --- a/core/src/main/scala/spark/util/IntParam.scala +++ b/core/src/main/scala/org/apache/spark/util/IntParam.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util /** * An extractor object for parsing strings into integers. diff --git a/core/src/main/scala/spark/util/MemoryParam.scala b/core/src/main/scala/org/apache/spark/util/MemoryParam.scala similarity index 95% rename from core/src/main/scala/spark/util/MemoryParam.scala rename to core/src/main/scala/org/apache/spark/util/MemoryParam.scala index 298562323ae..0ee6707826d 100644 --- a/core/src/main/scala/spark/util/MemoryParam.scala +++ b/core/src/main/scala/org/apache/spark/util/MemoryParam.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util -import spark.Utils +import org.apache.spark.Utils /** * An extractor object for parsing JVM memory strings, such as "10g", into an Int representing diff --git a/core/src/main/scala/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala similarity index 97% rename from core/src/main/scala/spark/util/MetadataCleaner.scala rename to core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 92909e09590..a430a754511 100644 --- a/core/src/main/scala/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util import java.util.concurrent.{TimeUnit, ScheduledFuture, Executors} import java.util.{TimerTask, Timer} -import spark.Logging +import org.apache.spark.Logging /** diff --git a/core/src/main/scala/spark/util/MutablePair.scala b/core/src/main/scala/org/apache/spark/util/MutablePair.scala similarity index 97% rename from core/src/main/scala/spark/util/MutablePair.scala rename to core/src/main/scala/org/apache/spark/util/MutablePair.scala index 78d404e66b1..34f1f6606fc 100644 --- a/core/src/main/scala/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util /** diff --git a/core/src/main/scala/spark/util/NextIterator.scala b/core/src/main/scala/org/apache/spark/util/NextIterator.scala similarity index 98% rename from core/src/main/scala/spark/util/NextIterator.scala rename to core/src/main/scala/org/apache/spark/util/NextIterator.scala index 22163ece8dd..8266e5e495e 100644 --- a/core/src/main/scala/spark/util/NextIterator.scala +++ b/core/src/main/scala/org/apache/spark/util/NextIterator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util /** Provides a basic/boilerplate Iterator implementation. */ private[spark] abstract class NextIterator[U] extends Iterator[U] { diff --git a/core/src/main/scala/spark/util/RateLimitedOutputStream.scala b/core/src/main/scala/org/apache/spark/util/RateLimitedOutputStream.scala similarity index 98% rename from core/src/main/scala/spark/util/RateLimitedOutputStream.scala rename to core/src/main/scala/org/apache/spark/util/RateLimitedOutputStream.scala index 00f782bbe76..47e1b450043 100644 --- a/core/src/main/scala/spark/util/RateLimitedOutputStream.scala +++ b/core/src/main/scala/org/apache/spark/util/RateLimitedOutputStream.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util import scala.annotation.tailrec diff --git a/core/src/main/scala/spark/util/SerializableBuffer.scala b/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala similarity index 98% rename from core/src/main/scala/spark/util/SerializableBuffer.scala rename to core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala index 7e6842628a6..f2b1ad7d0e9 100644 --- a/core/src/main/scala/spark/util/SerializableBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util import java.nio.ByteBuffer import java.io.{IOException, ObjectOutputStream, EOFException, ObjectInputStream} diff --git a/core/src/main/scala/spark/util/StatCounter.scala b/core/src/main/scala/org/apache/spark/util/StatCounter.scala similarity index 99% rename from core/src/main/scala/spark/util/StatCounter.scala rename to core/src/main/scala/org/apache/spark/util/StatCounter.scala index 76358d4151b..020d5edba95 100644 --- a/core/src/main/scala/spark/util/StatCounter.scala +++ b/core/src/main/scala/org/apache/spark/util/StatCounter.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util /** * A class for tracking the statistics of a set of numbers (count, mean and variance) in a diff --git a/core/src/main/scala/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala similarity index 95% rename from core/src/main/scala/spark/util/TimeStampedHashMap.scala rename to core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala index 07772a0afbe..277de2f8a6c 100644 --- a/core/src/main/scala/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala @@ -15,13 +15,14 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConversions import scala.collection.mutable.Map import scala.collection.immutable -import spark.scheduler.MapStatus +import org.apache.spark.scheduler.MapStatus +import org.apache.spark.Logging /** * This is a custom implementation of scala.collection.mutable.Map which stores the insertion @@ -29,7 +30,7 @@ import spark.scheduler.MapStatus * threshold time can them be removed using the clearOldValues method. This is intended to be a drop-in * replacement of scala.collection.mutable.HashMap. */ -class TimeStampedHashMap[A, B] extends Map[A, B]() with spark.Logging { +class TimeStampedHashMap[A, B] extends Map[A, B]() with Logging { val internalMap = new ConcurrentHashMap[A, (B, Long)]() def get(key: A): Option[B] = { diff --git a/core/src/main/scala/spark/util/TimeStampedHashSet.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala similarity index 98% rename from core/src/main/scala/spark/util/TimeStampedHashSet.scala rename to core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala index 41e3fd8cba1..26983138ff0 100644 --- a/core/src/main/scala/spark/util/TimeStampedHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util import scala.collection.mutable.Set import scala.collection.JavaConversions diff --git a/core/src/main/scala/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala similarity index 97% rename from core/src/main/scala/spark/util/Vector.scala rename to core/src/main/scala/org/apache/spark/util/Vector.scala index a47cac3b968..fe710c58acc 100644 --- a/core/src/main/scala/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util class Vector(val elements: Array[Double]) extends Serializable { def length = elements.length @@ -130,7 +130,7 @@ object Vector { implicit def doubleToMultiplier(num: Double) = new Multiplier(num) - implicit object VectorAccumParam extends spark.AccumulatorParam[Vector] { + implicit object VectorAccumParam extends org.apache.spark.AccumulatorParam[Vector] { def addInPlace(t1: Vector, t2: Vector) = t1 + t2 def zero(initialValue: Vector) = Vector.zeros(initialValue.length) diff --git a/core/src/test/resources/test_metrics_config.properties b/core/src/test/resources/test_metrics_config.properties index 2b31ddf2ebc..056a1584569 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 = seconds -*.source.jvm.class = spark.metrics.source.JvmSource +*.source.jvm.class = org.apache.spark.metrics.source.JvmSource master.sink.console.period = 20 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 d5479f02980..6f5ecea93a3 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 = 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.class = org.apache.spark.metrics.sink.ConsoleSink +test.sink.dummy.class = org.apache.spark.metrics.sink.DummySink +test.source.dummy.class = org.apache.spark.metrics.source.DummySource test.sink.console.period = 20 test.sink.console.unit = minutes diff --git a/core/src/test/scala/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala similarity index 98% rename from core/src/test/scala/spark/AccumulatorSuite.scala rename to core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 0af175f3168..4434f3b87c9 100644 --- a/core/src/test/scala/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers @@ -23,7 +23,7 @@ import collection.mutable import java.util.Random import scala.math.exp import scala.math.signum -import spark.SparkContext._ +import org.apache.spark.SparkContext._ class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkContext { diff --git a/core/src/test/scala/spark/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/BroadcastSuite.scala similarity index 98% rename from core/src/test/scala/spark/BroadcastSuite.scala rename to core/src/test/scala/org/apache/spark/BroadcastSuite.scala index 785721ece86..b3a53d928b4 100644 --- a/core/src/test/scala/spark/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/BroadcastSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala similarity index 99% rename from core/src/test/scala/spark/CheckpointSuite.scala rename to core/src/test/scala/org/apache/spark/CheckpointSuite.scala index 966dede2be1..23b14f4245e 100644 --- a/core/src/test/scala/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark +package org.apache.spark import org.scalatest.FunSuite import java.io.File -import spark.rdd._ -import spark.SparkContext._ +import org.apache.spark.rdd._ +import org.apache.spark.SparkContext._ import storage.StorageLevel class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { diff --git a/core/src/test/scala/spark/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ClosureCleanerSuite.scala similarity index 98% rename from core/src/test/scala/spark/ClosureCleanerSuite.scala rename to core/src/test/scala/org/apache/spark/ClosureCleanerSuite.scala index 7d2831e19c9..8494899b984 100644 --- a/core/src/test/scala/spark/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ClosureCleanerSuite.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark +package org.apache.spark import java.io.NotSerializableException import org.scalatest.FunSuite -import spark.LocalSparkContext._ +import org.apache.spark.LocalSparkContext._ import SparkContext._ class ClosureCleanerSuite extends FunSuite { diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala similarity index 99% rename from core/src/test/scala/spark/DistributedSuite.scala rename to core/src/test/scala/org/apache/spark/DistributedSuite.scala index e11efe459c8..7a856d40817 100644 --- a/core/src/test/scala/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import network.ConnectionManagerId import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala similarity index 93% rename from core/src/test/scala/spark/DriverSuite.scala rename to core/src/test/scala/org/apache/spark/DriverSuite.scala index 553c0309f66..b08aad1a6fd 100644 --- a/core/src/test/scala/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import java.io.File @@ -34,7 +34,7 @@ class DriverSuite extends FunSuite with Timeouts { val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]")) forAll(masters) { (master: String) => failAfter(30 seconds) { - Utils.execute(Seq("./spark-class", "spark.DriverWithoutCleanup", master), + Utils.execute(Seq("./spark-class", "org.apache.spark.DriverWithoutCleanup", master), new File(System.getenv("SPARK_HOME"))) } } diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala similarity index 99% rename from core/src/test/scala/spark/FailureSuite.scala rename to core/src/test/scala/org/apache/spark/FailureSuite.scala index 5b133cdd6e8..ee89a7a3872 100644 --- a/core/src/test/scala/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala similarity index 99% rename from core/src/test/scala/spark/FileServerSuite.scala rename to core/src/test/scala/org/apache/spark/FileServerSuite.scala index 242ae971f85..35d1d41af17 100644 --- a/core/src/test/scala/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import com.google.common.io.Files import org.scalatest.FunSuite diff --git a/core/src/test/scala/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala similarity index 99% rename from core/src/test/scala/spark/FileSuite.scala rename to core/src/test/scala/org/apache/spark/FileSuite.scala index 1e2c257c4b2..7b82a4cdd9c 100644 --- a/core/src/test/scala/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import java.io.{FileWriter, PrintWriter, File} diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/org/apache/spark/JavaAPISuite.java similarity index 98% rename from core/src/test/scala/spark/JavaAPISuite.java rename to core/src/test/scala/org/apache/spark/JavaAPISuite.java index c337c49268a..8a869c9005f 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/org/apache/spark/JavaAPISuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark; +package org.apache.spark; import java.io.File; import java.io.IOException; @@ -38,15 +38,15 @@ import org.junit.Before; import org.junit.Test; -import spark.api.java.JavaDoubleRDD; -import spark.api.java.JavaPairRDD; -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; -import spark.api.java.function.*; -import spark.partial.BoundedDouble; -import spark.partial.PartialResult; -import spark.storage.StorageLevel; -import spark.util.StatCounter; +import org.apache.spark.api.java.JavaDoubleRDD; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.*; +import org.apache.spark.partial.BoundedDouble; +import org.apache.spark.partial.PartialResult; +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.util.StatCounter; // The test suite itself is Serializable so that anonymous Function implementations can be diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/KryoSerializerSuite.scala similarity index 98% rename from core/src/test/scala/spark/KryoSerializerSuite.scala rename to core/src/test/scala/org/apache/spark/KryoSerializerSuite.scala index 7568a0bf659..d7b23c93fec 100644 --- a/core/src/test/scala/spark/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/KryoSerializerSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import scala.collection.mutable @@ -167,7 +167,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { } override def beforeAll() { - System.setProperty("spark.serializer", "spark.KryoSerializer") + System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer") System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName) super.beforeAll() } diff --git a/core/src/test/scala/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala similarity index 98% rename from core/src/test/scala/spark/LocalSparkContext.scala rename to core/src/test/scala/org/apache/spark/LocalSparkContext.scala index ddc212d290d..6ec124da9c7 100644 --- a/core/src/test/scala/spark/LocalSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import org.scalatest.Suite import org.scalatest.BeforeAndAfterEach diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala similarity index 97% rename from core/src/test/scala/spark/MapOutputTrackerSuite.scala rename to core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index c21f3331d08..6013320eaab 100644 --- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark +package org.apache.spark import org.scalatest.FunSuite import akka.actor._ -import spark.scheduler.MapStatus -import spark.storage.BlockManagerId -import spark.util.AkkaUtils +import org.apache.spark.scheduler.MapStatus +import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.AkkaUtils class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { diff --git a/core/src/test/scala/spark/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/PairRDDFunctionsSuite.scala similarity index 99% rename from core/src/test/scala/spark/PairRDDFunctionsSuite.scala rename to core/src/test/scala/org/apache/spark/PairRDDFunctionsSuite.scala index 328b3b5497e..f79752b34ef 100644 --- a/core/src/test/scala/spark/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/PairRDDFunctionsSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashSet @@ -23,7 +23,7 @@ import scala.collection.mutable.HashSet import org.scalatest.FunSuite import com.google.common.io.Files -import spark.SparkContext._ +import org.apache.spark.SparkContext._ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { diff --git a/core/src/test/scala/spark/PartitionPruningRDDSuite.scala b/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala similarity index 86% rename from core/src/test/scala/spark/PartitionPruningRDDSuite.scala rename to core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala index 88352b639f9..adbe805916b 100644 --- a/core/src/test/scala/spark/PartitionPruningRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala @@ -1,8 +1,8 @@ -package spark +package org.apache.spark import org.scalatest.FunSuite -import spark.SparkContext._ -import spark.rdd.PartitionPruningRDD +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.PartitionPruningRDD class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { diff --git a/core/src/test/scala/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala similarity index 99% rename from core/src/test/scala/spark/PartitioningSuite.scala rename to core/src/test/scala/org/apache/spark/PartitioningSuite.scala index b1e0b2b4d06..7669cf6fb14 100644 --- a/core/src/test/scala/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark +package org.apache.spark import org.scalatest.FunSuite import scala.collection.mutable.ArrayBuffer import SparkContext._ -import spark.util.StatCounter +import org.apache.spark.util.StatCounter import scala.math.abs class PartitioningSuite extends FunSuite with SharedSparkContext { diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala similarity index 99% rename from core/src/test/scala/spark/PipedRDDSuite.scala rename to core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index 35c04710a39..2e851d892dd 100644 --- a/core/src/test/scala/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import org.scalatest.FunSuite import SparkContext._ diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/org/apache/spark/RDDSuite.scala similarity index 99% rename from core/src/test/scala/spark/RDDSuite.scala rename to core/src/test/scala/org/apache/spark/RDDSuite.scala index e306952bbd6..342ba8adb20 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/RDDSuite.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark +package org.apache.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._ +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd._ import scala.collection.parallel.mutable class RDDSuite extends FunSuite with SharedSparkContext { diff --git a/core/src/test/scala/spark/SharedSparkContext.scala b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala similarity index 98% rename from core/src/test/scala/spark/SharedSparkContext.scala rename to core/src/test/scala/org/apache/spark/SharedSparkContext.scala index 70c24515be1..97cbca09bfa 100644 --- a/core/src/test/scala/spark/SharedSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import org.scalatest.Suite import org.scalatest.BeforeAndAfterAll diff --git a/core/src/test/scala/spark/ShuffleNettySuite.scala b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala similarity index 98% rename from core/src/test/scala/spark/ShuffleNettySuite.scala rename to core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala index 6bad6c1d13d..e121b162ad9 100644 --- a/core/src/test/scala/spark/ShuffleNettySuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import org.scalatest.BeforeAndAfterAll diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala similarity index 95% rename from core/src/test/scala/spark/ShuffleSuite.scala rename to core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 8745689c70b..357175e89ee 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -15,15 +15,15 @@ * limitations under the License. */ -package spark +package org.apache.spark import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers -import spark.SparkContext._ -import spark.ShuffleSuite.NonJavaSerializableClass -import spark.rdd.{SubtractedRDD, CoGroupedRDD, OrderedRDDFunctions, ShuffledRDD} -import spark.util.MutablePair +import org.apache.spark.SparkContext._ +import org.apache.spark.ShuffleSuite.NonJavaSerializableClass +import org.apache.spark.rdd.{SubtractedRDD, CoGroupedRDD, OrderedRDDFunctions, ShuffledRDD} +import org.apache.spark.util.MutablePair class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { @@ -54,7 +54,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { // 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[Int, NonJavaSerializableClass, (Int, NonJavaSerializableClass)]( - b, new HashPartitioner(NUM_BLOCKS)).setSerializer(classOf[spark.KryoSerializer].getName) + b, new HashPartitioner(NUM_BLOCKS)).setSerializer(classOf[KryoSerializer].getName) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId assert(c.count === 10) @@ -76,7 +76,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { // 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[Int, NonJavaSerializableClass, (Int, NonJavaSerializableClass)]( - b, new HashPartitioner(3)).setSerializer(classOf[spark.KryoSerializer].getName) + b, new HashPartitioner(3)).setSerializer(classOf[KryoSerializer].getName) assert(c.count === 10) } @@ -92,7 +92,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { // NOTE: The default Java serializer doesn't create zero-sized blocks. // So, use Kryo val c = new ShuffledRDD[Int, Int, (Int, Int)](b, new HashPartitioner(10)) - .setSerializer(classOf[spark.KryoSerializer].getName) + .setSerializer(classOf[KryoSerializer].getName) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId assert(c.count === 4) diff --git a/core/src/test/scala/spark/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/SizeEstimatorSuite.scala similarity index 99% rename from core/src/test/scala/spark/SizeEstimatorSuite.scala rename to core/src/test/scala/org/apache/spark/SizeEstimatorSuite.scala index 1ef812dfbdd..214ac74898f 100644 --- a/core/src/test/scala/spark/SizeEstimatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/SizeEstimatorSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import org.scalatest.FunSuite import org.scalatest.BeforeAndAfterAll diff --git a/core/src/test/scala/spark/SortingSuite.scala b/core/src/test/scala/org/apache/spark/SortingSuite.scala similarity index 99% rename from core/src/test/scala/spark/SortingSuite.scala rename to core/src/test/scala/org/apache/spark/SortingSuite.scala index b933c4aab83..f4fa9511dda 100644 --- a/core/src/test/scala/spark/SortingSuite.scala +++ b/core/src/test/scala/org/apache/spark/SortingSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter diff --git a/core/src/test/scala/spark/SparkContextInfoSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala similarity index 96% rename from core/src/test/scala/spark/SparkContextInfoSuite.scala rename to core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala index 6d50bf5e1b3..939fe518012 100644 --- a/core/src/test/scala/spark/SparkContextInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark +package org.apache.spark import org.scalatest.FunSuite -import spark.SparkContext._ +import org.apache.spark.SparkContext._ class SparkContextInfoSuite extends FunSuite with LocalSparkContext { test("getPersistentRDDs only returns RDDs that are marked as cached") { @@ -57,4 +57,4 @@ class SparkContextInfoSuite extends FunSuite with LocalSparkContext { rdd.collect() assert(sc.getRDDStorageInfo.size === 1) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/spark/ThreadingSuite.scala b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala similarity index 99% rename from core/src/test/scala/spark/ThreadingSuite.scala rename to core/src/test/scala/org/apache/spark/ThreadingSuite.scala index f2acd0bd3c1..69383ddfb8b 100644 --- a/core/src/test/scala/spark/ThreadingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import java.util.concurrent.Semaphore import java.util.concurrent.atomic.AtomicBoolean diff --git a/core/src/test/scala/spark/UnpersistSuite.scala b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala similarity index 96% rename from core/src/test/scala/spark/UnpersistSuite.scala rename to core/src/test/scala/org/apache/spark/UnpersistSuite.scala index 93977d16f4c..46a2da17240 100644 --- a/core/src/test/scala/spark/UnpersistSuite.scala +++ b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark +package org.apache.spark import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts._ import org.scalatest.time.{Span, Millis} -import spark.SparkContext._ +import org.apache.spark.SparkContext._ class UnpersistSuite extends FunSuite with LocalSparkContext { test("unpersist RDD") { diff --git a/core/src/test/scala/spark/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/UtilsSuite.scala similarity index 99% rename from core/src/test/scala/spark/UtilsSuite.scala rename to core/src/test/scala/org/apache/spark/UtilsSuite.scala index 98a6c1a1c9e..3a908720a85 100644 --- a/core/src/test/scala/spark/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/UtilsSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import com.google.common.base.Charsets import com.google.common.io.Files diff --git a/core/src/test/scala/spark/ZippedPartitionsSuite.scala b/core/src/test/scala/org/apache/spark/ZippedPartitionsSuite.scala similarity index 98% rename from core/src/test/scala/spark/ZippedPartitionsSuite.scala rename to core/src/test/scala/org/apache/spark/ZippedPartitionsSuite.scala index bb5d379273b..618b9c113b8 100644 --- a/core/src/test/scala/spark/ZippedPartitionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/ZippedPartitionsSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark +package org.apache.spark import scala.collection.immutable.NumericRange diff --git a/core/src/test/scala/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala similarity index 98% rename from core/src/test/scala/spark/io/CompressionCodecSuite.scala rename to core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala index 1ba82fe2b92..fd6f69041a9 100644 --- a/core/src/test/scala/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.io +package org.apache.spark.io import java.io.{ByteArrayInputStream, ByteArrayOutputStream} diff --git a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala similarity index 82% rename from core/src/test/scala/spark/metrics/MetricsConfigSuite.scala rename to core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala index b0213b62d9c..58c94a162d4 100644 --- a/core/src/test/scala/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.metrics +package org.apache.spark.metrics import org.scalatest.{BeforeAndAfter, FunSuite} @@ -35,7 +35,7 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { val property = conf.getInstance("random") assert(property.size() === 3) - assert(property.getProperty("sink.servlet.class") === "spark.metrics.sink.MetricsServlet") + assert(property.getProperty("sink.servlet.class") === "org.apache.spark.metrics.sink.MetricsServlet") assert(property.getProperty("sink.servlet.uri") === "/metrics/json") assert(property.getProperty("sink.servlet.sample") === "false") } @@ -48,8 +48,8 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { assert(masterProp.size() === 6) assert(masterProp.getProperty("sink.console.period") === "20") assert(masterProp.getProperty("sink.console.unit") === "minutes") - assert(masterProp.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") - assert(masterProp.getProperty("sink.servlet.class") === "spark.metrics.sink.MetricsServlet") + assert(masterProp.getProperty("source.jvm.class") === "org.apache.spark.metrics.source.JvmSource") + assert(masterProp.getProperty("sink.servlet.class") === "org.apache.spark.metrics.sink.MetricsServlet") assert(masterProp.getProperty("sink.servlet.uri") === "/metrics/master/json") assert(masterProp.getProperty("sink.servlet.sample") === "false") @@ -57,8 +57,8 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { assert(workerProp.size() === 6) assert(workerProp.getProperty("sink.console.period") === "10") assert(workerProp.getProperty("sink.console.unit") === "seconds") - assert(workerProp.getProperty("source.jvm.class") === "spark.metrics.source.JvmSource") - assert(workerProp.getProperty("sink.servlet.class") === "spark.metrics.sink.MetricsServlet") + assert(workerProp.getProperty("source.jvm.class") === "org.apache.spark.metrics.source.JvmSource") + assert(workerProp.getProperty("sink.servlet.class") === "org.apache.spark.metrics.sink.MetricsServlet") assert(workerProp.getProperty("sink.servlet.uri") === "/metrics/json") assert(workerProp.getProperty("sink.servlet.sample") === "false") } @@ -73,7 +73,7 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { val masterProp = conf.getInstance("master") val sourceProps = conf.subProperties(masterProp, MetricsSystem.SOURCE_REGEX) assert(sourceProps.size === 1) - assert(sourceProps("jvm").getProperty("class") === "spark.metrics.source.JvmSource") + assert(sourceProps("jvm").getProperty("class") === "org.apache.spark.metrics.source.JvmSource") val sinkProps = conf.subProperties(masterProp, MetricsSystem.SINK_REGEX) assert(sinkProps.size === 2) diff --git a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala similarity index 93% rename from core/src/test/scala/spark/metrics/MetricsSystemSuite.scala rename to core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala index dc65ac6994b..7181333adfe 100644 --- a/core/src/test/scala/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -15,9 +15,10 @@ * limitations under the License. */ -package spark.metrics +package org.apache.spark.metrics import org.scalatest.{BeforeAndAfter, FunSuite} +import org.apache.spark.deploy.master.MasterSource class MetricsSystemSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ @@ -46,7 +47,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { assert(sinks.length === 1) assert(!metricsSystem.getServletHandlers.isEmpty) - val source = new spark.deploy.master.MasterSource(null) + val source = new MasterSource(null) metricsSystem.registerSource(source) assert(sources.length === 1) } diff --git a/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala similarity index 95% rename from core/src/test/scala/spark/rdd/JdbcRDDSuite.scala rename to core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala index dc8ca941c10..3d39a31252e 100644 --- a/core/src/test/scala/spark/rdd/JdbcRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark +package org.apache.spark import org.scalatest.{ BeforeAndAfter, FunSuite } -import spark.SparkContext._ -import spark.rdd.JdbcRDD +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.JdbcRDD import java.sql._ class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { diff --git a/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala similarity index 99% rename from core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala rename to core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala index d1276d541f7..a80afdee7e7 100644 --- a/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.rdd +package org.apache.spark.rdd import scala.collection.immutable.NumericRange diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala similarity index 96% rename from core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 3b4a0d52fc7..94df282b284 100644 --- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -15,26 +15,26 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler import scala.collection.mutable.{Map, HashMap} import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter -import spark.LocalSparkContext -import spark.MapOutputTracker -import spark.RDD -import spark.SparkContext -import spark.Partition -import spark.TaskContext -import spark.{Dependency, ShuffleDependency, OneToOneDependency} -import spark.{FetchFailed, Success, TaskEndReason} -import spark.storage.{BlockManagerId, BlockManagerMaster} - -import spark.scheduler.cluster.Pool -import spark.scheduler.cluster.SchedulingMode -import spark.scheduler.cluster.SchedulingMode.SchedulingMode +import org.apache.spark.LocalSparkContext +import org.apache.spark.MapOutputTracker +import org.apache.spark.RDD +import org.apache.spark.SparkContext +import org.apache.spark.Partition +import org.apache.spark.TaskContext +import org.apache.spark.{Dependency, ShuffleDependency, OneToOneDependency} +import org.apache.spark.{FetchFailed, Success, TaskEndReason} +import org.apache.spark.storage.{BlockManagerId, BlockManagerMaster} + +import org.apache.spark.scheduler.cluster.Pool +import org.apache.spark.scheduler.cluster.SchedulingMode +import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode /** * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler diff --git a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala similarity index 98% rename from core/src/test/scala/spark/scheduler/JobLoggerSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala index bb9e715f95e..f5b3e972227 100644 --- a/core/src/test/scala/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala @@ -15,15 +15,15 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.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._ +import org.apache.spark._ +import org.apache.spark.SparkContext._ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers { diff --git a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala similarity index 96% rename from core/src/test/scala/spark/scheduler/SparkListenerSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 392d67d67bf..aac7c207cbe 100644 --- a/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler import org.scalatest.FunSuite -import spark.{SparkContext, LocalSparkContext} +import org.apache.spark.{SparkContext, LocalSparkContext} import scala.collection.mutable import org.scalatest.matchers.ShouldMatchers -import spark.SparkContext._ +import org.apache.spark.SparkContext._ /** * diff --git a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala similarity index 88% rename from core/src/test/scala/spark/scheduler/TaskContextSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index 95a6eee2fcf..0347cc02d7a 100644 --- a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -15,15 +15,15 @@ * limitations under the License. */ -package spark.scheduler +package org.apache.spark.scheduler import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter -import spark.TaskContext -import spark.RDD -import spark.SparkContext -import spark.Partition -import spark.LocalSparkContext +import org.apache.spark.TaskContext +import org.apache.spark.RDD +import org.apache.spark.SparkContext +import org.apache.spark.Partition +import org.apache.spark.LocalSparkContext class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { diff --git a/core/src/test/scala/spark/scheduler/cluster/ClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala similarity index 98% rename from core/src/test/scala/spark/scheduler/cluster/ClusterSchedulerSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala index abfdabf5fe6..92ad9f09b2d 100644 --- a/core/src/test/scala/spark/scheduler/cluster/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter -import spark._ -import spark.scheduler._ -import spark.scheduler.cluster._ +import org.apache.spark._ +import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster._ import scala.collection.mutable.ArrayBuffer import java.util.Properties diff --git a/core/src/test/scala/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala similarity index 98% rename from core/src/test/scala/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala index 5a0b949ef5f..a4f63baf3d8 100644 --- a/core/src/test/scala/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala @@ -15,18 +15,18 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster import scala.collection.mutable.ArrayBuffer import scala.collection.mutable import org.scalatest.FunSuite -import spark._ -import spark.scheduler._ -import spark.executor.TaskMetrics +import org.apache.spark._ +import org.apache.spark.scheduler._ +import org.apache.spark.executor.TaskMetrics import java.nio.ByteBuffer -import spark.util.FakeClock +import org.apache.spark.util.FakeClock /** * A mock ClusterScheduler implementation that just remembers information about tasks started and diff --git a/core/src/test/scala/spark/scheduler/cluster/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala similarity index 91% rename from core/src/test/scala/spark/scheduler/cluster/FakeTask.scala rename to core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala index de9e66be20a..2f12aaed18c 100644 --- a/core/src/test/scala/spark/scheduler/cluster/FakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster -import spark.scheduler.{TaskLocation, Task} +import org.apache.spark.scheduler.{TaskLocation, Task} class FakeTask(stageId: Int, prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId) { override def run(attemptId: Long): Int = 0 diff --git a/core/src/test/scala/spark/scheduler/local/LocalSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala similarity index 98% rename from core/src/test/scala/spark/scheduler/local/LocalSchedulerSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala index d28ee47fa33..111340a65ce 100644 --- a/core/src/test/scala/spark/scheduler/local/LocalSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.scheduler.local +package org.apache.spark.scheduler.local import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter -import spark._ -import spark.scheduler._ -import spark.scheduler.cluster._ +import org.apache.spark._ +import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster._ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.{ConcurrentMap, HashMap} import java.util.concurrent.Semaphore diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala similarity index 97% rename from core/src/test/scala/spark/storage/BlockManagerSuite.scala rename to core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index b719d65342d..88ba10f2f2d 100644 --- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.storage +package org.apache.spark.storage import java.nio.ByteBuffer @@ -29,11 +29,12 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.matchers.ShouldMatchers._ import org.scalatest.time.SpanSugar._ -import spark.JavaSerializer -import spark.KryoSerializer -import spark.SizeEstimator -import spark.util.AkkaUtils -import spark.util.ByteBufferInputStream +import org.apache.spark.JavaSerializer +import org.apache.spark.KryoSerializer +import org.apache.spark.SizeEstimator +import org.apache.spark.Utils +import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.ByteBufferInputStream class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester { @@ -56,7 +57,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT System.setProperty("spark.hostPort", "localhost:" + boundPort) master = new BlockManagerMaster( - actorSystem.actorOf(Props(new spark.storage.BlockManagerMasterActor(true)))) + actorSystem.actorOf(Props(new BlockManagerMasterActor(true)))) // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case oldArch = System.setProperty("os.arch", "amd64") @@ -65,7 +66,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() // Set some value ... - System.setProperty("spark.hostPort", spark.Utils.localHostName() + ":" + 1111) + System.setProperty("spark.hostPort", Utils.localHostName() + ":" + 1111) } after { @@ -105,10 +106,10 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(level2 === level1, "level2 is not same as level1") assert(level2.eq(level1), "level2 is not the same object as level1") assert(level3 != level1, "level3 is same as level1") - val bytes1 = spark.Utils.serialize(level1) - val level1_ = spark.Utils.deserialize[StorageLevel](bytes1) - val bytes2 = spark.Utils.serialize(level2) - val level2_ = spark.Utils.deserialize[StorageLevel](bytes2) + val bytes1 = Utils.serialize(level1) + val level1_ = Utils.deserialize[StorageLevel](bytes1) + val bytes2 = Utils.serialize(level2) + val level2_ = Utils.deserialize[StorageLevel](bytes2) assert(level1_ === level1, "Deserialized level1 not same as original level1") assert(level1_.eq(level1), "Deserialized level1 not the same object as original level2") assert(level2_ === level2, "Deserialized level2 not same as original level2") @@ -122,10 +123,10 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT 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") - val bytes1 = spark.Utils.serialize(id1) - val id1_ = spark.Utils.deserialize[BlockManagerId](bytes1) - val bytes2 = spark.Utils.serialize(id2) - val id2_ = spark.Utils.deserialize[BlockManagerId](bytes2) + val bytes1 = Utils.serialize(id1) + val id1_ = Utils.deserialize[BlockManagerId](bytes1) + val bytes2 = Utils.serialize(id2) + val id2_ = Utils.deserialize[BlockManagerId](bytes2) assert(id1_ === id1, "Deserialized id1 is not same as original id1") assert(id1_.eq(id1), "Deserialized id1 is not the same object as original id1") assert(id2_ === id2, "Deserialized id2 is not same as original id2") diff --git a/core/src/test/scala/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala similarity index 98% rename from core/src/test/scala/spark/ui/UISuite.scala rename to core/src/test/scala/org/apache/spark/ui/UISuite.scala index 735a794396f..3321fb5eb71 100644 --- a/core/src/test/scala/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.ui +package org.apache.spark.ui import scala.util.{Failure, Success, Try} import java.net.ServerSocket diff --git a/core/src/test/scala/spark/util/DistributionSuite.scala b/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala similarity index 97% rename from core/src/test/scala/spark/util/DistributionSuite.scala rename to core/src/test/scala/org/apache/spark/util/DistributionSuite.scala index 6578b55e824..63642461e44 100644 --- a/core/src/test/scala/spark/util/DistributionSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/DistributionSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers diff --git a/core/src/test/scala/spark/util/FakeClock.scala b/core/src/test/scala/org/apache/spark/util/FakeClock.scala similarity index 96% rename from core/src/test/scala/spark/util/FakeClock.scala rename to core/src/test/scala/org/apache/spark/util/FakeClock.scala index 236706317e3..0a45917b08d 100644 --- a/core/src/test/scala/spark/util/FakeClock.scala +++ b/core/src/test/scala/org/apache/spark/util/FakeClock.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util class FakeClock extends Clock { private var time = 0L diff --git a/core/src/test/scala/spark/util/NextIteratorSuite.scala b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala similarity index 98% rename from core/src/test/scala/spark/util/NextIteratorSuite.scala rename to core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala index fdbd43d941b..45867463a51 100644 --- a/core/src/test/scala/spark/util/NextIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers diff --git a/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala b/core/src/test/scala/org/apache/spark/util/RateLimitedOutputStreamSuite.scala similarity index 97% rename from core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala rename to core/src/test/scala/org/apache/spark/util/RateLimitedOutputStreamSuite.scala index 4c0044202f0..a9dd0b1a5b6 100644 --- a/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/RateLimitedOutputStreamSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.util +package org.apache.spark.util import org.scalatest.FunSuite import java.io.ByteArrayOutputStream diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 84749fda4e6..349eb92a47f 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -100,7 +100,7 @@
  • Tuning Guide
  • Hardware Provisioning
  • Building Spark with Maven
  • -
  • Contributing to Spark
  • +
  • Contributing to Spark
  • diff --git a/examples/pom.xml b/examples/pom.xml index 687fbcca8f3..13b55315113 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -19,13 +19,13 @@ 4.0.0 - org.spark-project + org.apache.spark spark-parent 0.8.0-SNAPSHOT ../pom.xml - org.spark-project + org.apache.spark spark-examples jar Spark Project Examples @@ -33,25 +33,25 @@ - org.spark-project + org.apache.spark spark-core ${project.version} provided - org.spark-project + org.apache.spark spark-streaming ${project.version} provided - org.spark-project + org.apache.spark spark-mllib ${project.version} provided - org.spark-project + org.apache.spark spark-bagel ${project.version} provided @@ -132,7 +132,7 @@ hadoop2-yarn - org.spark-project + org.apache.spark spark-yarn ${project.version} provided diff --git a/examples/src/main/java/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java similarity index 94% rename from examples/src/main/java/spark/examples/JavaHdfsLR.java rename to examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index 9485e0cfa96..be0d38589c5 100644 --- a/examples/src/main/java/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.examples; +package org.apache.spark.examples; -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; -import spark.api.java.function.Function; -import spark.api.java.function.Function2; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.Function2; import java.io.Serializable; import java.util.Arrays; diff --git a/examples/src/main/java/spark/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java similarity index 92% rename from examples/src/main/java/spark/examples/JavaKMeans.java rename to examples/src/main/java/org/apache/spark/examples/JavaKMeans.java index 2d34776177c..5a6afe7eaef 100644 --- a/examples/src/main/java/spark/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java @@ -15,15 +15,15 @@ * limitations under the License. */ -package spark.examples; +package org.apache.spark.examples; import scala.Tuple2; -import spark.api.java.JavaPairRDD; -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; -import spark.api.java.function.Function; -import spark.api.java.function.PairFunction; -import spark.util.Vector; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.util.Vector; import java.util.List; import java.util.Map; diff --git a/examples/src/main/java/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java similarity index 94% rename from examples/src/main/java/spark/examples/JavaLogQuery.java rename to examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java index d22684d9804..152f0292133 100644 --- a/examples/src/main/java/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -15,16 +15,16 @@ * limitations under the License. */ -package spark.examples; +package org.apache.spark.examples; import com.google.common.collect.Lists; import scala.Tuple2; import scala.Tuple3; -import spark.api.java.JavaPairRDD; -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; -import spark.api.java.function.Function2; -import spark.api.java.function.PairFunction; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.function.PairFunction; import java.io.Serializable; import java.util.Collections; diff --git a/examples/src/main/java/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java similarity index 89% rename from examples/src/main/java/spark/examples/JavaPageRank.java rename to examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index 75df1af2e30..c5603a639bd 100644 --- a/examples/src/main/java/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -15,17 +15,17 @@ * limitations under the License. */ -package spark.examples; +package org.apache.spark.examples; import scala.Tuple2; -import spark.api.java.JavaPairRDD; -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; -import spark.api.java.function.FlatMapFunction; -import spark.api.java.function.Function; -import spark.api.java.function.Function2; -import spark.api.java.function.PairFlatMapFunction; -import spark.api.java.function.PairFunction; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.function.PairFlatMapFunction; +import org.apache.spark.api.java.function.PairFunction; import java.util.List; import java.util.ArrayList; diff --git a/examples/src/main/java/spark/examples/JavaSparkPi.java b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java similarity index 89% rename from examples/src/main/java/spark/examples/JavaSparkPi.java rename to examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java index d5f42fbb38e..4a2380caf5a 100644 --- a/examples/src/main/java/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.examples; +package org.apache.spark.examples; -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; -import spark.api.java.function.Function; -import spark.api.java.function.Function2; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.Function2; import java.util.ArrayList; import java.util.List; diff --git a/examples/src/main/java/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java similarity index 94% rename from examples/src/main/java/spark/examples/JavaTC.java rename to examples/src/main/java/org/apache/spark/examples/JavaTC.java index 559d7f9e533..17f21f6b776 100644 --- a/examples/src/main/java/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.examples; +package org.apache.spark.examples; import scala.Tuple2; -import spark.api.java.JavaPairRDD; -import spark.api.java.JavaSparkContext; -import spark.api.java.function.PairFunction; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.PairFunction; import java.util.ArrayList; import java.util.HashSet; diff --git a/examples/src/main/java/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java similarity index 85% rename from examples/src/main/java/spark/examples/JavaWordCount.java rename to examples/src/main/java/org/apache/spark/examples/JavaWordCount.java index 1af370c1c38..07d32ad659a 100644 --- a/examples/src/main/java/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -15,15 +15,15 @@ * limitations under the License. */ -package spark.examples; +package org.apache.spark.examples; import scala.Tuple2; -import spark.api.java.JavaPairRDD; -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; -import spark.api.java.function.FlatMapFunction; -import spark.api.java.function.Function2; -import spark.api.java.function.PairFunction; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.function.PairFunction; import java.util.Arrays; import java.util.List; diff --git a/examples/src/main/java/spark/mllib/examples/JavaALS.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java similarity index 88% rename from examples/src/main/java/spark/mllib/examples/JavaALS.java rename to examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java index b48f459cb7e..628cb892b68 100644 --- a/examples/src/main/java/spark/mllib/examples/JavaALS.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java @@ -15,15 +15,15 @@ * limitations under the License. */ -package spark.mllib.examples; +package org.apache.spark.mllib.examples; -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; -import spark.api.java.function.Function; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; -import spark.mllib.recommendation.ALS; -import spark.mllib.recommendation.MatrixFactorizationModel; -import spark.mllib.recommendation.Rating; +import org.apache.spark.mllib.recommendation.ALS; +import org.apache.spark.mllib.recommendation.MatrixFactorizationModel; +import org.apache.spark.mllib.recommendation.Rating; import java.io.Serializable; import java.util.Arrays; diff --git a/examples/src/main/java/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java similarity index 89% rename from examples/src/main/java/spark/mllib/examples/JavaKMeans.java rename to examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java index 02f40438b82..cd59a139b9f 100644 --- a/examples/src/main/java/spark/mllib/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.mllib.examples; +package org.apache.spark.mllib.examples; -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; -import spark.api.java.function.Function; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; -import spark.mllib.clustering.KMeans; -import spark.mllib.clustering.KMeansModel; +import org.apache.spark.mllib.clustering.KMeans; +import org.apache.spark.mllib.clustering.KMeansModel; import java.util.Arrays; import java.util.StringTokenizer; diff --git a/examples/src/main/java/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java similarity index 87% rename from examples/src/main/java/spark/mllib/examples/JavaLR.java rename to examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java index bf4aeaf40f6..258061c8e6b 100644 --- a/examples/src/main/java/spark/mllib/examples/JavaLR.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java @@ -15,16 +15,16 @@ * limitations under the License. */ -package spark.mllib.examples; +package org.apache.spark.mllib.examples; -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; -import spark.api.java.function.Function; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; -import spark.mllib.classification.LogisticRegressionWithSGD; -import spark.mllib.classification.LogisticRegressionModel; -import spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.classification.LogisticRegressionWithSGD; +import org.apache.spark.mllib.classification.LogisticRegressionModel; +import org.apache.spark.mllib.regression.LabeledPoint; import java.util.Arrays; import java.util.StringTokenizer; diff --git a/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java similarity index 90% rename from examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java rename to examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java index 096a9ae2197..261813bf2f3 100644 --- a/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.streaming.examples; +package org.apache.spark.streaming.examples; -import spark.api.java.function.Function; -import spark.streaming.*; -import spark.streaming.api.java.*; -import spark.streaming.dstream.SparkFlumeEvent; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.streaming.*; +import org.apache.spark.streaming.api.java.*; +import org.apache.spark.streaming.dstream.SparkFlumeEvent; /** * Produces a count of events received from Flume. diff --git a/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java similarity index 86% rename from examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java rename to examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java index c54d3f3d599..def87c199be 100644 --- a/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java @@ -15,17 +15,17 @@ * limitations under the License. */ -package spark.streaming.examples; +package org.apache.spark.streaming.examples; import com.google.common.collect.Lists; import scala.Tuple2; -import spark.api.java.function.FlatMapFunction; -import spark.api.java.function.Function2; -import spark.api.java.function.PairFunction; -import spark.streaming.Duration; -import spark.streaming.api.java.JavaDStream; -import spark.streaming.api.java.JavaPairDStream; -import spark.streaming.api.java.JavaStreamingContext; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.streaming.Duration; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; /** * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. diff --git a/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java similarity index 85% rename from examples/src/main/java/spark/streaming/examples/JavaQueueStream.java rename to examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java index 1f4a9915428..c8c7389dd1b 100644 --- a/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java @@ -15,17 +15,17 @@ * limitations under the License. */ -package spark.streaming.examples; +package org.apache.spark.streaming.examples; import com.google.common.collect.Lists; import scala.Tuple2; -import spark.api.java.JavaRDD; -import spark.api.java.function.Function2; -import spark.api.java.function.PairFunction; -import spark.streaming.Duration; -import spark.streaming.api.java.JavaDStream; -import spark.streaming.api.java.JavaPairDStream; -import spark.streaming.api.java.JavaStreamingContext; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.streaming.Duration; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; import java.util.LinkedList; import java.util.List; diff --git a/examples/src/main/scala/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala similarity index 95% rename from examples/src/main/scala/spark/examples/BroadcastTest.scala rename to examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index 911490cb6c1..868ff81f67a 100644 --- a/examples/src/main/scala/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples -import spark.SparkContext +import org.apache.spark.SparkContext object BroadcastTest { def main(args: Array[String]) { diff --git a/examples/src/main/scala/spark/examples/CassandraTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala similarity index 98% rename from examples/src/main/scala/spark/examples/CassandraTest.scala rename to examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala index 104bfd52047..33bf7151a7c 100644 --- a/examples/src/main/scala/spark/examples/CassandraTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala @@ -15,15 +15,15 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples import org.apache.hadoop.mapreduce.Job 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.SparkContext -import spark.SparkContext._ +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ import java.nio.ByteBuffer import java.util.SortedMap import org.apache.cassandra.db.IColumn diff --git a/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala similarity index 94% rename from examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala rename to examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala index 67ddaec8d20..92eb96bd8e0 100644 --- a/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples -import spark.SparkContext +import org.apache.spark.SparkContext object ExceptionHandlingTest { def main(args: Array[String]) { diff --git a/examples/src/main/scala/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala similarity index 94% rename from examples/src/main/scala/spark/examples/GroupByTest.scala rename to examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala index 5cee4136158..42c2e0e8e19 100644 --- a/examples/src/main/scala/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples -import spark.SparkContext -import spark.SparkContext._ +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ import java.util.Random object GroupByTest { diff --git a/examples/src/main/scala/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala similarity index 94% rename from examples/src/main/scala/spark/examples/HBaseTest.scala rename to examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala index 4dd6c243acf..efe2e93b0dc 100644 --- a/examples/src/main/scala/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples -import spark._ -import spark.rdd.NewHadoopRDD +import org.apache.spark._ +import org.apache.spark.rdd.NewHadoopRDD import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor} import org.apache.hadoop.hbase.client.HBaseAdmin import org.apache.hadoop.hbase.mapreduce.TableInputFormat diff --git a/examples/src/main/scala/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala similarity index 95% rename from examples/src/main/scala/spark/examples/HdfsTest.scala rename to examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index 23258336e23..d6a88d3032c 100644 --- a/examples/src/main/scala/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples -import spark._ +import org.apache.spark._ object HdfsTest { def main(args: Array[String]) { diff --git a/examples/src/main/scala/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala similarity index 99% rename from examples/src/main/scala/spark/examples/LocalALS.scala rename to examples/src/main/scala/org/apache/spark/examples/LocalALS.scala index 7a449a9d720..4af45b2b4a0 100644 --- a/examples/src/main/scala/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples import scala.math.sqrt import cern.jet.math._ diff --git a/examples/src/main/scala/spark/examples/LocalFileLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala similarity index 96% rename from examples/src/main/scala/spark/examples/LocalFileLR.scala rename to examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala index c1f8d32aa87..fb130ea1988 100644 --- a/examples/src/main/scala/spark/examples/LocalFileLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples import java.util.Random -import spark.util.Vector +import org.apache.spark.util.Vector object LocalFileLR { val D = 10 // Numer of dimensions diff --git a/examples/src/main/scala/spark/examples/LocalKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala similarity index 96% rename from examples/src/main/scala/spark/examples/LocalKMeans.scala rename to examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala index 0a0bc6f476b..f90ea35cd44 100644 --- a/examples/src/main/scala/spark/examples/LocalKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples import java.util.Random -import spark.util.Vector -import spark.SparkContext._ +import org.apache.spark.util.Vector +import org.apache.spark.SparkContext._ import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet diff --git a/examples/src/main/scala/spark/examples/LocalLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala similarity index 96% rename from examples/src/main/scala/spark/examples/LocalLR.scala rename to examples/src/main/scala/org/apache/spark/examples/LocalLR.scala index ab99bf1fbe4..cd4e9f1af0e 100644 --- a/examples/src/main/scala/spark/examples/LocalLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples import java.util.Random -import spark.util.Vector +import org.apache.spark.util.Vector /** * Logistic regression based classification. diff --git a/examples/src/main/scala/spark/examples/LocalPi.scala b/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala similarity index 94% rename from examples/src/main/scala/spark/examples/LocalPi.scala rename to examples/src/main/scala/org/apache/spark/examples/LocalPi.scala index ccd69695df4..bb7f22ec8df 100644 --- a/examples/src/main/scala/spark/examples/LocalPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples import scala.math.random -import spark._ +import org.apache.spark._ import SparkContext._ object LocalPi { diff --git a/examples/src/main/scala/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala similarity index 97% rename from examples/src/main/scala/spark/examples/LogQuery.scala rename to examples/src/main/scala/org/apache/spark/examples/LogQuery.scala index e815ececf7a..17ff3ce7649 100644 --- a/examples/src/main/scala/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples -import spark.SparkContext -import spark.SparkContext._ +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ /** * Executes a roll up-style query against Apache logs. */ diff --git a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala similarity index 95% rename from examples/src/main/scala/spark/examples/MultiBroadcastTest.scala rename to examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala index d0b1cf06e59..f79f0142b86 100644 --- a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples -import spark.SparkContext +import org.apache.spark.SparkContext object MultiBroadcastTest { def main(args: Array[String]) { diff --git a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala similarity index 96% rename from examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala rename to examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index d197bbaf7c4..37ddfb5db76 100644 --- a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples -import spark.SparkContext -import spark.SparkContext._ +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ import java.util.Random object SimpleSkewedGroupByTest { diff --git a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala similarity index 95% rename from examples/src/main/scala/spark/examples/SkewedGroupByTest.scala rename to examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala index 4641b824448..9c954b2b5ba 100644 --- a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples -import spark.SparkContext -import spark.SparkContext._ +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ import java.util.Random object SkewedGroupByTest { diff --git a/examples/src/main/scala/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala similarity index 98% rename from examples/src/main/scala/spark/examples/SparkALS.scala rename to examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index ba0dfd8f9b5..814944ba1c6 100644 --- a/examples/src/main/scala/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples import scala.math.sqrt import cern.jet.math._ import cern.colt.matrix._ import cern.colt.matrix.linalg._ -import spark._ +import org.apache.spark._ /** * Alternating least squares matrix factorization. diff --git a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala similarity index 94% rename from examples/src/main/scala/spark/examples/SparkHdfsLR.scala rename to examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index 43c91156640..646682878fd 100644 --- a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples import java.util.Random import scala.math.exp -import spark.util.Vector -import spark._ -import spark.scheduler.InputFormatInfo +import org.apache.spark.util.Vector +import org.apache.spark._ +import org.apache.spark.scheduler.InputFormatInfo /** * Logistic regression based classification. diff --git a/examples/src/main/scala/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala similarity index 94% rename from examples/src/main/scala/spark/examples/SparkKMeans.scala rename to examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index 38ed3b149af..f7bf75b4e50 100644 --- a/examples/src/main/scala/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples import java.util.Random -import spark.SparkContext -import spark.util.Vector -import spark.SparkContext._ +import org.apache.spark.SparkContext +import org.apache.spark.util.Vector +import org.apache.spark.SparkContext._ import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet diff --git a/examples/src/main/scala/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala similarity index 95% rename from examples/src/main/scala/spark/examples/SparkLR.scala rename to examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index 52a0d69744a..9ed9fe4d761 100644 --- a/examples/src/main/scala/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples import java.util.Random import scala.math.exp -import spark.util.Vector -import spark._ +import org.apache.spark.util.Vector +import org.apache.spark._ /** * Logistic regression based classification. diff --git a/examples/src/main/scala/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala similarity index 91% rename from examples/src/main/scala/spark/examples/SparkPageRank.scala rename to examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index dedbbd01a3a..2721caf08bd 100644 --- a/examples/src/main/scala/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -1,7 +1,7 @@ -package spark.examples +package org.apache.spark.examples -import spark.SparkContext._ -import spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.SparkContext /** diff --git a/examples/src/main/scala/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala similarity index 96% rename from examples/src/main/scala/spark/examples/SparkPi.scala rename to examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index 00560ac9d16..5a2bc9b0d08 100644 --- a/examples/src/main/scala/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples import scala.math.random -import spark._ +import org.apache.spark._ import SparkContext._ /** Computes an approximation to pi */ diff --git a/examples/src/main/scala/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala similarity index 97% rename from examples/src/main/scala/spark/examples/SparkTC.scala rename to examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index bf988a953bb..5a7a9d1bd8f 100644 --- a/examples/src/main/scala/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.examples +package org.apache.spark.examples -import spark._ +import org.apache.spark._ import SparkContext._ import scala.util.Random import scala.collection.mutable diff --git a/examples/src/main/scala/spark/examples/bagel/PageRankUtils.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala similarity index 95% rename from examples/src/main/scala/spark/examples/bagel/PageRankUtils.scala rename to examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala index c23ee9895f2..b190e83c4d3 100644 --- a/examples/src/main/scala/spark/examples/bagel/PageRankUtils.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.examples.bagel +package org.apache.spark.examples.bagel -import spark._ -import spark.SparkContext._ +import org.apache.spark._ +import org.apache.spark.SparkContext._ -import spark.bagel._ -import spark.bagel.Bagel._ +import org.apache.spark.bagel._ +import org.apache.spark.bagel.Bagel._ import scala.collection.mutable.ArrayBuffer diff --git a/examples/src/main/scala/spark/examples/bagel/WikipediaPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala similarity index 93% rename from examples/src/main/scala/spark/examples/bagel/WikipediaPageRank.scala rename to examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala index 00635a7ffa2..b1f606e48e0 100644 --- a/examples/src/main/scala/spark/examples/bagel/WikipediaPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.examples.bagel +package org.apache.spark.examples.bagel -import spark._ -import spark.SparkContext._ +import org.apache.spark._ +import org.apache.spark.SparkContext._ -import spark.bagel._ -import spark.bagel.Bagel._ +import org.apache.spark.bagel._ +import org.apache.spark.bagel.Bagel._ import scala.xml.{XML,NodeSeq} @@ -37,7 +37,7 @@ object WikipediaPageRank { System.exit(-1) } - System.setProperty("spark.serializer", "spark.KryoSerializer") + System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer") System.setProperty("spark.kryo.registrator", classOf[PRKryoRegistrator].getName) val inputFile = args(0) diff --git a/examples/src/main/scala/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala similarity index 96% rename from examples/src/main/scala/spark/examples/bagel/WikipediaPageRankStandalone.scala rename to examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala index c416ddbc584..3bfa48eaf3d 100644 --- a/examples/src/main/scala/spark/examples/bagel/WikipediaPageRankStandalone.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.examples.bagel +package org.apache.spark.examples.bagel -import spark._ +import org.apache.spark._ import serializer.{DeserializationStream, SerializationStream, SerializerInstance} -import spark.SparkContext._ +import org.apache.spark.SparkContext._ -import spark.bagel._ -import spark.bagel.Bagel._ +import org.apache.spark.bagel._ +import org.apache.spark.bagel.Bagel._ import scala.xml.{XML,NodeSeq} @@ -131,7 +131,7 @@ object WikipediaPageRankStandalone { } } -class WPRSerializer extends spark.serializer.Serializer { +class WPRSerializer extends org.apache.spark.serializer.Serializer { def newInstance(): SerializerInstance = new WPRSerializerInstance() } diff --git a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala similarity index 93% rename from examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala rename to examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index 05d31764781..cd3423a07b6 100644 --- a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.streaming.examples +package org.apache.spark.streaming.examples import scala.collection.mutable.LinkedList import scala.util.Random @@ -25,11 +25,11 @@ import akka.actor.ActorRef import akka.actor.Props import akka.actor.actorRef2Scala -import spark.streaming.Seconds -import spark.streaming.StreamingContext -import spark.streaming.StreamingContext.toPairDStreamFunctions -import spark.streaming.receivers.Receiver -import spark.util.AkkaUtils +import org.apache.spark.streaming.Seconds +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions +import org.apache.spark.streaming.receivers.Receiver +import org.apache.spark.util.AkkaUtils case class SubscribeReceiver(receiverActor: ActorRef) case class UnsubscribeReceiver(receiverActor: ActorRef) @@ -80,7 +80,7 @@ class FeederActor extends Actor { * goes and subscribe to a typical publisher/feeder actor and receives * data. * - * @see [[spark.streaming.examples.FeederActor]] + * @see [[org.apache.spark.streaming.examples.FeederActor]] */ class SampleActorReceiver[T: ClassManifest](urlOfPublisher: String) extends Actor with Receiver { diff --git a/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala similarity index 92% rename from examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala rename to examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala index 3ab4fc2c37b..9f6e163454a 100644 --- a/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming.examples +package org.apache.spark.streaming.examples -import spark.util.IntParam -import spark.storage.StorageLevel -import spark.streaming._ +import org.apache.spark.util.IntParam +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming._ /** * Produces a count of events received from Flume. diff --git a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala similarity index 92% rename from examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala rename to examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala index 30af01a26f4..bc8564b3ba0 100644 --- a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.examples +package org.apache.spark.streaming.examples -import spark.streaming.{Seconds, StreamingContext} -import spark.streaming.StreamingContext._ +import org.apache.spark.streaming.{Seconds, StreamingContext} +import org.apache.spark.streaming.StreamingContext._ /** diff --git a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala similarity index 92% rename from examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala rename to examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index d9c76d1a33b..12f939d5a7e 100644 --- a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -15,17 +15,17 @@ * limitations under the License. */ -package spark.streaming.examples +package org.apache.spark.streaming.examples import java.util.Properties import kafka.message.Message import kafka.producer.SyncProducerConfig import kafka.producer._ -import spark.SparkContext -import spark.streaming._ -import spark.streaming.StreamingContext._ -import spark.storage.StorageLevel -import spark.streaming.util.RawTextHelper._ +import org.apache.spark.SparkContext +import org.apache.spark.streaming._ +import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.util.RawTextHelper._ /** * Consumes messages from one or more topics in Kafka and does wordcount. diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala similarity index 93% rename from examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala rename to examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala index b29d79aac53..e2487dca5f2 100644 --- a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.examples +package org.apache.spark.streaming.examples -import spark.streaming.{Seconds, StreamingContext} -import spark.streaming.StreamingContext._ +import org.apache.spark.streaming.{Seconds, StreamingContext} +import org.apache.spark.streaming.StreamingContext._ /** * Counts words in UTF8 encoded, '\n' delimited text received from the network every second. diff --git a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala similarity index 90% rename from examples/src/main/scala/spark/streaming/examples/QueueStream.scala rename to examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala index da36c8c23c6..822da8c9b5c 100644 --- a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming.examples +package org.apache.spark.streaming.examples -import spark.RDD -import spark.streaming.{Seconds, StreamingContext} -import spark.streaming.StreamingContext._ +import org.apache.spark.RDD +import org.apache.spark.streaming.{Seconds, StreamingContext} +import org.apache.spark.streaming.StreamingContext._ import scala.collection.mutable.SynchronizedQueue diff --git a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala similarity index 90% rename from examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala rename to examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala index 7fb680bcc31..2e3d9ccf004 100644 --- a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala @@ -15,20 +15,20 @@ * limitations under the License. */ -package spark.streaming.examples +package org.apache.spark.streaming.examples -import spark.util.IntParam -import spark.storage.StorageLevel +import org.apache.spark.util.IntParam +import org.apache.spark.storage.StorageLevel -import spark.streaming._ -import spark.streaming.util.RawTextHelper +import org.apache.spark.streaming._ +import org.apache.spark.streaming.util.RawTextHelper /** * Receives text from multiple rawNetworkStreams and counts how many '\n' delimited * lines have the word 'the' in them. This is useful for benchmarking purposes. This * will only work with spark.streaming.util.RawTextSender running on all worker nodes * and with Spark using Kryo serialization (set Java property "spark.serializer" to - * "spark.KryoSerializer"). + * "org.apache.spark.KryoSerializer"). * Usage: RawNetworkGrep * is the Spark master URL * is the number rawNetworkStreams, which should be same as number diff --git a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala similarity index 95% rename from examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala rename to examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala index b709fc3c87c..cb30c4edb30 100644 --- a/examples/src/main/scala/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.examples +package org.apache.spark.streaming.examples -import spark.streaming._ -import spark.streaming.StreamingContext._ +import org.apache.spark.streaming._ +import org.apache.spark.streaming.StreamingContext._ /** * Counts words cumulatively in UTF8 encoded, '\n' delimited text received from the network every second. diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala similarity index 94% rename from examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala rename to examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala index 8770abd57e0..35b6329ab31 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.streaming.examples +package org.apache.spark.streaming.examples -import spark.streaming.{Seconds, StreamingContext} -import spark.storage.StorageLevel +import org.apache.spark.streaming.{Seconds, StreamingContext} +import org.apache.spark.storage.StorageLevel import com.twitter.algebird._ -import spark.streaming.StreamingContext._ -import spark.SparkContext._ +import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.SparkContext._ /** * Illustrates the use of the Count-Min Sketch, from Twitter's Algebird library, to compute diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala similarity index 94% rename from examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala rename to examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala index cba5c986bea..8bfde2a8297 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.streaming.examples +package org.apache.spark.streaming.examples -import spark.streaming.{Seconds, StreamingContext} -import spark.storage.StorageLevel +import org.apache.spark.streaming.{Seconds, StreamingContext} +import org.apache.spark.storage.StorageLevel import com.twitter.algebird.HyperLogLog._ import com.twitter.algebird.HyperLogLogMonoid -import spark.streaming.dstream.TwitterInputDStream +import org.apache.spark.streaming.dstream.TwitterInputDStream /** * Illustrates the use of the HyperLogLog algorithm, from Twitter's Algebird library, to compute diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala similarity index 94% rename from examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala rename to examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala index 682b99f75e5..27aa6b14bf2 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming.examples +package org.apache.spark.streaming.examples -import spark.streaming.{Seconds, StreamingContext} +import org.apache.spark.streaming.{Seconds, StreamingContext} import StreamingContext._ -import spark.SparkContext._ +import org.apache.spark.SparkContext._ /** * Calculates popular hashtags (topics) over sliding 10 and 60 second windows from a Twitter diff --git a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala similarity index 95% rename from examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala rename to examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala index a0cae06c309..c8743b9e255 100644 --- a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.streaming.examples +package org.apache.spark.streaming.examples import akka.actor.ActorSystem import akka.actor.actorRef2Scala import akka.zeromq._ -import spark.streaming.{ Seconds, StreamingContext } -import spark.streaming.StreamingContext._ +import org.apache.spark.streaming.{ Seconds, StreamingContext } +import org.apache.spark.streaming.StreamingContext._ import akka.zeromq.Subscribe /** diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala similarity index 98% rename from examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala rename to examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala index dd36bbbf320..884d6d6f341 100644 --- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.streaming.examples.clickstream +package org.apache.spark.streaming.examples.clickstream import java.net.{InetAddress,ServerSocket,Socket,SocketException} import java.io.{InputStreamReader, BufferedReader, PrintWriter} diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala similarity index 95% rename from examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala rename to examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala index 152da234898..8282cc9269c 100644 --- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming.examples.clickstream +package org.apache.spark.streaming.examples.clickstream -import spark.streaming.{Seconds, StreamingContext} -import spark.streaming.StreamingContext._ -import spark.SparkContext._ +import org.apache.spark.streaming.{Seconds, StreamingContext} +import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.SparkContext._ /** Analyses a streaming dataset of web page views. This class demonstrates several types of * operators available in Spark streaming. diff --git a/mllib/pom.xml b/mllib/pom.xml index ab31d5734e4..2d5d3c00d10 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -19,13 +19,13 @@ 4.0.0 - org.spark-project + org.apache.spark spark-parent 0.8.0-SNAPSHOT ../pom.xml - org.spark-project + org.apache.spark spark-mllib jar Spark Project ML Library @@ -33,7 +33,7 @@ - org.spark-project + org.apache.spark spark-core ${project.version} diff --git a/mllib/src/main/scala/spark/mllib/classification/ClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala similarity index 88% rename from mllib/src/main/scala/spark/mllib/classification/ClassificationModel.scala rename to mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala index 70fae8c15a2..4f4a7f5296e 100644 --- a/mllib/src/main/scala/spark/mllib/classification/ClassificationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala @@ -1,6 +1,6 @@ -package spark.mllib.classification +package org.apache.spark.mllib.classification -import spark.RDD +import org.apache.spark.RDD trait ClassificationModel extends Serializable { /** diff --git a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala similarity index 96% rename from mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala rename to mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 482e4a67457..91bb50c829c 100644 --- a/mllib/src/main/scala/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -15,15 +15,15 @@ * limitations under the License. */ -package spark.mllib.classification +package org.apache.spark.mllib.classification import scala.math.round -import spark.{Logging, RDD, SparkContext} -import spark.mllib.optimization._ -import spark.mllib.regression._ -import spark.mllib.util.MLUtils -import spark.mllib.util.DataValidators +import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.regression._ +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.mllib.util.DataValidators import org.jblas.DoubleMatrix diff --git a/mllib/src/main/scala/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala similarity index 95% rename from mllib/src/main/scala/spark/mllib/classification/SVM.scala rename to mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index 69393cd7b07..c92c7cc3f34 100644 --- a/mllib/src/main/scala/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -15,15 +15,15 @@ * limitations under the License. */ -package spark.mllib.classification +package org.apache.spark.mllib.classification import scala.math.signum -import spark.{Logging, RDD, SparkContext} -import spark.mllib.optimization._ -import spark.mllib.regression._ -import spark.mllib.util.MLUtils -import spark.mllib.util.DataValidators +import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.regression._ +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.mllib.util.DataValidators import org.jblas.DoubleMatrix diff --git a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala similarity index 98% rename from mllib/src/main/scala/spark/mllib/clustering/KMeans.scala rename to mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 97e3d110ae5..2c3db099fa8 100644 --- a/mllib/src/main/scala/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -15,15 +15,15 @@ * limitations under the License. */ -package spark.mllib.clustering +package org.apache.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.apache.spark.{SparkContext, RDD} +import org.apache.spark.SparkContext._ +import org.apache.spark.Logging +import org.apache.spark.mllib.util.MLUtils import org.jblas.DoubleMatrix diff --git a/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala similarity index 90% rename from mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala rename to mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index b8f80e80cd7..d1fe5d138d3 100644 --- a/mllib/src/main/scala/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.mllib.clustering +package org.apache.spark.mllib.clustering -import spark.RDD -import spark.SparkContext._ -import spark.mllib.util.MLUtils +import org.apache.spark.RDD +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.util.MLUtils /** diff --git a/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala similarity index 98% rename from mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala rename to mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala index 89fe7d7e853..baf8251d8fc 100644 --- a/mllib/src/main/scala/spark/mllib/clustering/LocalKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.clustering +package org.apache.spark.mllib.clustering import scala.util.Random diff --git a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala similarity index 98% rename from mllib/src/main/scala/spark/mllib/optimization/Gradient.scala rename to mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 05568f55af0..749e7364f4d 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.optimization +package org.apache.spark.mllib.optimization import org.jblas.DoubleMatrix diff --git a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala similarity index 97% rename from mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala rename to mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index 31917df7e87..b62c9b3340e 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.mllib.optimization +package org.apache.spark.mllib.optimization -import spark.{Logging, RDD, SparkContext} -import spark.SparkContext._ +import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.SparkContext._ import org.jblas.DoubleMatrix diff --git a/mllib/src/main/scala/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala similarity index 93% rename from mllib/src/main/scala/spark/mllib/optimization/Optimizer.scala rename to mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala index 76a519c3388..50059d385d2 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Optimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.mllib.optimization +package org.apache.spark.mllib.optimization -import spark.RDD +import org.apache.spark.RDD trait Optimizer { diff --git a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala similarity index 98% rename from mllib/src/main/scala/spark/mllib/optimization/Updater.scala rename to mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala index db67d6b0bcd..4c51f4f881f 100644 --- a/mllib/src/main/scala/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.optimization +package org.apache.spark.mllib.optimization import scala.math._ import org.jblas.DoubleMatrix diff --git a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala similarity index 98% rename from mllib/src/main/scala/spark/mllib/recommendation/ALS.scala rename to mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index dbfbf599754..218217acfe5 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -15,16 +15,16 @@ * limitations under the License. */ -package spark.mllib.recommendation +package org.apache.spark.mllib.recommendation import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.util.Random import scala.util.Sorting -import spark.{HashPartitioner, Partitioner, SparkContext, RDD} -import spark.storage.StorageLevel -import spark.KryoRegistrator -import spark.SparkContext._ +import org.apache.spark.{HashPartitioner, Partitioner, SparkContext, RDD} +import org.apache.spark.storage.StorageLevel +import org.apache.spark.KryoRegistrator +import org.apache.spark.SparkContext._ import com.esotericsoftware.kryo.Kryo import org.jblas.{DoubleMatrix, SimpleBlas, Solve} @@ -432,7 +432,7 @@ object ALS { 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.serializer", "org.apache.spark.KryoSerializer") System.setProperty("spark.kryo.registrator", classOf[ALSRegistrator].getName) System.setProperty("spark.kryo.referenceTracking", "false") System.setProperty("spark.kryoserializer.buffer.mb", "8") diff --git a/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala similarity index 94% rename from mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala rename to mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 5e21717da52..ae9fe48aec3 100644 --- a/mllib/src/main/scala/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.mllib.recommendation +package org.apache.spark.mllib.recommendation -import spark.RDD -import spark.SparkContext._ +import org.apache.spark.RDD +import org.apache.spark.SparkContext._ import org.jblas._ diff --git a/mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala similarity index 96% rename from mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala rename to mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index d164d415d6a..06015110ac0 100644 --- a/mllib/src/main/scala/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.mllib.regression +package org.apache.spark.mllib.regression -import spark.{Logging, RDD, SparkException} -import spark.mllib.optimization._ +import org.apache.spark.{Logging, RDD, SparkException} +import org.apache.spark.mllib.optimization._ import org.jblas.DoubleMatrix @@ -52,7 +52,7 @@ abstract class GeneralizedLinearModel(val weights: Array[Double], val intercept: * @param testData RDD representing data points to be predicted * @return RDD[Double] where each entry contains the corresponding prediction */ - def predict(testData: spark.RDD[Array[Double]]): RDD[Double] = { + def predict(testData: RDD[Array[Double]]): RDD[Double] = { // A small optimization to avoid serializing the entire model. Only the weightsMatrix // and intercept is needed. val localWeights = weightsMatrix diff --git a/mllib/src/main/scala/spark/mllib/regression/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala similarity index 96% rename from mllib/src/main/scala/spark/mllib/regression/LabeledPoint.scala rename to mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala index 3de60482c54..63240e24dc2 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LabeledPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.regression +package org.apache.spark.mllib.regression /** * Class that represents the features and labels of a data point. diff --git a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala similarity index 97% rename from mllib/src/main/scala/spark/mllib/regression/Lasso.scala rename to mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index 0f33456ef43..df3beb1959d 100644 --- a/mllib/src/main/scala/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.mllib.regression +package org.apache.spark.mllib.regression -import spark.{Logging, RDD, SparkContext} -import spark.mllib.optimization._ -import spark.mllib.util.MLUtils +import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.util.MLUtils import org.jblas.DoubleMatrix diff --git a/mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala similarity index 97% rename from mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala rename to mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 885ff5a30d4..71f968471cd 100644 --- a/mllib/src/main/scala/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.mllib.regression +package org.apache.spark.mllib.regression -import spark.{Logging, RDD, SparkContext} -import spark.mllib.optimization._ -import spark.mllib.util.MLUtils +import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.util.MLUtils import org.jblas.DoubleMatrix diff --git a/mllib/src/main/scala/spark/mllib/regression/RegressionModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala similarity index 95% rename from mllib/src/main/scala/spark/mllib/regression/RegressionModel.scala rename to mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala index b845ba1a890..8dd325efc08 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RegressionModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.mllib.regression +package org.apache.spark.mllib.regression -import spark.RDD +import org.apache.spark.RDD trait RegressionModel extends Serializable { /** diff --git a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala similarity index 97% rename from mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala rename to mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index cb1303dd99e..228ab9e4e8f 100644 --- a/mllib/src/main/scala/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.mllib.regression +package org.apache.spark.mllib.regression -import spark.{Logging, RDD, SparkContext} -import spark.mllib.optimization._ -import spark.mllib.util.MLUtils +import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.util.MLUtils import org.jblas.DoubleMatrix diff --git a/mllib/src/main/scala/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala similarity index 91% rename from mllib/src/main/scala/spark/mllib/util/DataValidators.scala rename to mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala index 57553accf1f..7fd46230716 100644 --- a/mllib/src/main/scala/spark/mllib/util/DataValidators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.mllib.util +package org.apache.spark.mllib.util -import spark.{RDD, Logging} -import spark.mllib.regression.LabeledPoint +import org.apache.spark.{RDD, Logging} +import org.apache.spark.mllib.regression.LabeledPoint /** * A collection of methods used to validate data before applying ML algorithms. diff --git a/mllib/src/main/scala/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala similarity index 97% rename from mllib/src/main/scala/spark/mllib/util/KMeansDataGenerator.scala rename to mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala index 672b63f65ab..6500d47183e 100644 --- a/mllib/src/main/scala/spark/mllib/util/KMeansDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.mllib.util +package org.apache.spark.mllib.util import scala.util.Random -import spark.{RDD, SparkContext} +import org.apache.spark.{RDD, SparkContext} /** * Generate test data for KMeans. This class first chooses k cluster centers diff --git a/mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala similarity index 95% rename from mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala rename to mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 9f48477f84a..4c49d484b4c 100644 --- a/mllib/src/main/scala/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -15,16 +15,16 @@ * limitations under the License. */ -package spark.mllib.util +package org.apache.spark.mllib.util import scala.collection.JavaConversions._ import scala.util.Random import org.jblas.DoubleMatrix -import spark.{RDD, SparkContext} -import spark.mllib.regression.LabeledPoint -import spark.mllib.regression.LabeledPoint +import org.apache.spark.{RDD, SparkContext} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.LabeledPoint /** * Generate sample data used for Linear Data. This class generates diff --git a/mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala similarity index 95% rename from mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala rename to mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala index d6402f23e2d..f553298fc5c 100644 --- a/mllib/src/main/scala/spark/mllib/util/LogisticRegressionDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.mllib.util +package org.apache.spark.mllib.util import scala.util.Random -import spark.{RDD, SparkContext} -import spark.mllib.regression.LabeledPoint +import org.apache.spark.{RDD, SparkContext} +import org.apache.spark.mllib.regression.LabeledPoint /** * Generate test data for LogisticRegression. This class chooses positive labels diff --git a/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala similarity index 97% rename from mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala rename to mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala index 88992cde0cf..7eb69ae81cb 100644 --- a/mllib/src/main/scala/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.mllib.recommendation +package org.apache.spark.mllib.recommendation import scala.util.Random import org.jblas.DoubleMatrix -import spark.{RDD, SparkContext} -import spark.mllib.util.MLUtils +import org.apache.spark.{RDD, SparkContext} +import org.apache.spark.mllib.util.MLUtils /** * Generate RDD(s) containing data for Matrix Factorization. @@ -110,4 +110,4 @@ object MFDataGenerator{ sc.stop() } -} \ No newline at end of file +} diff --git a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala similarity index 96% rename from mllib/src/main/scala/spark/mllib/util/MLUtils.scala rename to mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index a8e6ae9953d..0aeafbe23c7 100644 --- a/mllib/src/main/scala/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.mllib.util +package org.apache.spark.mllib.util -import spark.{RDD, SparkContext} -import spark.SparkContext._ +import org.apache.spark.{RDD, SparkContext} +import org.apache.spark.SparkContext._ import org.jblas.DoubleMatrix -import spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.LabeledPoint /** * Helper methods to load, save and pre-process data used in ML Lib. diff --git a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala similarity index 91% rename from mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala rename to mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index eff456cad63..d3f191b05b1 100644 --- a/mllib/src/main/scala/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -1,11 +1,11 @@ -package spark.mllib.util +package org.apache.spark.mllib.util import scala.util.Random -import spark.{RDD, SparkContext} +import org.apache.spark.{RDD, SparkContext} import org.jblas.DoubleMatrix -import spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.LabeledPoint /** * Generate sample data used for SVM. This class generates uniform random values diff --git a/mllib/src/test/java/spark/mllib/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java similarity index 93% rename from mllib/src/test/java/spark/mllib/classification/JavaLogisticRegressionSuite.java rename to mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java index e0ebd45cd8d..e18e3bc6a86 100644 --- a/mllib/src/test/java/spark/mllib/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.classification; +package org.apache.spark.mllib.classification; import java.io.Serializable; import java.util.List; @@ -25,10 +25,10 @@ import org.junit.Before; import org.junit.Test; -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; -import spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.regression.LabeledPoint; public class JavaLogisticRegressionSuite implements Serializable { private transient JavaSparkContext sc; diff --git a/mllib/src/test/java/spark/mllib/classification/JavaSVMSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java similarity index 93% rename from mllib/src/test/java/spark/mllib/classification/JavaSVMSuite.java rename to mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java index 7881b3c38f0..117e5eaa8b7 100644 --- a/mllib/src/test/java/spark/mllib/classification/JavaSVMSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.classification; +package org.apache.spark.mllib.classification; import java.io.Serializable; @@ -26,10 +26,10 @@ import org.junit.Before; import org.junit.Test; -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; -import spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.regression.LabeledPoint; public class JavaSVMSuite implements Serializable { private transient JavaSparkContext sc; diff --git a/mllib/src/test/java/spark/mllib/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java similarity index 96% rename from mllib/src/test/java/spark/mllib/clustering/JavaKMeansSuite.java rename to mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java index 3f2d82bfb4a..32d3934ac13 100644 --- a/mllib/src/test/java/spark/mllib/clustering/JavaKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.clustering; +package org.apache.spark.mllib.clustering; import java.io.Serializable; import java.util.ArrayList; @@ -26,8 +26,8 @@ import org.junit.Before; import org.junit.Test; -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; public class JavaKMeansSuite implements Serializable { private transient JavaSparkContext sc; diff --git a/mllib/src/test/java/spark/mllib/recommendation/JavaALSSuite.java b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java similarity index 96% rename from mllib/src/test/java/spark/mllib/recommendation/JavaALSSuite.java rename to mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java index 7993629a6d5..3323f6cee2b 100644 --- a/mllib/src/test/java/spark/mllib/recommendation/JavaALSSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.recommendation; +package org.apache.spark.mllib.recommendation; import java.io.Serializable; import java.util.List; @@ -27,8 +27,8 @@ import org.junit.Before; import org.junit.Test; -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; import org.jblas.DoubleMatrix; diff --git a/mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java similarity index 94% rename from mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java rename to mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java index 5863140bafe..f44b25cd44d 100644 --- a/mllib/src/test/java/spark/mllib/regression/JavaLassoSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.regression; +package org.apache.spark.mllib.regression; import java.io.Serializable; import java.util.List; @@ -25,9 +25,9 @@ import org.junit.Before; import org.junit.Test; -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; -import spark.mllib.util.LinearDataGenerator; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.util.LinearDataGenerator; public class JavaLassoSuite implements Serializable { private transient JavaSparkContext sc; diff --git a/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java similarity index 94% rename from mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java rename to mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java index 50716c78617..5a4410a6326 100644 --- a/mllib/src/test/java/spark/mllib/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.regression; +package org.apache.spark.mllib.regression; import java.io.Serializable; import java.util.List; @@ -25,9 +25,9 @@ import org.junit.Before; import org.junit.Test; -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; -import spark.mllib.util.LinearDataGenerator; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.util.LinearDataGenerator; public class JavaLinearRegressionSuite implements Serializable { private transient JavaSparkContext sc; diff --git a/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java similarity index 95% rename from mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java rename to mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java index 2c0aabad309..2fdd5fc8fdc 100644 --- a/mllib/src/test/java/spark/mllib/regression/JavaRidgeRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.regression; +package org.apache.spark.mllib.regression; import java.io.Serializable; import java.util.List; @@ -27,9 +27,9 @@ import org.jblas.DoubleMatrix; -import spark.api.java.JavaRDD; -import spark.api.java.JavaSparkContext; -import spark.mllib.util.LinearDataGenerator; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.util.LinearDataGenerator; public class JavaRidgeRegressionSuite implements Serializable { private transient JavaSparkContext sc; diff --git a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala similarity index 97% rename from mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index bd87c528c35..34c67294e9a 100644 --- a/mllib/src/test/scala/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.classification +package org.apache.spark.mllib.classification import scala.util.Random import scala.collection.JavaConversions._ @@ -24,8 +24,8 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers -import spark.SparkContext -import spark.mllib.regression._ +import org.apache.spark.SparkContext +import org.apache.spark.mllib.regression._ object LogisticRegressionSuite { diff --git a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala similarity index 96% rename from mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala index 894ae458ad0..6a957e3ddca 100644 --- a/mllib/src/test/scala/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.classification +package org.apache.spark.mllib.classification import scala.util.Random import scala.math.signum @@ -24,11 +24,11 @@ import scala.collection.JavaConversions._ import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite -import spark.SparkContext -import spark.mllib.regression._ - import org.jblas.DoubleMatrix +import org.apache.spark.{SparkException, SparkContext} +import org.apache.spark.mllib.regression._ + object SVMSuite { def generateSVMInputAsList( @@ -159,7 +159,7 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll { } } - intercept[spark.SparkException] { + intercept[SparkException] { val model = SVMWithSGD.train(testRDDInvalid, 100) } diff --git a/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala similarity index 98% rename from mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index d5d95c86393..94245f6027b 100644 --- a/mllib/src/test/scala/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -15,15 +15,15 @@ * limitations under the License. */ -package spark.mllib.clustering +package org.apache.spark.mllib.clustering import scala.util.Random import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite -import spark.SparkContext -import spark.SparkContext._ +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ import org.jblas._ diff --git a/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala similarity index 97% rename from mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala index 15a60efda6e..347ef238f40 100644 --- a/mllib/src/test/scala/spark/mllib/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.recommendation +package org.apache.spark.mllib.recommendation import scala.collection.JavaConversions._ import scala.util.Random @@ -23,8 +23,8 @@ import scala.util.Random import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite -import spark.SparkContext -import spark.SparkContext._ +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ import org.jblas._ diff --git a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala similarity index 96% rename from mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala index 622dbbab7fe..db980c7bae6 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.regression +package org.apache.spark.mllib.regression import scala.collection.JavaConversions._ import scala.util.Random @@ -23,8 +23,8 @@ import scala.util.Random import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite -import spark.SparkContext -import spark.mllib.util.LinearDataGenerator +import org.apache.spark.SparkContext +import org.apache.spark.mllib.util.LinearDataGenerator class LassoSuite extends FunSuite with BeforeAndAfterAll { diff --git a/mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala similarity index 93% rename from mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala index acc48a3283c..ef500c704c8 100644 --- a/mllib/src/test/scala/spark/mllib/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.mllib.regression +package org.apache.spark.mllib.regression import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite -import spark.SparkContext -import spark.SparkContext._ -import spark.mllib.util.LinearDataGenerator +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.util.LinearDataGenerator class LinearRegressionSuite extends FunSuite with BeforeAndAfterAll { @transient private var sc: SparkContext = _ diff --git a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala similarity index 94% rename from mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala index c4820357062..c18092d804f 100644 --- a/mllib/src/test/scala/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.mllib.regression +package org.apache.spark.mllib.regression import scala.collection.JavaConversions._ import scala.util.Random @@ -24,9 +24,9 @@ import org.jblas.DoubleMatrix import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite -import spark.SparkContext -import spark.SparkContext._ -import spark.mllib.util.LinearDataGenerator +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.util.LinearDataGenerator class RidgeRegressionSuite extends FunSuite with BeforeAndAfterAll { @transient private var sc: SparkContext = _ diff --git a/pom.xml b/pom.xml index e2fd54a9660..9230611eaea 100644 --- a/pom.xml +++ b/pom.xml @@ -18,22 +18,22 @@ 4.0.0 - org.spark-project + org.apache.spark spark-parent 0.8.0-SNAPSHOT pom Spark Project Parent POM - http://spark-project.org/ + http://spark.incubator.apache.org/ - BSD License - https://github.com/mesos/spark/blob/master/LICENSE + Apache 2.0 License + http://www.apache.org/licenses/LICENSE-2.0.html repo - scm:git:git@github.com:mesos/spark.git - scm:git:git@github.com:mesos/spark.git + scm:git:git@github.com:apache/incubator-spark.git + scm:git:git@github.com:apache/incubator-spark.git @@ -46,7 +46,7 @@ - github + JIRA https://spark-project.atlassian.net/browse/SPARK diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 2e268126713..18e86d2cae7 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -74,7 +74,7 @@ object SparkBuild extends Build { core, repl, examples, bagel, streaming, mllib, tools, assemblyProj) ++ maybeYarnRef def sharedSettings = Defaults.defaultSettings ++ Seq( - organization := "org.spark-project", + organization := "org.apache.spark", version := "0.8.0-SNAPSHOT", scalaVersion := "2.9.3", scalacOptions := Seq("-unchecked", "-optimize", "-deprecation"), @@ -103,7 +103,7 @@ object SparkBuild extends Build { //useGpg in Global := true, pomExtra := ( - http://spark-project.org/ + http://spark.incubator.apache.org/ Apache 2.0 License @@ -112,8 +112,8 @@ object SparkBuild extends Build { - scm:git:git@github.com:mesos/spark.git - scm:git:git@github.com:mesos/spark.git + scm:git:git@github.com:apache/incubator-spark.git + scm:git:git@github.com:apache/incubator-spark.git @@ -125,6 +125,10 @@ object SparkBuild extends Build { http://www.cs.berkeley.edu/ + + JIRA + https://spark-project.atlassian.net/browse/SPARK + ), /* diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 2803ce90f3e..906e9221a1d 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -114,9 +114,9 @@ def __init__(self, master, jobName, sparkHome=None, pyFiles=None, self.addPyFile(path) # Create a temporary directory inside spark.local.dir: - local_dir = self._jvm.spark.Utils.getLocalDir() + local_dir = self._jvm.org.apache.spark.Utils.getLocalDir() self._temp_dir = \ - self._jvm.spark.Utils.createTempDir(local_dir).getAbsolutePath() + self._jvm.org.apache.spark.Utils.createTempDir(local_dir).getAbsolutePath() @property def defaultParallelism(self): diff --git a/python/pyspark/files.py b/python/pyspark/files.py index 89bcbcfe068..57ee14eeb77 100644 --- a/python/pyspark/files.py +++ b/python/pyspark/files.py @@ -52,4 +52,4 @@ def getRootDirectory(cls): return cls._root_directory else: # This will have to change if we support multiple SparkContexts: - return cls._sc._jvm.spark.SparkFiles.getRootDirectory() + return cls._sc._jvm.org.apache.spark.SparkFiles.getRootDirectory() diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 3ccf062c86c..26fbe0f0804 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -53,7 +53,7 @@ def run(self): # Connect to the gateway gateway = JavaGateway(GatewayClient(port=port), auto_convert=False) # Import the classes used by PySpark - java_import(gateway.jvm, "spark.api.java.*") - java_import(gateway.jvm, "spark.api.python.*") + java_import(gateway.jvm, "org.apache.spark.api.java.*") + java_import(gateway.jvm, "org.apache.spark.api.python.*") java_import(gateway.jvm, "scala.Tuple2") return gateway diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 919e35f2404..6a1b09e8df7 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -19,13 +19,13 @@ 4.0.0 - org.spark-project + org.apache.spark spark-parent 0.8.0-SNAPSHOT ../pom.xml - org.spark-project + org.apache.spark spark-repl-bin pom Spark Project REPL binary packaging @@ -39,18 +39,18 @@ - org.spark-project + org.apache.spark spark-core ${project.version} - org.spark-project + org.apache.spark spark-bagel ${project.version} runtime - org.spark-project + org.apache.spark spark-repl ${project.version} runtime @@ -109,7 +109,7 @@ hadoop2-yarn - org.spark-project + org.apache.spark spark-yarn ${project.version} diff --git a/repl/pom.xml b/repl/pom.xml index f800664cff6..f6276f18958 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -19,13 +19,13 @@ 4.0.0 - org.spark-project + org.apache.spark spark-parent 0.8.0-SNAPSHOT ../pom.xml - org.spark-project + org.apache.spark spark-repl jar Spark Project REPL @@ -38,18 +38,18 @@ - org.spark-project + org.apache.spark spark-core ${project.version} - org.spark-project + org.apache.spark spark-bagel ${project.version} runtime - org.spark-project + org.apache.spark spark-mllib ${project.version} runtime @@ -136,7 +136,7 @@ hadoop2-yarn - org.spark-project + org.apache.spark spark-yarn ${project.version} diff --git a/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala similarity index 99% rename from repl/src/main/scala/spark/repl/ExecutorClassLoader.scala rename to repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala index 274bc585db9..3e171849e34 100644 --- a/repl/src/main/scala/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.repl +package org.apache.spark.repl import java.io.{ByteArrayOutputStream, InputStream} import java.net.{URI, URL, URLClassLoader, URLEncoder} diff --git a/repl/src/main/scala/spark/repl/Main.scala b/repl/src/main/scala/org/apache/spark/repl/Main.scala similarity index 97% rename from repl/src/main/scala/spark/repl/Main.scala rename to repl/src/main/scala/org/apache/spark/repl/Main.scala index d824d62fd10..17e149f8abc 100644 --- a/repl/src/main/scala/spark/repl/Main.scala +++ b/repl/src/main/scala/org/apache/spark/repl/Main.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.repl +package org.apache.spark.repl import scala.collection.mutable.Set diff --git a/repl/src/main/scala/spark/repl/SparkHelper.scala b/repl/src/main/scala/org/apache/spark/repl/SparkHelper.scala similarity index 100% rename from repl/src/main/scala/spark/repl/SparkHelper.scala rename to repl/src/main/scala/org/apache/spark/repl/SparkHelper.scala diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala similarity index 98% rename from repl/src/main/scala/spark/repl/SparkILoop.scala rename to repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 0cecbd71adc..193ccb48eeb 100644 --- a/repl/src/main/scala/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -3,7 +3,7 @@ * @author Alexander Spoon */ -package spark.repl +package org.apache.spark.repl import scala.tools.nsc._ import scala.tools.nsc.interpreter._ @@ -22,8 +22,8 @@ import util.{ ClassPath, Exceptional, stringFromWriter, stringFromStream } import interpreter._ import io.{ File, Sources } -import spark.Logging -import spark.SparkContext +import org.apache.spark.Logging +import org.apache.spark.SparkContext /** The Scala interactive shell. It provides a read-eval-print loop * around the Interpreter class. @@ -816,13 +816,13 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: def initializeSpark() { intp.beQuietDuring { command(""" - spark.repl.Main.interp.out.println("Creating SparkContext..."); - spark.repl.Main.interp.out.flush(); - @transient val sc = spark.repl.Main.interp.createSparkContext(); - spark.repl.Main.interp.out.println("Spark context available as sc."); - spark.repl.Main.interp.out.flush(); + org.apache.spark.repl.Main.interp.out.println("Creating SparkContext..."); + org.apache.spark.repl.Main.interp.out.flush(); + @transient val sc = org.apache.spark.repl.Main.interp.createSparkContext(); + org.apache.spark.repl.Main.interp.out.println("Spark context available as sc."); + org.apache.spark.repl.Main.interp.out.flush(); """) - command("import spark.SparkContext._") + command("import org.apache.spark.SparkContext._") } echo("Type in expressions to have them evaluated.") echo("Type :help for more information.") diff --git a/repl/src/main/scala/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala similarity index 99% rename from repl/src/main/scala/spark/repl/SparkIMain.scala rename to repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala index 43b6a6c9500..7e244e48a2a 100644 --- a/repl/src/main/scala/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -3,7 +3,7 @@ * @author Martin Odersky */ -package spark.repl +package org.apache.spark.repl import scala.tools.nsc._ import scala.tools.nsc.interpreter._ @@ -27,9 +27,9 @@ import scala.util.control.Exception.{ ultimately } import scala.reflect.NameTransformer import SparkIMain._ -import spark.HttpServer -import spark.Utils -import spark.SparkEnv +import org.apache.spark.HttpServer +import org.apache.spark.Utils +import org.apache.spark.SparkEnv /** An interpreter for Scala code. * @@ -883,7 +883,7 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends val execution = lineManager.set(originalLine) { // MATEI: set the right SparkEnv for our SparkContext, because // this execution will happen in a separate thread - val sc = spark.repl.Main.interp.sparkContext + val sc = org.apache.spark.repl.Main.interp.sparkContext if (sc != null && sc.env != null) SparkEnv.set(sc.env) // Execute the line diff --git a/repl/src/main/scala/spark/repl/SparkISettings.scala b/repl/src/main/scala/org/apache/spark/repl/SparkISettings.scala similarity index 98% rename from repl/src/main/scala/spark/repl/SparkISettings.scala rename to repl/src/main/scala/org/apache/spark/repl/SparkISettings.scala index 8ebb01d1460..605b7b259b5 100644 --- a/repl/src/main/scala/spark/repl/SparkISettings.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkISettings.scala @@ -3,7 +3,7 @@ * @author Alexander Spoon */ -package spark.repl +package org.apache.spark.repl import scala.tools.nsc._ import scala.tools.nsc.interpreter._ diff --git a/repl/src/main/scala/spark/repl/SparkImports.scala b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala similarity index 99% rename from repl/src/main/scala/spark/repl/SparkImports.scala rename to repl/src/main/scala/org/apache/spark/repl/SparkImports.scala index 5caf5ca51a4..41a1731d600 100644 --- a/repl/src/main/scala/spark/repl/SparkImports.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala @@ -3,7 +3,7 @@ * @author Paul Phillips */ -package spark.repl +package org.apache.spark.repl import scala.tools.nsc._ import scala.tools.nsc.interpreter._ diff --git a/repl/src/main/scala/spark/repl/SparkJLineCompletion.scala b/repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala similarity index 99% rename from repl/src/main/scala/spark/repl/SparkJLineCompletion.scala rename to repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala index 0069d8b2f49..fdc172d7530 100644 --- a/repl/src/main/scala/spark/repl/SparkJLineCompletion.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala @@ -3,7 +3,7 @@ * @author Paul Phillips */ -package spark.repl +package org.apache.spark.repl import scala.tools.nsc._ import scala.tools.nsc.interpreter._ diff --git a/repl/src/main/scala/spark/repl/SparkJLineReader.scala b/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala similarity index 98% rename from repl/src/main/scala/spark/repl/SparkJLineReader.scala rename to repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala index ef6b6e092e0..d9e1de105c2 100644 --- a/repl/src/main/scala/spark/repl/SparkJLineReader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala @@ -3,7 +3,7 @@ * @author Stepan Koltsov */ -package spark.repl +package org.apache.spark.repl import scala.tools.nsc._ import scala.tools.nsc.interpreter._ diff --git a/repl/src/main/scala/spark/repl/SparkMemberHandlers.scala b/repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala similarity index 99% rename from repl/src/main/scala/spark/repl/SparkMemberHandlers.scala rename to repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala index 2980dfcd768..a3409bf6654 100644 --- a/repl/src/main/scala/spark/repl/SparkMemberHandlers.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala @@ -3,7 +3,7 @@ * @author Martin Odersky */ -package spark.repl +package org.apache.spark.repl import scala.tools.nsc._ import scala.tools.nsc.interpreter._ diff --git a/repl/src/test/scala/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala similarity index 98% rename from repl/src/test/scala/spark/repl/ReplSuite.scala rename to repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 80ae6055580..8f9b632c0ee 100644 --- a/repl/src/test/scala/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.repl +package org.apache.spark.repl import java.io._ import java.net.URLClassLoader @@ -41,10 +41,10 @@ class ReplSuite extends FunSuite { } } val interp = new SparkILoop(in, new PrintWriter(out), master) - spark.repl.Main.interp = interp + org.apache.spark.repl.Main.interp = interp val separator = System.getProperty("path.separator") interp.process(Array("-classpath", paths.mkString(separator))) - spark.repl.Main.interp = null + org.apache.spark.repl.Main.interp = null if (interp.sparkContext != null) { interp.sparkContext.stop() } diff --git a/spark-executor b/spark-executor index 63692bd46c9..2c07c548433 100755 --- a/spark-executor +++ b/spark-executor @@ -19,4 +19,4 @@ FWDIR="`dirname $0`" echo "Running spark-executor with framework dir = $FWDIR" -exec $FWDIR/spark-class spark.executor.MesosExecutorBackend +exec $FWDIR/spark-class org.apache.spark.executor.MesosExecutorBackend diff --git a/spark-shell b/spark-shell index 4d379c5cfb9..9608bd3f30b 100755 --- a/spark-shell +++ b/spark-shell @@ -79,7 +79,7 @@ if [[ ! $? ]]; then saved_stty="" fi -$FWDIR/spark-class $OPTIONS spark.repl.Main "$@" +$FWDIR/spark-class $OPTIONS org.apache.spark.repl.Main "$@" # record the exit status lest it be overwritten: # then reenable echo and propagate the code. diff --git a/spark-shell.cmd b/spark-shell.cmd index ec65eabb741..b9b4d4bfb2b 100644 --- a/spark-shell.cmd +++ b/spark-shell.cmd @@ -19,4 +19,4 @@ rem set FWDIR=%~dp0 set SPARK_LAUNCH_WITH_SCALA=1 -cmd /V /E /C %FWDIR%run2.cmd spark.repl.Main %* +cmd /V /E /C %FWDIR%run2.cmd org.apache.spark.repl.Main %* diff --git a/streaming/pom.xml b/streaming/pom.xml index 5c0582d6fb6..c8946313ee9 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -19,13 +19,13 @@ 4.0.0 - org.spark-project + org.apache.spark spark-parent 0.8.0-SNAPSHOT ../pom.xml - org.spark-project + org.apache.spark spark-streaming jar Spark Project Streaming @@ -41,7 +41,7 @@ - org.spark-project + org.apache.spark spark-core ${project.version} diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala similarity index 98% rename from streaming/src/main/scala/spark/streaming/Checkpoint.scala rename to streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 070d930b5eb..2d8f0726242 100644 --- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming import java.io._ import java.util.concurrent.Executors @@ -24,8 +24,8 @@ import java.util.concurrent.RejectedExecutionException import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration -import spark.Logging -import spark.io.CompressionCodec +import org.apache.spark.Logging +import org.apache.spark.io.CompressionCodec private[streaming] diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala similarity index 98% rename from streaming/src/main/scala/spark/streaming/DStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/DStream.scala index 684d3abb564..362247cc381 100644 --- a/streaming/src/main/scala/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala @@ -15,14 +15,16 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming -import spark.streaming.dstream._ +import org.apache.spark.streaming.dstream._ import StreamingContext._ +import org.apache.spark.util.MetadataCleaner + //import Time._ -import spark.{RDD, Logging} -import spark.storage.StorageLevel +import org.apache.spark.{RDD, Logging} +import org.apache.spark.storage.StorageLevel import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap @@ -34,7 +36,7 @@ import org.apache.hadoop.conf.Configuration /** * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous - * sequence of RDDs (of the same type) representing a continuous stream of data (see [[spark.RDD]] + * sequence of RDDs (of the same type) representing a continuous stream of data (see [[org.apache.spark.RDD]] * for more details on RDDs). DStreams can either be created from live data (such as, data from * HDFS, Kafka or Flume) or it can be generated by transformation existing DStreams using operations * such as `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each @@ -42,7 +44,7 @@ import org.apache.hadoop.conf.Configuration * by a parent DStream. * * This class contains the basic operations available on all DStreams, such as `map`, `filter` and - * `window`. In addition, [[spark.streaming.PairDStreamFunctions]] contains operations available + * `window`. In addition, [[org.apache.spark.streaming.PairDStreamFunctions]] contains operations available * only on DStreams of key-value pairs, such as `groupByKeyAndWindow` and `join`. These operations * are automatically available on any DStream of the right type (e.g., DStream[(Int, Int)] through * implicit conversions when `spark.streaming.StreamingContext._` is imported. @@ -209,7 +211,7 @@ abstract class DStream[T: ClassManifest] ( checkpointDuration + "). Please set it to higher than " + checkpointDuration + "." ) - val metadataCleanerDelay = spark.util.MetadataCleaner.getDelaySeconds + val metadataCleanerDelay = MetadataCleaner.getDelaySeconds logInfo("metadataCleanupDelay = " + metadataCleanerDelay) assert( metadataCleanerDelay < 0 || rememberDuration.milliseconds < metadataCleanerDelay * 1000, diff --git a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala similarity index 98% rename from streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala rename to streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala index 399ca1c63d8..58a0da28705 100644 --- a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming import org.apache.hadoop.fs.Path import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.conf.Configuration import collection.mutable.HashMap -import spark.Logging +import org.apache.spark.Logging diff --git a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala similarity index 98% rename from streaming/src/main/scala/spark/streaming/DStreamGraph.scala rename to streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index c09a332d44c..b9a58fded67 100644 --- a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming import dstream.InputDStream import java.io.{ObjectInputStream, IOException, ObjectOutputStream} import collection.mutable.ArrayBuffer -import spark.Logging +import org.apache.spark.Logging final private[streaming] class DStreamGraph extends Serializable with Logging { initLogging() diff --git a/streaming/src/main/scala/spark/streaming/Duration.scala b/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala similarity index 86% rename from streaming/src/main/scala/spark/streaming/Duration.scala rename to streaming/src/main/scala/org/apache/spark/streaming/Duration.scala index 12a14e233d9..290ad378123 100644 --- a/streaming/src/main/scala/spark/streaming/Duration.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming -import spark.Utils +import org.apache.spark.Utils case class Duration (private val millis: Long) { @@ -57,7 +57,7 @@ case class Duration (private val millis: Long) { } /** - * Helper object that creates instance of [[spark.streaming.Duration]] representing + * Helper object that creates instance of [[org.apache.spark.streaming.Duration]] representing * a given number of milliseconds. */ object Milliseconds { @@ -65,7 +65,7 @@ object Milliseconds { } /** - * Helper object that creates instance of [[spark.streaming.Duration]] representing + * Helper object that creates instance of [[org.apache.spark.streaming.Duration]] representing * a given number of seconds. */ object Seconds { @@ -73,7 +73,7 @@ object Seconds { } /** - * Helper object that creates instance of [[spark.streaming.Duration]] representing + * Helper object that creates instance of [[org.apache.spark.streaming.Duration]] representing * a given number of minutes. */ object Minutes { diff --git a/streaming/src/main/scala/spark/streaming/Interval.scala b/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala similarity index 98% rename from streaming/src/main/scala/spark/streaming/Interval.scala rename to streaming/src/main/scala/org/apache/spark/streaming/Interval.scala index b30cd969e9f..04c994c1369 100644 --- a/streaming/src/main/scala/spark/streaming/Interval.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming private[streaming] class Interval(val beginTime: Time, val endTime: Time) { diff --git a/streaming/src/main/scala/spark/streaming/Job.scala b/streaming/src/main/scala/org/apache/spark/streaming/Job.scala similarity index 97% rename from streaming/src/main/scala/spark/streaming/Job.scala rename to streaming/src/main/scala/org/apache/spark/streaming/Job.scala index ceb3f92b65b..2128b7c7a64 100644 --- a/streaming/src/main/scala/spark/streaming/Job.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Job.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming import java.util.concurrent.atomic.AtomicLong diff --git a/streaming/src/main/scala/spark/streaming/JobManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/JobManager.scala similarity index 96% rename from streaming/src/main/scala/spark/streaming/JobManager.scala rename to streaming/src/main/scala/org/apache/spark/streaming/JobManager.scala index a31230689f9..5233129506f 100644 --- a/streaming/src/main/scala/spark/streaming/JobManager.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/JobManager.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming -import spark.Logging -import spark.SparkEnv +import org.apache.spark.Logging +import org.apache.spark.SparkEnv import java.util.concurrent.Executors import collection.mutable.HashMap import collection.mutable.ArrayBuffer diff --git a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala similarity index 95% rename from streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala rename to streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala index d4cf2e568ca..aae79a4e6fe 100644 --- a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming -import spark.streaming.dstream.{NetworkInputDStream, NetworkReceiver} -import spark.streaming.dstream.{StopReceiver, ReportBlock, ReportError} -import spark.Logging -import spark.SparkEnv -import spark.SparkContext._ +import org.apache.spark.streaming.dstream.{NetworkInputDStream, NetworkReceiver} +import org.apache.spark.streaming.dstream.{StopReceiver, ReportBlock, ReportError} +import org.apache.spark.Logging +import org.apache.spark.SparkEnv +import org.apache.spark.SparkContext._ import scala.collection.mutable.HashMap import scala.collection.mutable.Queue diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala similarity index 96% rename from streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala rename to streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala index 47bf07bee13..d8a7381e87d 100644 --- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala @@ -15,16 +15,16 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming -import spark.streaming.StreamingContext._ -import spark.streaming.dstream.{ReducedWindowedDStream, StateDStream} -import spark.streaming.dstream.{CoGroupedDStream, ShuffledDStream} -import spark.streaming.dstream.{MapValuedDStream, FlatMapValuedDStream} +import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.streaming.dstream.{ReducedWindowedDStream, StateDStream} +import org.apache.spark.streaming.dstream.{CoGroupedDStream, ShuffledDStream} +import org.apache.spark.streaming.dstream.{MapValuedDStream, FlatMapValuedDStream} -import spark.{Manifests, RDD, Partitioner, HashPartitioner} -import spark.SparkContext._ -import spark.storage.StorageLevel +import org.apache.spark.{Manifests, RDD, Partitioner, HashPartitioner} +import org.apache.spark.SparkContext._ +import org.apache.spark.storage.StorageLevel import scala.collection.mutable.ArrayBuffer @@ -60,7 +60,7 @@ extends Serializable { } /** - * Return a new DStream by applying `groupByKey` on each RDD. The supplied [[spark.Partitioner]] + * Return a new DStream by applying `groupByKey` on each RDD. The supplied [[org.apache.spark.Partitioner]] * is used to control the partitioning of each RDD. */ def groupByKey(partitioner: Partitioner): DStream[(K, Seq[V])] = { @@ -91,7 +91,7 @@ extends Serializable { /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are - * merged using the supplied reduce function. [[spark.Partitioner]] is used to control the + * merged using the supplied reduce function. [[org.apache.spark.Partitioner]] is used to control the * partitioning of each RDD. */ def reduceByKey(reduceFunc: (V, V) => V, partitioner: Partitioner): DStream[(K, V)] = { @@ -101,7 +101,7 @@ extends Serializable { /** * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the - * combineByKey for RDDs. Please refer to combineByKey in [[spark.PairRDDFunctions]] for more + * combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.PairRDDFunctions]] for more * information. */ def combineByKey[C: ClassManifest]( @@ -360,7 +360,7 @@ extends Serializable { /** * Create a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of the key. - * [[spark.Partitioner]] is used to control the partitioning of each RDD. + * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. * @param updateFunc State update function. If `this` function returns None, then * corresponding state key-value pair will be eliminated. * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream. @@ -379,7 +379,7 @@ extends Serializable { /** * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of each key. - * [[spark.Paxrtitioner]] is used to control the partitioning of each RDD. + * [[org.apache.spark.Paxrtitioner]] is used to control the partitioning of each RDD. * @param updateFunc State update function. If `this` function returns None, then * corresponding state key-value pair will be eliminated. Note, that * this function may generate a different a tuple with a different key diff --git a/streaming/src/main/scala/spark/streaming/Scheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala similarity index 95% rename from streaming/src/main/scala/spark/streaming/Scheduler.scala rename to streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala index 252cc2a3032..ed892e33e6c 100644 --- a/streaming/src/main/scala/spark/streaming/Scheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming import util.{ManualClock, RecurringTimer, Clock} -import spark.SparkEnv -import spark.Logging +import org.apache.spark.SparkEnv +import org.apache.spark.Logging private[streaming] class Scheduler(ssc: StreamingContext) extends Logging { @@ -34,7 +34,8 @@ class Scheduler(ssc: StreamingContext) extends Logging { null } - val clockClass = System.getProperty("spark.streaming.clock", "spark.streaming.util.SystemClock") + val clockClass = System.getProperty( + "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") val clock = Class.forName(clockClass).newInstance().asInstanceOf[Clock] val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds, longTime => generateJobs(new Time(longTime))) diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala similarity index 97% rename from streaming/src/main/scala/spark/streaming/StreamingContext.scala rename to streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 62c95b573aa..3852ac2daba 100644 --- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -15,21 +15,21 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming import akka.actor.Props import akka.actor.SupervisorStrategy import akka.zeromq.Subscribe -import spark.streaming.dstream._ +import org.apache.spark.streaming.dstream._ -import spark._ -import spark.streaming.receivers.ActorReceiver -import spark.streaming.receivers.ReceiverSupervisorStrategy -import spark.streaming.receivers.ZeroMQReceiver -import spark.storage.StorageLevel -import spark.util.MetadataCleaner -import spark.streaming.receivers.ActorReceiver +import org.apache.spark._ +import org.apache.spark.streaming.receivers.ActorReceiver +import org.apache.spark.streaming.receivers.ReceiverSupervisorStrategy +import org.apache.spark.streaming.receivers.ZeroMQReceiver +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.MetadataCleaner +import org.apache.spark.streaming.receivers.ActorReceiver import scala.collection.mutable.Queue import scala.collection.Map diff --git a/streaming/src/main/scala/spark/streaming/Time.scala b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala similarity index 98% rename from streaming/src/main/scala/spark/streaming/Time.scala rename to streaming/src/main/scala/org/apache/spark/streaming/Time.scala index ad5eab9dd2a..2678334f538 100644 --- a/streaming/src/main/scala/spark/streaming/Time.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming /** * This is a simple class that represents an absolute instant of time. diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala similarity index 90% rename from streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala index 7dcb1d713d9..f8c8d8ece1e 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala @@ -15,17 +15,17 @@ * limitations under the License. */ -package spark.streaming.api.java +package org.apache.spark.streaming.api.java -import spark.streaming.{Duration, Time, DStream} -import spark.api.java.function.{Function => JFunction} -import spark.api.java.JavaRDD -import spark.storage.StorageLevel -import spark.RDD +import org.apache.spark.streaming.{Duration, Time, DStream} +import org.apache.spark.api.java.function.{Function => JFunction} +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.RDD /** * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous - * sequence of RDDs (of the same type) representing a continuous stream of data (see [[spark.RDD]] + * sequence of RDDs (of the same type) representing a continuous stream of data (see [[org.apache.spark.RDD]] * for more details on RDDs). DStreams can either be created from live data (such as, data from * HDFS, Kafka or Flume) or it can be generated by transformation existing DStreams using operations * such as `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each @@ -33,7 +33,7 @@ import spark.RDD * by a parent DStream. * * This class contains the basic operations available on all DStreams, such as `map`, `filter` and - * `window`. In addition, [[spark.streaming.api.java.JavaPairDStream]] contains operations available + * `window`. In addition, [[org.apache.spark.streaming.api.java.JavaPairDStream]] contains operations available * only on DStreams of key-value pairs, such as `groupByKeyAndWindow` and `join`. * * DStreams internally is characterized by a few basic properties: diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala similarity index 97% rename from streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala rename to streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index 3ab5c1fddeb..2e6fe9a9c4f 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -15,18 +15,18 @@ * limitations under the License. */ -package spark.streaming.api.java +package org.apache.spark.streaming.api.java import java.util.{List => JList} import java.lang.{Long => JLong} import scala.collection.JavaConversions._ -import spark.streaming._ -import spark.api.java.{JavaPairRDD, JavaRDDLike, JavaRDD} -import spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _} +import org.apache.spark.streaming._ +import org.apache.spark.api.java.{JavaPairRDD, JavaRDDLike, JavaRDD} +import org.apache.spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _} import java.util -import spark.RDD +import org.apache.spark.RDD import JavaDStream._ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]] diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala similarity index 97% rename from streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index ea08fb38267..c203dccd171 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -15,24 +15,24 @@ * limitations under the License. */ -package spark.streaming.api.java +package org.apache.spark.streaming.api.java import java.util.{List => JList} import java.lang.{Long => JLong} import scala.collection.JavaConversions._ -import spark.streaming._ -import spark.streaming.StreamingContext._ -import spark.api.java.function.{Function => JFunction, Function2 => JFunction2} -import spark.{RDD, Partitioner} +import org.apache.spark.streaming._ +import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} +import org.apache.spark.{RDD, Partitioner} import org.apache.hadoop.mapred.{JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.hadoop.conf.Configuration -import spark.api.java.{JavaUtils, JavaRDD, JavaPairRDD} -import spark.storage.StorageLevel +import org.apache.spark.api.java.{JavaUtils, JavaRDD, JavaPairRDD} +import org.apache.spark.storage.StorageLevel import com.google.common.base.Optional -import spark.RDD +import org.apache.spark.RDD class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( implicit val kManifiest: ClassManifest[K], @@ -114,7 +114,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Return a new DStream by applying `groupByKey` on each RDD of `this` DStream. * Therefore, the values for each key in `this` DStream's RDDs are grouped into a - * single sequence to generate the RDDs of the new DStream. [[spark.Partitioner]] + * single sequence to generate the RDDs of the new DStream. [[org.apache.spark.Partitioner]] * is used to control the partitioning of each RDD. */ def groupByKey(partitioner: Partitioner): JavaPairDStream[K, JList[V]] = @@ -138,7 +138,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are - * merged using the supplied reduce function. [[spark.Partitioner]] is used to control the + * merged using the supplied reduce function. [[org.apache.spark.Partitioner]] is used to control the * partitioning of each RDD. */ def reduceByKey(func: JFunction2[V, V, V], partitioner: Partitioner): JavaPairDStream[K, V] = { @@ -147,7 +147,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Combine elements of each key in DStream's RDDs using custom function. This is similar to the - * combineByKey for RDDs. Please refer to combineByKey in [[spark.PairRDDFunctions]] for more + * combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.PairRDDFunctions]] for more * information. */ def combineByKey[C](createCombiner: JFunction[V, C], @@ -445,7 +445,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Create a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of the key. - * [[spark.Partitioner]] is used to control the partitioning of each RDD. + * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. * @param updateFunc State update function. If `this` function returns None, then * corresponding state key-value pair will be eliminated. * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream. diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala similarity index 97% rename from streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala rename to streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index b7720ad0ea1..f10beb1db39 100644 --- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.streaming.api.java +package org.apache.spark.streaming.api.java -import spark.streaming._ +import org.apache.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.spark.streaming.dstream._ +import org.apache.spark.storage.StorageLevel +import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} +import org.apache.spark.api.java.{JavaSparkContext, JavaRDD} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import twitter4j.Status import akka.actor.Props @@ -33,6 +33,7 @@ import java.lang.{Long => JLong, Integer => JInt} import java.io.InputStream import java.util.{Map => JMap} import twitter4j.auth.Authorization +import org.apache.spark.RDD /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic @@ -537,7 +538,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { def queueStream[T](queue: java.util.Queue[JavaRDD[T]]): JavaDStream[T] = { implicit val cm: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] - val sQueue = new scala.collection.mutable.Queue[spark.RDD[T]] + val sQueue = new scala.collection.mutable.Queue[RDD[T]] sQueue.enqueue(queue.map(_.rdd).toSeq: _*) ssc.queueStream(sQueue) } @@ -554,7 +555,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { def queueStream[T](queue: java.util.Queue[JavaRDD[T]], oneAtATime: Boolean): JavaDStream[T] = { implicit val cm: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] - val sQueue = new scala.collection.mutable.Queue[spark.RDD[T]] + val sQueue = new scala.collection.mutable.Queue[RDD[T]] sQueue.enqueue(queue.map(_.rdd).toSeq: _*) ssc.queueStream(sQueue, oneAtATime) } @@ -575,7 +576,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { defaultRDD: JavaRDD[T]): JavaDStream[T] = { implicit val cm: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] - val sQueue = new scala.collection.mutable.Queue[spark.RDD[T]] + val sQueue = new scala.collection.mutable.Queue[RDD[T]] sQueue.enqueue(queue.map(_.rdd).toSeq: _*) ssc.queueStream(sQueue, oneAtATime, defaultRDD.rdd) } diff --git a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala similarity index 90% rename from streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala index 99553d295d5..4a9d82211fc 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.{RDD, Partitioner} -import spark.rdd.CoGroupedRDD -import spark.streaming.{Time, DStream, Duration} +import org.apache.spark.{RDD, Partitioner} +import org.apache.spark.rdd.CoGroupedRDD +import org.apache.spark.streaming.{Time, DStream, Duration} private[streaming] class CoGroupedDStream[K : ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala similarity index 89% rename from streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala index 095137092a9..35cc4cb396b 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.RDD -import spark.streaming.{Time, StreamingContext} +import org.apache.spark.RDD +import org.apache.spark.streaming.{Time, StreamingContext} /** * An input stream that always returns the same RDD on each timestep. Useful for testing. diff --git a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala similarity index 97% rename from streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index de0536125de..1c265ed972d 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.RDD -import spark.rdd.UnionRDD -import spark.streaming.{DStreamCheckpointData, StreamingContext, Time} +import org.apache.spark.RDD +import org.apache.spark.rdd.UnionRDD +import org.apache.spark.streaming.{DStreamCheckpointData, StreamingContext, Time} import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} import org.apache.hadoop.conf.Configuration diff --git a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala similarity index 90% rename from streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala index 9d8c5c3175b..3166c687608 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Duration, DStream, Time} -import spark.RDD +import org.apache.spark.streaming.{Duration, DStream, Time} +import org.apache.spark.RDD private[streaming] class FilteredDStream[T: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala similarity index 88% rename from streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala index 78d7117f0f7..21950ad6ac9 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Duration, DStream, Time} -import spark.RDD -import spark.SparkContext._ +import org.apache.spark.streaming.{Duration, DStream, Time} +import org.apache.spark.RDD +import org.apache.spark.SparkContext._ private[streaming] class FlatMapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala similarity index 90% rename from streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala index d13bebb10f6..8377cfe60cd 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Duration, DStream, Time} -import spark.RDD +import org.apache.spark.streaming.{Duration, DStream, Time} +import org.apache.spark.RDD private[streaming] class FlatMappedDStream[T: ClassManifest, U: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala similarity index 96% rename from streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala index 4906f503c29..3fb443143cd 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.StreamingContext +import org.apache.spark.streaming.StreamingContext -import spark.Utils -import spark.storage.StorageLevel +import org.apache.spark.Utils +import org.apache.spark.storage.StorageLevel import org.apache.flume.source.avro.AvroSourceProtocol import org.apache.flume.source.avro.AvroFlumeEvent diff --git a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala similarity index 91% rename from streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala index 7df537eb560..c1f95650c8d 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.RDD -import spark.streaming.{Duration, DStream, Job, Time} +import org.apache.spark.RDD +import org.apache.spark.streaming.{Duration, DStream, Job, Time} private[streaming] class ForEachDStream[T: ClassManifest] ( diff --git a/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala similarity index 89% rename from streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala index 06fda6fe8e4..1e4c7e7fde8 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Duration, DStream, Time} -import spark.RDD +import org.apache.spark.streaming.{Duration, DStream, Time} +import org.apache.spark.RDD private[streaming] class GlommedDStream[T: ClassManifest](parent: DStream[T]) diff --git a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala similarity index 95% rename from streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala index 4dbdec459d0..674b27118ca 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Time, Duration, StreamingContext, DStream} +import org.apache.spark.streaming.{Time, Duration, StreamingContext, DStream} /** * This is the abstract base class for all input streams. This class provides to methods diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala similarity index 96% rename from streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala index 6ee588af154..51e913675d2 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.Logging -import spark.storage.StorageLevel -import spark.streaming.{Time, DStreamCheckpointData, StreamingContext} +import org.apache.spark.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Time, DStreamCheckpointData, StreamingContext} import java.util.Properties import java.util.concurrent.Executors diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala similarity index 90% rename from streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala index af41a1b9acd..1d79d707bb4 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Duration, DStream, Time} -import spark.RDD +import org.apache.spark.streaming.{Duration, DStream, Time} +import org.apache.spark.RDD private[streaming] class MapPartitionedDStream[T: ClassManifest, U: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala similarity index 87% rename from streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala index 8d8a6161c67..312e0c0567d 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Duration, DStream, Time} -import spark.RDD -import spark.SparkContext._ +import org.apache.spark.streaming.{Duration, DStream, Time} +import org.apache.spark.RDD +import org.apache.spark.SparkContext._ private[streaming] class MapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala similarity index 90% rename from streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala index 3fda84a38a1..af688dde5f3 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Duration, DStream, Time} -import spark.RDD +import org.apache.spark.streaming.{Duration, DStream, Time} +import org.apache.spark.RDD private[streaming] class MappedDStream[T: ClassManifest, U: ClassManifest] ( diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala similarity index 93% rename from streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala index 1db0a69a2f1..3d68da36a27 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Time, StreamingContext, AddBlocks, RegisterReceiver, DeregisterReceiver} +import org.apache.spark.streaming.{Time, StreamingContext, AddBlocks, RegisterReceiver, DeregisterReceiver} -import spark.{Logging, SparkEnv, RDD} -import spark.rdd.BlockRDD -import spark.storage.StorageLevel +import org.apache.spark.{Logging, SparkEnv, RDD} +import org.apache.spark.rdd.BlockRDD +import org.apache.spark.storage.StorageLevel import scala.collection.mutable.ArrayBuffer @@ -31,14 +31,14 @@ import akka.actor.{Props, Actor} import akka.pattern.ask import akka.dispatch.Await import akka.util.duration._ -import spark.streaming.util.{RecurringTimer, SystemClock} +import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} import java.util.concurrent.ArrayBlockingQueue /** * Abstract class for defining any InputDStream that has to start a receiver on worker * nodes to receive external data. Specific implementations of NetworkInputDStream must * define the getReceiver() function that gets the receiver object of type - * [[spark.streaming.dstream.NetworkReceiver]] that will be sent to the workers to receive + * [[org.apache.spark.streaming.dstream.NetworkReceiver]] that will be sent to the workers to receive * data. * @param ssc_ Streaming context that will execute this input stream * @tparam T Class type of the object of this stream @@ -83,7 +83,7 @@ private[streaming] case class ReportError(msg: String) extends NetworkReceiverMe /** * Abstract class of a receiver that can be run on worker nodes to receive external data. See - * [[spark.streaming.dstream.NetworkInputDStream]] for an explanation. + * [[org.apache.spark.streaming.dstream.NetworkInputDStream]] for an explanation. */ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Logging { @@ -202,7 +202,7 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log } /** - * Batches objects created by a [[spark.streaming.dstream.NetworkReceiver]] and puts them into + * Batches objects created by a [[org.apache.spark.streaming.dstream.NetworkReceiver]] and puts them into * appropriately named blocks at regular intervals. This class starts two threads, * one to periodically start a new batch and prepare the previous batch of as a block, * the other to push the blocks into the block manager. diff --git a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala similarity index 91% rename from streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala index 33f7cd063f3..15782f5c119 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.StreamingContext +import org.apache.spark.streaming.StreamingContext private[streaming] class PluggableInputDStream[T: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala similarity index 90% rename from streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala index b269061b737..b43ecaeebe0 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.RDD -import spark.rdd.UnionRDD +import org.apache.spark.RDD +import org.apache.spark.rdd.UnionRDD import scala.collection.mutable.Queue import scala.collection.mutable.ArrayBuffer -import spark.streaming.{Time, StreamingContext} +import org.apache.spark.streaming.{Time, StreamingContext} private[streaming] class QueueInputDStream[T: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala similarity index 95% rename from streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala index 236f74f575e..c91f12ecd7a 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.Logging -import spark.storage.StorageLevel -import spark.streaming.StreamingContext +import org.apache.spark.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext import java.net.InetSocketAddress import java.nio.ByteBuffer diff --git a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala similarity index 94% rename from streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala index 96260501ab7..b6c672f899e 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -15,18 +15,18 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.StreamingContext._ +import org.apache.spark.streaming.StreamingContext._ -import spark.RDD -import spark.rdd.{CoGroupedRDD, MapPartitionsRDD} -import spark.Partitioner -import spark.SparkContext._ -import spark.storage.StorageLevel +import org.apache.spark.RDD +import org.apache.spark.rdd.{CoGroupedRDD, MapPartitionsRDD} +import org.apache.spark.Partitioner +import org.apache.spark.SparkContext._ +import org.apache.spark.storage.StorageLevel import scala.collection.mutable.ArrayBuffer -import spark.streaming.{Duration, Interval, Time, DStream} +import org.apache.spark.streaming.{Duration, Interval, Time, DStream} private[streaming] class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala similarity index 88% rename from streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala index 83b57b27f77..3a0bd2acd71 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.{RDD, Partitioner} -import spark.SparkContext._ -import spark.streaming.{Duration, DStream, Time} +import org.apache.spark.{RDD, Partitioner} +import org.apache.spark.SparkContext._ +import org.apache.spark.streaming.{Duration, DStream, Time} private[streaming] class ShuffledDStream[K: ClassManifest, V: ClassManifest, C: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala similarity index 93% rename from streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala index 5877b10e0e1..e2539c73961 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.StreamingContext -import spark.storage.StorageLevel -import spark.util.NextIterator +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.NextIterator import java.io._ import java.net.Socket diff --git a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala similarity index 94% rename from streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala index 4b46613d5e1..c1c9f808f08 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.RDD -import spark.Partitioner -import spark.SparkContext._ -import spark.storage.StorageLevel -import spark.streaming.{Duration, Time, DStream} +import org.apache.spark.RDD +import org.apache.spark.Partitioner +import org.apache.spark.SparkContext._ +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Duration, Time, DStream} private[streaming] class StateDStream[K: ClassManifest, V: ClassManifest, S: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala similarity index 90% rename from streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala index e7fbc5bbcfa..edba2032b45 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.RDD -import spark.streaming.{Duration, DStream, Time} +import org.apache.spark.RDD +import org.apache.spark.streaming.{Duration, DStream, Time} private[streaming] class TransformedDStream[T: ClassManifest, U: ClassManifest] ( diff --git a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala similarity index 97% rename from streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala index f09a8b9f907..387e15b0e6d 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark._ -import spark.streaming._ +import org.apache.spark._ +import org.apache.spark.streaming._ import storage.StorageLevel import twitter4j._ import twitter4j.auth.Authorization diff --git a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala similarity index 91% rename from streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala index 3eaa9a7e7f4..97eab97b2fa 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.streaming.{Duration, DStream, Time} -import spark.RDD +import org.apache.spark.streaming.{Duration, DStream, Time} +import org.apache.spark.RDD import collection.mutable.ArrayBuffer -import spark.rdd.UnionRDD +import org.apache.spark.rdd.UnionRDD private[streaming] class UnionDStream[T: ClassManifest](parents: Array[DStream[T]]) diff --git a/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala similarity index 89% rename from streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala rename to streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala index fd24d61730d..dbbea39e812 100644 --- a/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.streaming.dstream +package org.apache.spark.streaming.dstream -import spark.RDD -import spark.rdd.UnionRDD -import spark.storage.StorageLevel -import spark.streaming.{Duration, Interval, Time, DStream} +import org.apache.spark.RDD +import org.apache.spark.rdd.UnionRDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Duration, Interval, Time, DStream} private[streaming] class WindowedDStream[T: ClassManifest]( diff --git a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala similarity index 97% rename from streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala rename to streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala index abeeff11b93..4b5d8c467e6 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.streaming.receivers +package org.apache.spark.streaming.receivers import akka.actor.{ Actor, PoisonPill, Props, SupervisorStrategy } import akka.actor.{ actorRef2Scala, ActorRef } import akka.actor.{ PossiblyHarmful, OneForOneStrategy } -import spark.storage.StorageLevel -import spark.streaming.dstream.NetworkReceiver +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.dstream.NetworkReceiver import java.util.concurrent.atomic.AtomicInteger diff --git a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala similarity index 95% rename from streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala rename to streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala index 22d554e7e49..043bb8c8bf7 100644 --- a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.streaming.receivers +package org.apache.spark.streaming.receivers import akka.actor.Actor import akka.zeromq._ -import spark.Logging +import org.apache.spark.Logging /** * A receiver to subscribe to ZeroMQ stream. diff --git a/streaming/src/main/scala/spark/streaming/util/Clock.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala similarity index 98% rename from streaming/src/main/scala/spark/streaming/util/Clock.scala rename to streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala index d9ac722df5c..f67bb2f6ac5 100644 --- a/streaming/src/main/scala/spark/streaming/util/Clock.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.streaming.util +package org.apache.spark.streaming.util private[streaming] trait Clock { diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala similarity index 98% rename from streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala rename to streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala index 8ce5d8daf52..50d72298e48 100644 --- a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala @@ -15,11 +15,11 @@ * limitations under the License. */ -package spark.streaming.util +package org.apache.spark.streaming.util -import spark.{Logging, RDD} -import spark.streaming._ -import spark.streaming.dstream.ForEachDStream +import org.apache.spark.{Logging, RDD} +import org.apache.spark.streaming._ +import org.apache.spark.streaming.dstream.ForEachDStream import StreamingContext._ import scala.util.Random diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala similarity index 96% rename from streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala rename to streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala index bf04120293e..4e6ce6eabd7 100644 --- a/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming.util +package org.apache.spark.streaming.util -import spark.SparkContext -import spark.SparkContext._ +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} import scala.collection.JavaConversions.mapAsScalaMap diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala similarity index 94% rename from streaming/src/main/scala/spark/streaming/util/RawTextSender.scala rename to streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala index 5cc6ad9dee3..249f6a22aeb 100644 --- a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.streaming.util +package org.apache.spark.streaming.util import java.nio.ByteBuffer -import spark.util.{RateLimitedOutputStream, IntParam} +import org.apache.spark.util.{RateLimitedOutputStream, IntParam} import java.net.ServerSocket -import spark.{Logging, KryoSerializer} +import org.apache.spark.{Logging, KryoSerializer} import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream import scala.io.Source import java.io.IOException diff --git a/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala similarity index 98% rename from streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala rename to streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala index 7ecc44236d4..d644240405c 100644 --- a/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.streaming.util +package org.apache.spark.streaming.util private[streaming] class RecurringTimer(val clock: Clock, val period: Long, val callback: (Long) => Unit) { diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java similarity index 98% rename from streaming/src/test/java/spark/streaming/JavaAPISuite.java rename to streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 3b93790baa8..c0d729ff876 100644 --- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.streaming; +package org.apache.spark.streaming; import com.google.common.base.Optional; import com.google.common.collect.Lists; @@ -28,20 +28,20 @@ import org.junit.Before; import org.junit.Test; import scala.Tuple2; -import spark.HashPartitioner; -import spark.api.java.JavaPairRDD; -import spark.api.java.JavaRDD; -import spark.api.java.JavaRDDLike; -import spark.api.java.JavaPairRDD; -import spark.api.java.JavaSparkContext; -import spark.api.java.function.*; -import spark.storage.StorageLevel; -import spark.streaming.api.java.JavaDStream; -import spark.streaming.api.java.JavaPairDStream; -import spark.streaming.api.java.JavaStreamingContext; -import spark.streaming.JavaTestUtils; -import spark.streaming.JavaCheckpointTestUtils; -import spark.streaming.InputStreamsSuite; +import org.apache.spark.HashPartitioner; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaRDDLike; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.*; +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; +import org.apache.spark.streaming.JavaTestUtils; +import org.apache.spark.streaming.JavaCheckpointTestUtils; +import org.apache.spark.streaming.InputStreamsSuite; import java.io.*; import java.util.*; @@ -59,7 +59,7 @@ public class JavaAPISuite implements Serializable { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock"); + System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala similarity index 84% rename from streaming/src/test/java/spark/streaming/JavaTestUtils.scala rename to streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala index f9d25db8da6..8a6604904de 100644 --- a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala @@ -15,20 +15,21 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming import collection.mutable.{SynchronizedBuffer, ArrayBuffer} import java.util.{List => JList} -import spark.streaming.api.java.{JavaPairDStream, JavaDStreamLike, JavaDStream, JavaStreamingContext} -import spark.streaming._ +import org.apache.spark.streaming.api.java.{JavaPairDStream, JavaDStreamLike, JavaDStream, JavaStreamingContext} +import org.apache.spark.streaming._ import java.util.ArrayList import collection.JavaConversions._ +import org.apache.spark.api.java.JavaRDDLike /** Exposes streaming test functionality in a Java-friendly way. */ trait JavaTestBase extends TestSuiteBase { /** - * Create a [[spark.streaming.TestInputStream]] and attach it to the supplied context. + * Create a [[org.apache.spark.streaming.TestInputStream]] and attach it to the supplied context. * The stream will be derived from the supplied lists of Java objects. **/ def attachTestInputStream[T]( @@ -46,11 +47,11 @@ trait JavaTestBase extends TestSuiteBase { /** * Attach a provided stream to it's associated StreamingContext as a - * [[spark.streaming.TestOutputStream]]. + * [[org.apache.spark.streaming.TestOutputStream]]. **/ - def attachTestOutputStream[T, This <: spark.streaming.api.java.JavaDStreamLike[T, This, R], - R <: spark.api.java.JavaRDDLike[T, R]]( - dstream: JavaDStreamLike[T, This, R]) = { + def attachTestOutputStream[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]]( + dstream: JavaDStreamLike[T, This, R]) = + { implicit val cm: ClassManifest[T] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]] val ostream = new TestOutputStream(dstream.dstream, diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala similarity index 98% rename from streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala rename to streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index 67e3e0cd30c..11586f72b6d 100644 --- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming -import spark.streaming.StreamingContext._ +import org.apache.spark.streaming.StreamingContext._ import scala.runtime.RichInt import util.ManualClock @@ -26,7 +26,7 @@ class BasicOperationsSuite extends TestSuiteBase { override def framework() = "BasicOperationsSuite" before { - System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") + System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } after { diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala similarity index 98% rename from streaming/src/test/scala/spark/streaming/CheckpointSuite.scala rename to streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 8c639648f00..a327de80b3e 100644 --- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming import dstream.FileInputDStream -import spark.streaming.StreamingContext._ +import org.apache.spark.streaming.StreamingContext._ import java.io.File import runtime.RichInt import org.scalatest.BeforeAndAfter @@ -36,7 +36,7 @@ import com.google.common.io.Files */ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { - System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") + System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") before { FileUtils.deleteDirectory(new File(checkpointDir)) @@ -63,7 +63,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter { assert(batchDuration === Milliseconds(500), "batchDuration for this test must be 1 second") - System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") + System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") val stateStreamCheckpointInterval = Seconds(1) diff --git a/streaming/src/test/scala/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala similarity index 93% rename from streaming/src/test/scala/spark/streaming/FailureSuite.scala rename to streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala index 7fc649fe270..6337c5359c3 100644 --- a/streaming/src/test/scala/spark/streaming/FailureSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming -import spark.Logging -import spark.streaming.util.MasterFailureTest +import org.apache.spark.Logging +import org.apache.spark.streaming.util.MasterFailureTest import StreamingContext._ import org.scalatest.{FunSuite, BeforeAndAfter} diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala similarity index 97% rename from streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala rename to streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 1c5419b16da..42e3e51e3fa 100644 --- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming import akka.actor.Actor import akka.actor.IO @@ -29,9 +29,9 @@ import java.io.{File, BufferedWriter, OutputStreamWriter} import java.util.concurrent.{TimeUnit, ArrayBlockingQueue} import collection.mutable.{SynchronizedBuffer, ArrayBuffer} import util.ManualClock -import spark.storage.StorageLevel -import spark.streaming.receivers.Receiver -import spark.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.receivers.Receiver +import org.apache.spark.Logging import scala.util.Random import org.apache.commons.io.FileUtils import org.scalatest.BeforeAndAfter @@ -52,7 +52,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { override def checkpointDir = "checkpoint" before { - System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") + System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } after { @@ -207,7 +207,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { FileUtils.deleteDirectory(testDir) // Enable manual clock back again for other tests - System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") + System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } diff --git a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala similarity index 98% rename from streaming/src/test/scala/spark/streaming/TestSuiteBase.scala rename to streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index cb34b5a7cc4..31c2fa0208e 100644 --- a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming -import spark.streaming.dstream.{InputDStream, ForEachDStream} -import spark.streaming.util.ManualClock +import org.apache.spark.streaming.dstream.{InputDStream, ForEachDStream} +import org.apache.spark.streaming.util.ManualClock -import spark.{RDD, Logging} +import org.apache.spark.{RDD, Logging} import collection.mutable.ArrayBuffer import collection.mutable.SynchronizedBuffer diff --git a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala similarity index 98% rename from streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala rename to streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala index 894b765fc62..f50e05c0d88 100644 --- a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package spark.streaming +package org.apache.spark.streaming -import spark.streaming.StreamingContext._ +import org.apache.spark.streaming.StreamingContext._ import collection.mutable.ArrayBuffer class WindowOperationsSuite extends TestSuiteBase { - System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock") + System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") override def framework = "WindowOperationsSuite" diff --git a/tools/pom.xml b/tools/pom.xml index 95b5e80e5b1..664bf7dcbdb 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -18,13 +18,13 @@ 4.0.0 - org.spark-project + org.apache.spark spark-parent 0.8.0-SNAPSHOT ../pom.xml - org.spark-project + org.apache.spark spark-tools jar Spark Project Tools @@ -32,12 +32,12 @@ - org.spark-project + org.apache.spark spark-core ${project.version} - org.spark-project + org.apache.spark spark-streaming ${project.version} diff --git a/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala similarity index 64% rename from tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala rename to tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala index f45d0b281cf..50335e5736b 100644 --- a/tools/src/main/scala/spark/tools/JavaAPICompletenessChecker.scala +++ b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala @@ -15,17 +15,17 @@ * limitations under the License. */ -package spark.tools +package org.apache.spark.tools import java.lang.reflect.Method import scala.collection.mutable.ArrayBuffer -import spark._ -import spark.api.java._ -import spark.rdd.OrderedRDDFunctions -import spark.streaming.{PairDStreamFunctions, DStream, StreamingContext} -import spark.streaming.api.java.{JavaPairDStream, JavaDStream, JavaStreamingContext} +import org.apache.spark._ +import org.apache.spark.api.java._ +import org.apache.spark.rdd.OrderedRDDFunctions +import org.apache.spark.streaming.{PairDStreamFunctions, DStream, StreamingContext} +import org.apache.spark.streaming.api.java.{JavaPairDStream, JavaDStream, JavaStreamingContext} private[spark] abstract class SparkType(val name: String) @@ -129,7 +129,7 @@ object JavaAPICompletenessChecker { // 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.Function2" -> "org.apache.spark.api.java.function.Function2", "scala.collection.Iterator" -> "java.util.Iterator", "scala.collection.mutable.Queue" -> "java.util.Queue", "double" -> "java.lang.Double" @@ -139,7 +139,7 @@ object JavaAPICompletenessChecker { scalaType match { case ParameterizedType(name, parameters, typebounds) => name match { - case "spark.RDD" => + case "org.apache.spark.RDD" => if (parameters(0).name == classOf[Tuple2[_, _]].getName) { val tupleParams = parameters(0).asInstanceOf[ParameterizedType].parameters.map(applySubs) @@ -147,13 +147,13 @@ object JavaAPICompletenessChecker { } else { ParameterizedType(classOf[JavaRDD[_]].getName, parameters.map(applySubs)) } - case "spark.streaming.DStream" => + case "org.apache.spark.streaming.DStream" => if (parameters(0).name == classOf[Tuple2[_, _]].getName) { val tupleParams = parameters(0).asInstanceOf[ParameterizedType].parameters.map(applySubs) - ParameterizedType("spark.streaming.api.java.JavaPairDStream", tupleParams) + ParameterizedType("org.apache.spark.streaming.api.java.JavaPairDStream", tupleParams) } else { - ParameterizedType("spark.streaming.api.java.JavaDStream", + ParameterizedType("org.apache.spark.streaming.api.java.JavaDStream", parameters.map(applySubs)) } case "scala.Option" => { @@ -167,14 +167,14 @@ object JavaAPICompletenessChecker { val firstParamName = parameters.last.name if (firstParamName.startsWith("scala.collection.Traversable") || firstParamName.startsWith("scala.collection.Iterator")) { - ParameterizedType("spark.api.java.function.FlatMapFunction", + ParameterizedType("org.apache.spark.api.java.function.FlatMapFunction", Seq(parameters(0), parameters.last.asInstanceOf[ParameterizedType].parameters(0)).map(applySubs)) } else if (firstParamName == "scala.runtime.BoxedUnit") { - ParameterizedType("spark.api.java.function.VoidFunction", + ParameterizedType("org.apache.spark.api.java.function.VoidFunction", parameters.dropRight(1).map(applySubs)) } else { - ParameterizedType("spark.api.java.function.Function", parameters.map(applySubs)) + ParameterizedType("org.apache.spark.api.java.function.Function", parameters.map(applySubs)) } case _ => ParameterizedType(renameSubstitutions.getOrElse(name, name), @@ -211,85 +211,85 @@ object JavaAPICompletenessChecker { // 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" + "org.apache.spark.RDD.origin", + "org.apache.spark.RDD.elementClassManifest", + "org.apache.spark.RDD.checkpointData", + "org.apache.spark.RDD.partitioner", + "org.apache.spark.RDD.partitions", + "org.apache.spark.RDD.firstParent", + "org.apache.spark.RDD.doCheckpoint", + "org.apache.spark.RDD.markCheckpointed", + "org.apache.spark.RDD.clearDependencies", + "org.apache.spark.RDD.getDependencies", + "org.apache.spark.RDD.getPartitions", + "org.apache.spark.RDD.dependencies", + "org.apache.spark.RDD.getPreferredLocations", + "org.apache.spark.RDD.collectPartitions", + "org.apache.spark.RDD.computeOrReadCheckpoint", + "org.apache.spark.PairRDDFunctions.getKeyClass", + "org.apache.spark.PairRDDFunctions.getValueClass", + "org.apache.spark.SparkContext.stringToText", + "org.apache.spark.SparkContext.makeRDD", + "org.apache.spark.SparkContext.runJob", + "org.apache.spark.SparkContext.runApproximateJob", + "org.apache.spark.SparkContext.clean", + "org.apache.spark.SparkContext.metadataCleaner", + "org.apache.spark.SparkContext.ui", + "org.apache.spark.SparkContext.newShuffleId", + "org.apache.spark.SparkContext.newRddId", + "org.apache.spark.SparkContext.cleanup", + "org.apache.spark.SparkContext.receiverJobThread", + "org.apache.spark.SparkContext.getRDDStorageInfo", + "org.apache.spark.SparkContext.addedFiles", + "org.apache.spark.SparkContext.addedJars", + "org.apache.spark.SparkContext.persistentRdds", + "org.apache.spark.SparkContext.executorEnvs", + "org.apache.spark.SparkContext.checkpointDir", + "org.apache.spark.SparkContext.getSparkHome", + "org.apache.spark.SparkContext.executorMemoryRequested", + "org.apache.spark.SparkContext.getExecutorStorageStatus", + "org.apache.spark.streaming.DStream.generatedRDDs", + "org.apache.spark.streaming.DStream.zeroTime", + "org.apache.spark.streaming.DStream.rememberDuration", + "org.apache.spark.streaming.DStream.storageLevel", + "org.apache.spark.streaming.DStream.mustCheckpoint", + "org.apache.spark.streaming.DStream.checkpointDuration", + "org.apache.spark.streaming.DStream.checkpointData", + "org.apache.spark.streaming.DStream.graph", + "org.apache.spark.streaming.DStream.isInitialized", + "org.apache.spark.streaming.DStream.parentRememberDuration", + "org.apache.spark.streaming.DStream.initialize", + "org.apache.spark.streaming.DStream.validate", + "org.apache.spark.streaming.DStream.setContext", + "org.apache.spark.streaming.DStream.setGraph", + "org.apache.spark.streaming.DStream.remember", + "org.apache.spark.streaming.DStream.getOrCompute", + "org.apache.spark.streaming.DStream.generateJob", + "org.apache.spark.streaming.DStream.clearOldMetadata", + "org.apache.spark.streaming.DStream.addMetadata", + "org.apache.spark.streaming.DStream.updateCheckpointData", + "org.apache.spark.streaming.DStream.restoreCheckpointData", + "org.apache.spark.streaming.DStream.isTimeValid", + "org.apache.spark.streaming.StreamingContext.nextNetworkInputStreamId", + "org.apache.spark.streaming.StreamingContext.networkInputTracker", + "org.apache.spark.streaming.StreamingContext.checkpointDir", + "org.apache.spark.streaming.StreamingContext.checkpointDuration", + "org.apache.spark.streaming.StreamingContext.receiverJobThread", + "org.apache.spark.streaming.StreamingContext.scheduler", + "org.apache.spark.streaming.StreamingContext.initialCheckpoint", + "org.apache.spark.streaming.StreamingContext.getNewNetworkStreamId", + "org.apache.spark.streaming.StreamingContext.validate", + "org.apache.spark.streaming.StreamingContext.createNewSparkContext", + "org.apache.spark.streaming.StreamingContext.rddToFileName", + "org.apache.spark.streaming.StreamingContext.getSparkCheckpointDir", + "org.apache.spark.streaming.StreamingContext.env", + "org.apache.spark.streaming.StreamingContext.graph", + "org.apache.spark.streaming.StreamingContext.isCheckpointPresent" ) val excludedPatterns = Seq( - """^spark\.SparkContext\..*To.*Functions""", - """^spark\.SparkContext\..*WritableConverter""", - """^spark\.SparkContext\..*To.*Writable""" + """^org\.apache\.spark\.SparkContext\..*To.*Functions""", + """^org\.apache\.spark\.SparkContext\..*WritableConverter""", + """^org\.apache\.spark\.SparkContext\..*To.*Writable""" ).map(_.r) lazy val excludedByPattern = !excludedPatterns.map(_.findFirstIn(name)).filter(_.isDefined).isEmpty @@ -298,7 +298,7 @@ object JavaAPICompletenessChecker { private def isExcludedByInterface(method: Method): Boolean = { val excludedInterfaces = - Set("spark.Logging", "org.apache.hadoop.mapreduce.HadoopMapReduceUtil") + Set("org.apache.spark.Logging", "org.apache.hadoop.mapreduce.HadoopMapReduceUtil") def toComparisionKey(method: Method) = (method.getReturnType, method.getName, method.getGenericReturnType) val interfaces = method.getDeclaringClass.getInterfaces.filter { i => diff --git a/yarn/pom.xml b/yarn/pom.xml index 07dd170eae5..a2afbeabff4 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -18,13 +18,13 @@ 4.0.0 - org.spark-project + org.apache.spark spark-parent 0.8.0-SNAPSHOT ../pom.xml - org.spark-project + org.apache.spark spark-yarn jar Spark Project YARN Support @@ -81,7 +81,7 @@ hadoop2-yarn - org.spark-project + org.apache.spark spark-core ${project.version} diff --git a/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala similarity index 99% rename from yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 0f3b6bc1a65..139a977a031 100644 --- a/yarn/src/main/scala/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy.yarn +package org.apache.spark.deploy.yarn import java.net.Socket import java.util.concurrent.CopyOnWriteArrayList @@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import scala.collection.JavaConversions._ -import spark.{SparkContext, Logging, Utils} +import org.apache.spark.{SparkContext, Logging, Utils} import org.apache.hadoop.security.UserGroupInformation import java.security.PrivilegedExceptionAction diff --git a/yarn/src/main/scala/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala similarity index 97% rename from yarn/src/main/scala/spark/deploy/yarn/ApplicationMasterArguments.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index 8de44b1f661..f47e23b63f4 100644 --- a/yarn/src/main/scala/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.deploy.yarn +package org.apache.spark.deploy.yarn -import spark.util.IntParam +import org.apache.spark.util.IntParam import collection.mutable.ArrayBuffer class ApplicationMasterArguments(val args: Array[String]) { diff --git a/yarn/src/main/scala/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala similarity index 99% rename from yarn/src/main/scala/spark/deploy/yarn/Client.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index eb2a8cc6420..48e737ed797 100644 --- a/yarn/src/main/scala/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy.yarn +package org.apache.spark.deploy.yarn import java.net.{InetSocketAddress, URI} import java.nio.ByteBuffer @@ -33,10 +33,10 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import scala.collection.mutable.HashMap import scala.collection.JavaConversions._ -import spark.{Logging, Utils} +import org.apache.spark.{Logging, Utils} import org.apache.hadoop.yarn.util.{Apps, Records, ConverterUtils} import org.apache.hadoop.yarn.api.ApplicationConstants.Environment -import spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.SparkHadoopUtil class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging { diff --git a/yarn/src/main/scala/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala similarity index 95% rename from yarn/src/main/scala/spark/deploy/yarn/ClientArguments.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 67aff03781c..6cbfadc23be 100644 --- a/yarn/src/main/scala/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package spark.deploy.yarn +package org.apache.spark.deploy.yarn -import spark.util.MemoryParam -import spark.util.IntParam +import org.apache.spark.util.MemoryParam +import org.apache.spark.util.IntParam import collection.mutable.{ArrayBuffer, HashMap} -import spark.scheduler.{InputFormatInfo, SplitInfo} +import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo} // TODO: Add code and support for ensuring that yarn resource 'asks' are location aware ! class ClientArguments(val args: Array[String]) { diff --git a/yarn/src/main/scala/spark/deploy/yarn/WorkerRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala similarity index 98% rename from yarn/src/main/scala/spark/deploy/yarn/WorkerRunnable.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala index 0e1fd9b680e..72dcf7178eb 100644 --- a/yarn/src/main/scala/spark/deploy/yarn/WorkerRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package spark.deploy.yarn +package org.apache.spark.deploy.yarn import java.net.URI import java.nio.ByteBuffer @@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import scala.collection.JavaConversions._ import scala.collection.mutable.HashMap -import spark.{Logging, Utils} +import org.apache.spark.{Logging, Utils} class WorkerRunnable(container: Container, conf: Configuration, masterAddress: String, slaveId: String, hostname: String, workerMemory: Int, workerCores: Int) @@ -119,7 +119,7 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S // TODO: If the OOM is not recoverable by rescheduling it on different node, then do 'something' to fail job ... akin to blacklisting trackers in mapred ? " -XX:OnOutOfMemoryError='kill %p' " + JAVA_OPTS + - " spark.executor.StandaloneExecutorBackend " + + " org.apache.spark.executor.StandaloneExecutorBackend " + masterAddress + " " + slaveId + " " + hostname + " " + diff --git a/yarn/src/main/scala/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala similarity index 99% rename from yarn/src/main/scala/spark/deploy/yarn/YarnAllocationHandler.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index b0af8baf08c..26ff214e122 100644 --- a/yarn/src/main/scala/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package spark.deploy.yarn +package org.apache.spark.deploy.yarn -import spark.{Logging, Utils} -import spark.scheduler.SplitInfo +import org.apache.spark.{Logging, Utils} +import org.apache.spark.scheduler.SplitInfo import scala.collection import org.apache.hadoop.yarn.api.records.{AMResponse, ApplicationAttemptId, ContainerId, Priority, Resource, ResourceRequest, ContainerStatus, Container} -import spark.scheduler.cluster.{ClusterScheduler, StandaloneSchedulerBackend} +import org.apache.spark.scheduler.cluster.{ClusterScheduler, StandaloneSchedulerBackend} import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse} import org.apache.hadoop.yarn.util.{RackResolver, Records} import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap} diff --git a/yarn/src/main/scala/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala similarity index 95% rename from yarn/src/main/scala/spark/deploy/yarn/YarnSparkHadoopUtil.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 77c4ee7f3f6..ca2f1e2565b 100644 --- a/yarn/src/main/scala/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package spark.deploy.yarn +package org.apache.spark.deploy.yarn -import spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.SparkHadoopUtil import collection.mutable.HashMap import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.UserGroupInformation diff --git a/yarn/src/main/scala/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala similarity index 93% rename from yarn/src/main/scala/spark/scheduler/cluster/YarnClusterScheduler.scala rename to yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala index bb58353e0cc..3828ddfc4f1 100644 --- a/yarn/src/main/scala/spark/scheduler/cluster/YarnClusterScheduler.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala @@ -15,10 +15,10 @@ * limitations under the License. */ -package spark.scheduler.cluster +package org.apache.spark.scheduler.cluster -import spark._ -import spark.deploy.yarn.{ApplicationMaster, YarnAllocationHandler} +import org.apache.spark._ +import org.apache.spark.deploy.yarn.{ApplicationMaster, YarnAllocationHandler} import org.apache.hadoop.conf.Configuration /** From 4d1cb59fe1dc9181268c8e61fc042990854f83c5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 31 Aug 2013 21:47:55 -0700 Subject: [PATCH 0900/2521] Small tweak to docs gradient --- docs/css/bootstrap.min.css | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/css/bootstrap.min.css b/docs/css/bootstrap.min.css index cf4c519cdc7..3444ed79e24 100644 --- a/docs/css/bootstrap.min.css +++ b/docs/css/bootstrap.min.css @@ -6,4 +6,4 @@ * 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{height:auto;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;line-height:0;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:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-static-top .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;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-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.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:20px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:1;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1{font-size:36px;line-height:40px}h2{font-size:30px;line-height:40px}h3{font-size:24px;line-height:40px}h4{font-size:18px;line-height:20px}h5{font-size:14px;line-height:20px}h6{font-size:12px;line-height:20px}h1 small{font-size:24px}h2 small{font-size:18px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.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:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}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 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:25px}blockquote small{display:block;line-height:20px;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}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,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:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;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:20px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}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:20px;padding:4px 6px;margin-bottom:9px;font-size:14px;line-height:20px;color:#555;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}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-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;-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:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal;cursor:pointer}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;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}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-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)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.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[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .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}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"]{float:left}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}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;-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)}.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:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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;-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)}.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:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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;-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)}.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:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{margin-bottom:5px;font-size:0;white-space:nowrap}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;font-size:14px;vertical-align:top;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn{margin-left:-1px;vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .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 .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}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 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;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{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:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;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:10px;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:20px}.table th,.table td{padding:8px;line-height:20px;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:collapse;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,.table-bordered tfoot: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,.table-bordered tfoot: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-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-right-topleft:4px}.table-striped tbody tr:nth-child(odd) td,.table-striped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table-hover tbody tr:hover td,.table-hover tbody tr:hover th{background-color:#f5f5f5}table [class*=span],.row-fluid table [class*=span]{display:table-cell;float:none;margin-left:0}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}.table tbody tr.success td{background-color:#dff0d8}.table tbody tr.error td{background-color:#f2dede}.table tbody tr.info td{background-color:#d9edf7}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*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}.icon-white,.nav>.active>a>[class^="icon-"],.nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"]{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{width:16px;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{width:16px;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:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 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:6px;-moz-border-radius:6px;border-radius:6px;-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:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu li>a:focus,.dropdown-submenu:hover>a{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0088cc;background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0081c2;background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-repeat:repeat-x;outline:0;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .disabled>a,.dropdown-menu .disabled>a:hover{color:#999}.dropdown-menu .disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.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}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover .dropdown-menu{display:block}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.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 #e3e3e3;-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;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;overflow:visible \9;-webkit-transition:height .35s ease;-moz-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:20px;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 14px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;*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:-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(to bottom,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #bbb;*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:#a2a2a2;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',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]{color:#333;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;-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:16px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:2px}.btn-small{padding:3px 9px;font-size:12px;line-height:18px}.btn-small [class^="icon-"]{margin-top:0}.btn-mini{padding:2px 6px;font-size:11px;line-height:16px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}.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:#c5c5c5;border-color:rgba(0,0,0,0.15) rgba(0,0,0,0.15) rgba(0,0,0,0.25)}.btn-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-image:-moz-linear-gradient(top,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;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='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#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(to bottom,#fbb450,#f89406);background-image:-moz-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='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#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:linear-gradient(to bottom,#ee5f5b,#bd362f);background-image:-moz-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='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#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:linear-gradient(to bottom,#62c462,#51a351);background-image:-moz-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='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#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:linear-gradient(to bottom,#5bc0de,#2f96b4);background-image:-moz-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='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-image:-moz-linear-gradient(top,#444,#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='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}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-link,.btn-link:active{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover{color:#005580;text-decoration:underline;background-color:transparent}.btn-group{position:relative;*margin-left:.3em;font-size:0;white-space:nowrap}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-toolbar .btn+.btn,.btn-toolbar .btn-group+.btn,.btn-toolbar .btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu{font-size:14px}.btn-group>.btn-mini{font-size:11px}.btn-group>.btn-small{font-size:12px}.btn-group>.btn-large{font-size:16px}.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>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;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-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;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:#04c}.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:8px;margin-left:0}.btn-mini .caret,.btn-small .caret,.btn-large .caret{margin-top:6px}.btn-large .caret{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}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical .btn{display:block;float:none;width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical .btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical .btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical .btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical .btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical .btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;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 h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.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:20px;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-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;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:9px 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;line-height:0;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:20px;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-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft: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 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover{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;line-height:0;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}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible;color:#555}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#e2f2e2);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e2f2e2));background-image:-webkit-linear-gradient(top,#fff,#d8e8f8);background-image:-o-linear-gradient(top,#fff,#d8e8f8);background-image:linear-gradient(to bottom,#fff,#d8e8f8);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffddff',endColorstr='#ffe2f2e2',GradientType=0);-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#555;text-shadow:0 1px 0 #fff}.navbar .brand:hover{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px}.navbar-link{color:#555}.navbar-link:hover{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:6px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;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,.navbar-form .btn{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:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;width:100%;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.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,.navbar-static-top .navbar-inner{border: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-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#555;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;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='#fff2f2f2',endColorstr='#ffe5e5e5',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]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#ccc \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 .nav>li>.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 .nav>li>.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 .nav>li>.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 .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse{color:#999}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover{color:#fff}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-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;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;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-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-image:-moz-linear-gradient(top,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #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='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#ccc}.breadcrumb .active{color:#999}.pagination{height:40px;margin:20px 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,.pagination span{float:left;padding:0 14px;line-height:38px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a,.pagination .active span{background-color:#f5f5f5}.pagination .active a,.pagination .active span{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,.pagination li:first-child span{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,.pagination li:last-child span{-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:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;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:2080}.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;-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-header h3{margin:0;line-height:30px}.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;line-height:0;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:1030;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:-3px}.tooltip.right{margin-left:3px}.tooltip.bottom{margin-top:3px}.tooltip.left{margin-left:-3px}.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;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;width:236px;padding:1px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-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}.popover.top{margin-bottom:10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-right:10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.popover .arrow,.popover .arrow:after{position:absolute;display:inline-block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow:after{z-index:-1;content:""}.popover.top .arrow{bottom:-10px;left:50%;margin-left:-10px;border-top-color:#fff;border-width:10px 10px 0}.popover.top .arrow:after{bottom:-1px;left:-11px;border-top-color:rgba(0,0,0,0.25);border-width:11px 11px 0}.popover.right .arrow{top:50%;left:-10px;margin-top:-10px;border-right-color:#fff;border-width:10px 10px 10px 0}.popover.right .arrow:after{bottom:-11px;left:-1px;border-right-color:rgba(0,0,0,0.25);border-width:11px 11px 11px 0}.popover.bottom .arrow{top:-10px;left:50%;margin-left:-10px;border-bottom-color:#fff;border-width:0 10px 10px}.popover.bottom .arrow:after{top:-1px;left:-11px;border-bottom-color:rgba(0,0,0,0.25);border-width:0 11px 11px}.popover.left .arrow{top:50%;right:-10px;margin-top:-10px;border-left-color:#fff;border-width:10px 0 10px 10px}.popover.left .arrow:after{right:-1px;bottom:-11px;border-left-color:rgba(0,0,0,0.25);border-width:11px 0 11px 11px}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}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;color:#555}.label,.badge{font-size:11.844px;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}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-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:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-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(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#fff9f9f9',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{float:left;width:0;height:100%;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(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',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;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;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:-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);-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,.progress .bar-danger{background-color:#dd514c;background-image:-moz-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(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{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:-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,.progress .bar-success{background-color:#5eb95e;background-image:-moz-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(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{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:-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,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-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(to bottom,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff339bb9',GradientType=0)}.progress-info.progress-striped .bar,.progress-striped .bar-info{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:-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,.progress .bar-warning{background-color:#faa732;background-image:-moz-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(to bottom,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0)}.progress-warning.progress-striped .bar,.progress-striped .bar-warning{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:-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:20px}.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:20px;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;-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:15px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{line-height:20px;color:#fff}.carousel-caption h4{margin:0 0 5px}.carousel-caption p{margin-bottom:0}.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:30px;color:inherit}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden}.affix{position:fixed} + */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{height:auto;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;line-height:0;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:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-static-top .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;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-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.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:20px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:1;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1{font-size:36px;line-height:40px}h2{font-size:30px;line-height:40px}h3{font-size:24px;line-height:40px}h4{font-size:18px;line-height:20px}h5{font-size:14px;line-height:20px}h6{font-size:12px;line-height:20px}h1 small{font-size:24px}h2 small{font-size:18px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.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:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}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 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:25px}blockquote small{display:block;line-height:20px;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}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,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:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;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:20px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}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:20px;padding:4px 6px;margin-bottom:9px;font-size:14px;line-height:20px;color:#555;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}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-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;-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:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal;cursor:pointer}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;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}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-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)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.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[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .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}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"]{float:left}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}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;-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)}.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:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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;-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)}.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:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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;-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)}.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:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{margin-bottom:5px;font-size:0;white-space:nowrap}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;font-size:14px;vertical-align:top;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn{margin-left:-1px;vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .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 .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}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 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;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{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:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;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:10px;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:20px}.table th,.table td{padding:8px;line-height:20px;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:collapse;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,.table-bordered tfoot: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,.table-bordered tfoot: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-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-right-topleft:4px}.table-striped tbody tr:nth-child(odd) td,.table-striped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table-hover tbody tr:hover td,.table-hover tbody tr:hover th{background-color:#f5f5f5}table [class*=span],.row-fluid table [class*=span]{display:table-cell;float:none;margin-left:0}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}.table tbody tr.success td{background-color:#dff0d8}.table tbody tr.error td{background-color:#f2dede}.table tbody tr.info td{background-color:#d9edf7}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*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}.icon-white,.nav>.active>a>[class^="icon-"],.nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"]{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{width:16px;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{width:16px;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:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 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:6px;-moz-border-radius:6px;border-radius:6px;-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:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu li>a:focus,.dropdown-submenu:hover>a{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0088cc;background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0081c2;background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-repeat:repeat-x;outline:0;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .disabled>a,.dropdown-menu .disabled>a:hover{color:#999}.dropdown-menu .disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.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}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover .dropdown-menu{display:block}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.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 #e3e3e3;-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;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;overflow:visible \9;-webkit-transition:height .35s ease;-moz-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:20px;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 14px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;*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:-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(to bottom,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #bbb;*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:#a2a2a2;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',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]{color:#333;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;-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:16px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:2px}.btn-small{padding:3px 9px;font-size:12px;line-height:18px}.btn-small [class^="icon-"]{margin-top:0}.btn-mini{padding:2px 6px;font-size:11px;line-height:16px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}.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:#c5c5c5;border-color:rgba(0,0,0,0.15) rgba(0,0,0,0.15) rgba(0,0,0,0.25)}.btn-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-image:-moz-linear-gradient(top,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;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='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#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(to bottom,#fbb450,#f89406);background-image:-moz-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='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#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:linear-gradient(to bottom,#ee5f5b,#bd362f);background-image:-moz-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='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#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:linear-gradient(to bottom,#62c462,#51a351);background-image:-moz-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='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#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:linear-gradient(to bottom,#5bc0de,#2f96b4);background-image:-moz-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='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-image:-moz-linear-gradient(top,#444,#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='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}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-link,.btn-link:active{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover{color:#005580;text-decoration:underline;background-color:transparent}.btn-group{position:relative;*margin-left:.3em;font-size:0;white-space:nowrap}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-toolbar .btn+.btn,.btn-toolbar .btn-group+.btn,.btn-toolbar .btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu{font-size:14px}.btn-group>.btn-mini{font-size:11px}.btn-group>.btn-small{font-size:12px}.btn-group>.btn-large{font-size:16px}.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>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;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-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;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:#04c}.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:8px;margin-left:0}.btn-mini .caret,.btn-small .caret,.btn-large .caret{margin-top:6px}.btn-large .caret{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}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical .btn{display:block;float:none;width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical .btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical .btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical .btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical .btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical .btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;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 h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.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:20px;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-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;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:9px 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;line-height:0;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:20px;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-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft: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 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover{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;line-height:0;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}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible;color:#555}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#d6f2e2);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#d6f2e2));background-image:-webkit-linear-gradient(top,#fff,#d2e2f2);background-image:-o-linear-gradient(top,#fff,#d2e2f2);background-image:linear-gradient(to bottom,#fff,#d2e2f2);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffd2e2f2',GradientType=0);-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#555;text-shadow:0 1px 0 #fff}.navbar .brand:hover{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px}.navbar-link{color:#555}.navbar-link:hover{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:6px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;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,.navbar-form .btn{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:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;width:100%;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.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,.navbar-static-top .navbar-inner{border: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-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#555;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;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='#fff2f2f2',endColorstr='#ffe5e5e5',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]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#ccc \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 .nav>li>.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 .nav>li>.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 .nav>li>.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 .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse{color:#999}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover{color:#fff}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-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;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;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-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-image:-moz-linear-gradient(top,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #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='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#ccc}.breadcrumb .active{color:#999}.pagination{height:40px;margin:20px 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,.pagination span{float:left;padding:0 14px;line-height:38px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a,.pagination .active span{background-color:#f5f5f5}.pagination .active a,.pagination .active span{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,.pagination li:first-child span{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,.pagination li:last-child span{-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:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;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:2080}.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;-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-header h3{margin:0;line-height:30px}.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;line-height:0;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:1030;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:-3px}.tooltip.right{margin-left:3px}.tooltip.bottom{margin-top:3px}.tooltip.left{margin-left:-3px}.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;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;width:236px;padding:1px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-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}.popover.top{margin-bottom:10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-right:10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.popover .arrow,.popover .arrow:after{position:absolute;display:inline-block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow:after{z-index:-1;content:""}.popover.top .arrow{bottom:-10px;left:50%;margin-left:-10px;border-top-color:#fff;border-width:10px 10px 0}.popover.top .arrow:after{bottom:-1px;left:-11px;border-top-color:rgba(0,0,0,0.25);border-width:11px 11px 0}.popover.right .arrow{top:50%;left:-10px;margin-top:-10px;border-right-color:#fff;border-width:10px 10px 10px 0}.popover.right .arrow:after{bottom:-11px;left:-1px;border-right-color:rgba(0,0,0,0.25);border-width:11px 11px 11px 0}.popover.bottom .arrow{top:-10px;left:50%;margin-left:-10px;border-bottom-color:#fff;border-width:0 10px 10px}.popover.bottom .arrow:after{top:-1px;left:-11px;border-bottom-color:rgba(0,0,0,0.25);border-width:0 11px 11px}.popover.left .arrow{top:50%;right:-10px;margin-top:-10px;border-left-color:#fff;border-width:10px 0 10px 10px}.popover.left .arrow:after{right:-1px;bottom:-11px;border-left-color:rgba(0,0,0,0.25);border-width:11px 0 11px 11px}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}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;color:#555}.label,.badge{font-size:11.844px;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}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-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:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-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(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#fff9f9f9',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{float:left;width:0;height:100%;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(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',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;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;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:-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);-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,.progress .bar-danger{background-color:#dd514c;background-image:-moz-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(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{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:-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,.progress .bar-success{background-color:#5eb95e;background-image:-moz-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(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{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:-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,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-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(to bottom,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff339bb9',GradientType=0)}.progress-info.progress-striped .bar,.progress-striped .bar-info{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:-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,.progress .bar-warning{background-color:#faa732;background-image:-moz-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(to bottom,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0)}.progress-warning.progress-striped .bar,.progress-striped .bar-warning{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:-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:20px}.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:20px;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;-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:15px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{line-height:20px;color:#fff}.carousel-caption h4{margin:0 0 5px}.carousel-caption p{margin-bottom:0}.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:30px;color:inherit}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden}.affix{position:fixed} From 4f422032e507d698b9c717b5228154d4527a639a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 31 Aug 2013 22:17:40 -0700 Subject: [PATCH 0901/2521] Update docs for new package --- docs/bagel-programming-guide.md | 6 +++--- docs/configuration.md | 18 +++++++++--------- docs/index.md | 8 ++++---- docs/java-programming-guide.md | 28 ++++++++++++++-------------- docs/quick-start.md | 10 +++++----- docs/scala-programming-guide.md | 16 ++++++++-------- docs/streaming-programming-guide.md | 28 ++++++++++++++-------------- 7 files changed, 57 insertions(+), 57 deletions(-) diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md index c526da3ca0b..20b1e9b26db 100644 --- a/docs/bagel-programming-guide.md +++ b/docs/bagel-programming-guide.md @@ -29,8 +29,8 @@ representing the current PageRank of the vertex, and similarly extend the `Message` and `Edge` classes. Note that these need to be marked `@serializable` to allow Spark to transfer them across machines. We also import the Bagel types and implicit conversions. {% highlight scala %} -import spark.bagel._ -import spark.bagel.Bagel._ +import org.apache.spark.bagel._ +import org.apache.spark.bagel.Bagel._ @serializable class PREdge(val targetId: String) extends Edge @@ -158,4 +158,4 @@ trait Message[K] { ## Where to Go from Here -Two example jobs, PageRank and shortest path, are included in `examples/src/main/scala/spark/examples/bagel`. You can run them by passing the class name to the `run-example` script included in Spark -- for example, `./run-example spark.examples.bagel.WikipediaPageRank`. Each example program prints usage help when run without any arguments. +Two example jobs, PageRank and shortest path, are included in `examples/src/main/scala/org/apache/spark/examples/bagel`. You can run them by passing the class name to the `run-example` script included in Spark -- for example, `./run-example org.apache.spark.examples.bagel.WikipediaPageRank`. Each example program prints usage help when run without any arguments. diff --git a/docs/configuration.md b/docs/configuration.md index 1c0492efb36..55df18b6fb8 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -36,13 +36,13 @@ there are at least five properties that you will commonly want to control: spark.serializer - spark.JavaSerializer + org.apache.spark.JavaSerializer Class to use for serializing objects that will be sent over the network or need to be cached in serialized form. The default of Java serialization works with any Serializable Java object but is - quite slow, so we recommend using spark.KryoSerializer + quite slow, so we recommend using org.apache.spark.KryoSerializer and configuring Kryo serialization when speed is necessary. Can be any subclass of - spark.Serializer. + org.apache.spark.Serializer. @@ -50,8 +50,8 @@ there are at least five properties that you will commonly want to control: (none) If you use Kryo serialization, set this class to register your custom classes with Kryo. - You need to set it to a class that extends - spark.KryoRegistrator. + It should be set to a class that extends + KryoRegistrator. See the tuning guide for more details. @@ -147,10 +147,10 @@ Apart from these, the following properties are also available, and may be useful spark.io.compression.codec - spark.io.SnappyCompressionCodec + org.apache.spark.io.
    SnappyCompressionCodec The compression codec class to use for various compressions. By default, Spark provides two - codecs: spark.io.LZFCompressionCodec and spark.io.SnappyCompressionCodec. + codecs: org.apache.spark.io.LZFCompressionCodec and org.apache.spark.io.SnappyCompressionCodec. @@ -171,7 +171,7 @@ Apart from these, the following properties are also available, and may be useful spark.closure.serializer - spark.JavaSerializer + org.apache.spark.JavaSerializer Serializer class to use for closures. Generally Java is fine unless your distributed functions (e.g. map functions) reference large objects in the driver program. @@ -198,7 +198,7 @@ Apart from these, the following properties are also available, and may be useful spark.broadcast.factory - spark.broadcast.HttpBroadcastFactory + org.apache.spark.broadcast.
    HttpBroadcastFactory Which broadcast implementation to use. diff --git a/docs/index.md b/docs/index.md index 0ea0e103e43..35a597d7bb1 100644 --- a/docs/index.md +++ b/docs/index.md @@ -3,8 +3,8 @@ layout: global title: Spark Overview --- -Apache Spark is a cluster computing system that aims to make data analytics faster to run and faster to write. -It provides high-level APIs in [Scala](scala-programming-guide.html), [Java](java-programming-guide.html), and [Python](python-programming-guide.html), and a general execution engine that supports rich operator graphs. +Apache Spark is a fast and general-purpose cluster computing system. +It provides high-level APIs in [Scala](scala-programming-guide.html), [Java](java-programming-guide.html), and [Python](python-programming-guide.html) that make parallel jobs easy to write, and an optimized engine that supports general computation graphs. Spark can run on the Apache Mesos cluster manager, Hadoop YARN, Amazon EC2, or without an independent resource manager ("standalone mode"). # Downloading @@ -24,8 +24,8 @@ For its Scala API, Spark {{site.SPARK_VERSION}} depends on Scala {{site.SCALA_VE Spark comes with several sample programs in the `examples` directory. To run one of the samples, use `./run-example ` in the top-level Spark directory (the `run-example` script sets up the appropriate paths and launches that program). -For example, `./run-example spark.examples.SparkPi` will run a sample program that estimates Pi. Each -example prints usage help if no params are given. +For example, try `./run-example org.apache.spark.examples.SparkPi local`. +Each example prints usage help when run with no parameters. Note that all of the sample programs take a `` parameter specifying the cluster URL to connect to. This can be a [URL for a distributed cluster](scala-programming-guide.html#master-urls), diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md index dd19a5f0c97..48bf36631ab 100644 --- a/docs/java-programming-guide.md +++ b/docs/java-programming-guide.md @@ -10,9 +10,9 @@ easy to follow even if you don't know Scala. This guide will show how to use the Spark features described there in Java. The Spark Java API is defined in the -[`spark.api.java`](api/core/index.html#spark.api.java.package) package, and includes -a [`JavaSparkContext`](api/core/index.html#spark.api.java.JavaSparkContext) for -initializing Spark and [`JavaRDD`](api/core/index.html#spark.api.java.JavaRDD) classes, +[`org.apache.spark.api.java`](api/core/index.html#org.apache.spark.api.java.package) package, and includes +a [`JavaSparkContext`](api/core/index.html#org.apache.spark.api.java.JavaSparkContext) for +initializing Spark and [`JavaRDD`](api/core/index.html#org.apache.spark.api.java.JavaRDD) classes, which support the same methods as their Scala counterparts but take Java functions and return Java data and collection types. The main differences have to do with passing functions to RDD operations (e.g. map) and handling RDDs of different types, as discussed next. @@ -23,12 +23,12 @@ There are a few key differences between the Java and Scala APIs: * Java does not support anonymous or first-class functions, so functions must be implemented by extending the - [`spark.api.java.function.Function`](api/core/index.html#spark.api.java.function.Function), - [`Function2`](api/core/index.html#spark.api.java.function.Function2), etc. + [`org.apache.spark.api.java.function.Function`](api/core/index.html#org.apache.spark.api.java.function.Function), + [`Function2`](api/core/index.html#org.apache.spark.api.java.function.Function2), etc. classes. * To maintain type safety, the Java API defines specialized Function and RDD classes for key-value pairs and doubles. For example, - [`JavaPairRDD`](api/core/index.html#spark.api.java.JavaPairRDD) + [`JavaPairRDD`](api/core/index.html#org.apache.spark.api.java.JavaPairRDD) stores key-value pairs. * RDD methods like `collect()` and `countByKey()` return Java collections types, such as `java.util.List` and `java.util.Map`. @@ -44,8 +44,8 @@ In the Scala API, these methods are automatically added using Scala's [implicit conversions](http://www.scala-lang.org/node/130) mechanism. In the Java API, the extra methods are defined in the -[`JavaPairRDD`](api/core/index.html#spark.api.java.JavaPairRDD) -and [`JavaDoubleRDD`](api/core/index.html#spark.api.java.JavaDoubleRDD) +[`JavaPairRDD`](api/core/index.html#org.apache.spark.api.java.JavaPairRDD) +and [`JavaDoubleRDD`](api/core/index.html#org.apache.spark.api.java.JavaDoubleRDD) classes. RDD methods like `map` are overloaded by specialized `PairFunction` and `DoubleFunction` classes, allowing them to return RDDs of the appropriate types. Common methods like `filter` and `sample` are implemented by @@ -75,7 +75,7 @@ class has a single abstract method, `call()`, that must be implemented. ## Storage Levels RDD [storage level](scala-programming-guide.html#rdd-persistence) constants, such as `MEMORY_AND_DISK`, are -declared in the [spark.api.java.StorageLevels](api/core/index.html#spark.api.java.StorageLevels) class. To +declared in the [org.apache.spark.api.java.StorageLevels](api/core/index.html#org.apache.spark.api.java.StorageLevels) class. To define your own storage level, you can use StorageLevels.create(...). @@ -92,8 +92,8 @@ The Java API supports other Spark features, including As an example, we will implement word count using the Java API. {% highlight java %} -import spark.api.java.*; -import spark.api.java.function.*; +import org.apache.spark.api.java.*; +import org.apache.spark.api.java.function.*; JavaSparkContext sc = new JavaSparkContext(...); JavaRDD lines = ctx.textFile("hdfs://..."); @@ -179,7 +179,7 @@ just a matter of style. # Javadoc We currently provide documentation for the Java API as Scaladoc, in the -[`spark.api.java` package](api/core/index.html#spark.api.java.package), because +[`org.apache.spark.api.java` package](api/core/index.html#org.apache.spark.api.java.package), because some of the classes are implemented in Scala. The main downside is that the types and function definitions show Scala syntax (for example, `def reduce(func: Function2[T, T]): T` instead of `T reduce(Function2 func)`). @@ -189,7 +189,7 @@ We hope to generate documentation with Java-style syntax in the future. # Where to Go from Here Spark includes several sample programs using the Java API in -[`examples/src/main/java`](https://github.com/mesos/spark/tree/master/examples/src/main/java/spark/examples). You can run them by passing the class name to the +[`examples/src/main/java`](https://github.com/mesos/spark/tree/master/examples/src/main/java/org/apache/spark/examples). You can run them by passing the class name to the `run-example` script included in Spark -- for example, `./run-example -spark.examples.JavaWordCount`. Each example program prints usage help when run +org.apache.spark.examples.JavaWordCount`. Each example program prints usage help when run without any arguments. diff --git a/docs/quick-start.md b/docs/quick-start.md index 4507b21c5ed..8cbc55bed28 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -108,7 +108,7 @@ We'll create a very simple Spark job in Scala. So simple, in fact, that it's nam {% highlight scala %} /*** SimpleJob.scala ***/ -import spark.SparkContext +import org.apache.spark.SparkContext import SparkContext._ object SimpleJob { @@ -135,7 +135,7 @@ version := "1.0" scalaVersion := "{{site.SCALA_VERSION}}" -libraryDependencies += "org.spark-project" %% "spark-core" % "{{site.SPARK_VERSION}}" +libraryDependencies += "org.apache.spark" %% "spark-core" % "{{site.SPARK_VERSION}}" resolvers += "Akka Repository" at "http://repo.akka.io/releases/" {% endhighlight %} @@ -170,8 +170,8 @@ We'll create a very simple Spark job, `SimpleJob.java`: {% highlight java %} /*** SimpleJob.java ***/ -import spark.api.java.*; -import spark.api.java.function.Function; +import org.apache.spark.api.java.*; +import org.apache.spark.api.java.function.Function; public class SimpleJob { public static void main(String[] args) { @@ -213,7 +213,7 @@ To build the job, we also write a Maven `pom.xml` file that lists Spark as a dep - org.spark-project + org.apache.spark spark-core_{{site.SCALA_VERSION}} {{site.SPARK_VERSION}} diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index e321b8f5b88..5aa2b64d33f 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -21,7 +21,7 @@ Spark {{site.SPARK_VERSION}} uses Scala {{site.SCALA_VERSION}}. If you write app To write a Spark application, you need to add a dependency on Spark. If you use SBT or Maven, Spark is available through Maven Central at: - groupId = org.spark-project + groupId = org.apache.spark artifactId = spark-core_{{site.SCALA_VERSION}} version = {{site.SPARK_VERSION}} @@ -36,7 +36,7 @@ For other build systems, you can run `sbt/sbt assembly` to pack Spark and its de Finally, you need to import some Spark classes and implicit conversions into your program. Add the following lines: {% highlight scala %} -import spark.SparkContext +import org.apache.spark.SparkContext import SparkContext._ {% endhighlight %} @@ -142,7 +142,7 @@ All transformations in Spark are lazy, in that they do not compute their By default, each transformed RDD is recomputed each time you run an action on it. However, you may also *persist* an RDD in memory using the `persist` (or `cache`) method, in which case Spark will keep the elements around on the cluster for much faster access the next time you query it. There is also support for persisting datasets on disk, or replicated across the cluster. The next section in this document describes these options. -The following tables list the transformations and actions currently supported (see also the [RDD API doc](api/core/index.html#spark.RDD) for details): +The following tables list the transformations and actions currently supported (see also the [RDD API doc](api/core/index.html#org.apache.spark.RDD) for details): ### Transformations @@ -211,7 +211,7 @@ The following tables list the transformations and actions currently supported (s -A complete list of transformations is available in the [RDD API doc](api/core/index.html#spark.RDD). +A complete list of transformations is available in the [RDD API doc](api/core/index.html#org.apache.spark.RDD). ### Actions @@ -259,7 +259,7 @@ A complete list of transformations is available in the [RDD API doc](api/core/in -A complete list of actions is available in the [RDD API doc](api/core/index.html#spark.RDD). +A complete list of actions is available in the [RDD API doc](api/core/index.html#org.apache.spark.RDD). ## RDD Persistence @@ -267,7 +267,7 @@ One of the most important capabilities in Spark is *persisting* (or *caching*) a You can mark an RDD to be persisted using the `persist()` or `cache()` methods on it. The first time it is computed in an action, it will be kept in memory on the nodes. The cache is fault-tolerant -- if any partition of an RDD is lost, it will automatically be recomputed using the transformations that originally created it. -In addition, each RDD can be stored using a different *storage level*, allowing you, for example, to persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), or even replicate it across nodes. These levels are chosen by passing a [`spark.storage.StorageLevel`](api/core/index.html#spark.storage.StorageLevel) object to `persist()`. The `cache()` method is a shorthand for using the default storage level, which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The complete set of available storage levels is: +In addition, each RDD can be stored using a different *storage level*, allowing you, for example, to persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), or even replicate it across nodes. These levels are chosen by passing a [`org.apache.spark.storage.StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel) object to `persist()`. The `cache()` method is a shorthand for using the default storage level, which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The complete set of available storage levels is: @@ -318,7 +318,7 @@ We recommend going through the following process to select one: application). *All* the storage levels provide full fault tolerance by recomputing lost data, but the replicated ones let you continue running tasks on the RDD without waiting to recompute a lost partition. -If you want to define your own storage level (say, with replication factor of 3 instead of 2), then use the function factor method `apply()` of the [`StorageLevel`](api/core/index.html#spark.storage.StorageLevel$) singleton object. +If you want to define your own storage level (say, with replication factor of 3 instead of 2), then use the function factor method `apply()` of the [`StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel$) singleton object. # Shared Variables @@ -364,7 +364,7 @@ res2: Int = 10 # Where to Go from Here You can see some [example Spark programs](http://www.spark-project.org/examples.html) on the Spark website. -In addition, Spark includes several sample programs in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `run-example` script included in Spark -- for example, `./run-example spark.examples.SparkPi`. Each example program prints usage help when run without any arguments. +In addition, Spark includes several sample programs in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `run-example` script included in Spark -- for example, `./run-example org.apache.spark.examples.SparkPi`. Each example program prints usage help when run without any arguments. For help on optimizing your program, the [configuration](configuration.html) and [tuning](tuning.html) guides provide information on best practices. They are especially important for diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 3330e635986..e59d93df4b9 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -34,7 +34,7 @@ ssc.textFileStream(directory) // Creates a stream by monitoring and process ssc.socketStream(hostname, port) // Creates a stream that uses a TCP socket to read data from hostname:port {% endhighlight %} -We also provide a input streams for Kafka, Flume, Akka actor, etc. For a complete list of input streams, take a look at the [StreamingContext API documentation](api/streaming/index.html#spark.streaming.StreamingContext). +We also provide a input streams for Kafka, Flume, Akka actor, etc. For a complete list of input streams, take a look at the [StreamingContext API documentation](api/streaming/index.html#org.apache.spark.streaming.StreamingContext). @@ -156,7 +156,7 @@ Spark Streaming features windowed computations, which allow you to apply transfo
    Storage LevelMeaning
    -A complete list of DStream operations is available in the API documentation of [DStream](api/streaming/index.html#spark.streaming.DStream) and [PairDStreamFunctions](api/streaming/index.html#spark.streaming.PairDStreamFunctions). +A complete list of DStream operations is available in the API documentation of [DStream](api/streaming/index.html#org.apache.spark.streaming.DStream) and [PairDStreamFunctions](api/streaming/index.html#org.apache.spark.streaming.PairDStreamFunctions). ## Output Operations When an output operator is called, it triggers the computation of a stream. Currently the following output operators are defined: @@ -206,8 +206,8 @@ ssc.stop() A simple example to start off is the [NetworkWordCount](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala). This example counts the words received from a network server every second. Given below is the relevant sections of the source code. You can find the full source code in `/streaming/src/main/scala/spark/streaming/examples/NetworkWordCount.scala` . {% highlight scala %} -import spark.streaming.{Seconds, StreamingContext} -import spark.streaming.StreamingContext._ +import org.apache.spark.streaming.{Seconds, StreamingContext} +import StreamingContext._ ... // Create the context and set up a network input stream to receive from a host:port @@ -234,7 +234,7 @@ $ nc -lk 9999 Then, in a different terminal, you can start NetworkWordCount by using {% highlight bash %} -$ ./run-example spark.streaming.examples.NetworkWordCount local[2] localhost 9999 +$ ./run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999 {% endhighlight %} This will make NetworkWordCount connect to the netcat server. Any lines typed in the terminal running the netcat server will be counted and printed on screen. @@ -272,7 +272,7 @@ Time: 1357008430000 ms -You can find more examples in `/streaming/src/main/scala/spark/streaming/examples/`. They can be run in the similar manner using `./run-example spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files. +You can find more examples in `/streaming/src/main/scala/spark/streaming/examples/`. They can be run in the similar manner using `./run-example org.apache.spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files. # DStream Persistence Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is, using `persist()` method on a DStream would automatically persist every RDD of that DStream in memory. This is useful if the data in the DStream will be computed multiple times (e.g., multiple operations on the same data). For window-based operations like `reduceByWindow` and `reduceByKeyAndWindow` and state-based operations like `updateStateByKey`, this is implicitly true. Hence, DStreams generated by window-based operations are automatically persisted in memory, without the developer calling `persist()`. @@ -315,7 +315,7 @@ Getting the best performance of a Spark Streaming application on a cluster requi There are a number of optimizations that can be done in Spark to minimize the processing time of each batch. These have been discussed in detail in [Tuning Guide](tuning.html). This section highlights some of the most important ones. ### Level of Parallelism -Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of parallelism as an argument (see the [`spark.PairDStreamFunctions`](api/streaming/index.html#spark.PairDStreamFunctions) documentation), or set the system property `spark.default.parallelism` to change the default. +Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of parallelism as an argument (see the [`PairDStreamFunctions`](api/streaming/index.html#org.apache.spark.PairDStreamFunctions) documentation), or set the system property `spark.default.parallelism` to change the default. ### Data Serialization The overhead of data serialization can be significant, especially when sub-second batch sizes are to be achieved. There are two aspects to it. @@ -345,7 +345,7 @@ This value is closely tied with any window operation that is being used. Any win ## Memory Tuning Tuning the memory usage and GC behavior of Spark applications have been discussed in great detail in the [Tuning Guide](tuning.html). It is recommended that you read that. In this section, we highlight a few customizations that are strongly recommended to minimize GC related pauses in Spark Streaming applications and achieving more consistent batch processing times. -* **Default persistence level of DStreams**: Unlike RDDs, the default persistence level of DStreams serializes the data in memory (that is, [StorageLevel.MEMORY_ONLY_SER](api/core/index.html#spark.storage.StorageLevel$) for DStream compared to [StorageLevel.MEMORY_ONLY](api/core/index.html#spark.storage.StorageLevel$) for RDDs). Even though keeping the data serialized incurs a higher serialization overheads, it significantly reduces GC pauses. +* **Default persistence level of DStreams**: Unlike RDDs, the default persistence level of DStreams serializes the data in memory (that is, [StorageLevel.MEMORY_ONLY_SER](api/core/index.html#org.apache.spark.storage.StorageLevel$) for DStream compared to [StorageLevel.MEMORY_ONLY](api/core/index.html#org.apache.spark.storage.StorageLevel$) for RDDs). Even though keeping the data serialized incurs a higher serialization overheads, it significantly reduces GC pauses. * **Concurrent garbage collector**: Using the concurrent mark-and-sweep GC further minimizes the variability of GC pauses. Even though concurrent GC is known to reduce the overall processing throughput of the system, its use is still recommended to achieve more consistent batch processing times. @@ -467,10 +467,10 @@ If the driver had crashed in the middle of the processing of time 3, then it wil # Java API -Similar to [Spark's Java API](java-programming-guide.html), we also provide a Java API for Spark Streaming which allows all its features to be accessible from a Java program. This is defined in [spark.streaming.api.java] (api/streaming/index.html#spark.streaming.api.java.package) package and includes [JavaStreamingContext](api/streaming/index.html#spark.streaming.api.java.JavaStreamingContext) and [JavaDStream](api/streaming/index.html#spark.streaming.api.java.JavaDStream) classes that provide the same methods as their Scala counterparts, but take Java functions (that is, Function, and Function2) and return Java data and collection types. Some of the key points to note are: +Similar to [Spark's Java API](java-programming-guide.html), we also provide a Java API for Spark Streaming which allows all its features to be accessible from a Java program. This is defined in [org.apache.spark.streaming.api.java] (api/streaming/index.html#org.apache.spark.streaming.api.java.package) package and includes [JavaStreamingContext](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) and [JavaDStream](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaDStream) classes that provide the same methods as their Scala counterparts, but take Java functions (that is, Function, and Function2) and return Java data and collection types. Some of the key points to note are: -1. Functions for transformations must be implemented as subclasses of [Function](api/core/index.html#spark.api.java.function.Function) and [Function2](api/core/index.html#spark.api.java.function.Function2) -1. Unlike the Scala API, the Java API handles DStreams for key-value pairs using a separate [JavaPairDStream](api/streaming/index.html#spark.streaming.api.java.JavaPairDStream) class(similar to [JavaRDD and JavaPairRDD](java-programming-guide.html#rdd-classes). DStream functions like `map` and `filter` are implemented separately by JavaDStreams and JavaPairDStream to return DStreams of appropriate types. +1. Functions for transformations must be implemented as subclasses of [Function](api/core/index.html#org.apache.spark.api.java.function.Function) and [Function2](api/core/index.html#org.apache.spark.api.java.function.Function2) +1. Unlike the Scala API, the Java API handles DStreams for key-value pairs using a separate [JavaPairDStream](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaPairDStream) class(similar to [JavaRDD and JavaPairRDD](java-programming-guide.html#rdd-classes). DStream functions like `map` and `filter` are implemented separately by JavaDStreams and JavaPairDStream to return DStreams of appropriate types. Spark's [Java Programming Guide](java-programming-guide.html) gives more ideas about using the Java API. To extends the ideas presented for the RDDs to DStreams, we present parts of the Java version of the same NetworkWordCount example presented above. The full source code is given at `/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java` @@ -482,7 +482,7 @@ JavaDStream lines = ssc.socketTextStream(ip, port); {% endhighlight %} -Then the `lines` are split into words by using the `flatMap` function and [FlatMapFunction](api/core/index.html#spark.api.java.function.FlatMapFunction). +Then the `lines` are split into words by using the `flatMap` function and [FlatMapFunction](api/core/index.html#org.apache.spark.api.java.function.FlatMapFunction). {% highlight java %} JavaDStream words = lines.flatMap( @@ -494,7 +494,7 @@ JavaDStream words = lines.flatMap( }); {% endhighlight %} -The `words` is then mapped to a [JavaPairDStream](api/streaming/index.html#spark.streaming.api.java.JavaPairDStream) of `(word, 1)` pairs using `map` and [PairFunction](api/core/index.html#spark.api.java.function.PairFunction). This is reduced by using `reduceByKey` and [Function2](api/core/index.html#spark.api.java.function.Function2). +The `words` is then mapped to a [JavaPairDStream](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaPairDStream) of `(word, 1)` pairs using `map` and [PairFunction](api/core/index.html#org.apache.spark.api.java.function.PairFunction). This is reduced by using `reduceByKey` and [Function2](api/core/index.html#org.apache.spark.api.java.function.Function2). {% highlight java %} JavaPairDStream wordCounts = words.map( @@ -515,6 +515,6 @@ JavaPairDStream wordCounts = words.map( # Where to Go from Here -* API docs - [Scala](api/streaming/index.html#spark.streaming.package) and [Java](api/streaming/index.html#spark.streaming.api.java.package) +* API docs - [Scala](api/streaming/index.html#org.apache.spark.streaming.package) and [Java](api/streaming/index.html#org.apache.spark.streaming.api.java.package) * More examples - [Scala](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/streaming/examples) and [Java](https://github.com/mesos/spark/tree/master/examples/src/main/java/spark/streaming/examples) * [Paper describing Spark Streaming](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) From 12495ec63ab56e4352bc466ac37c95921f685dc3 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 31 Aug 2013 22:17:46 -0700 Subject: [PATCH 0902/2521] Remove shutdown hook to stop jetty; this is unnecessary for releasing ports and creates noisy log messages --- core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index cfa18f6ea4c..7211dbc7c66 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -117,7 +117,6 @@ private[spark] object JettyUtils extends Logging { Try { server.start() } match { case s: Success[_] => - sys.addShutdownHook(server.stop()) // Be kind, un-bind (server, server.getConnectors.head.getLocalPort) case f: Failure[_] => server.stop() From bbaa9d7d6e2eef59ff58f97af69dbc5e1cdff82d Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 31 Aug 2013 22:38:32 -0700 Subject: [PATCH 0903/2521] Add banner to PySpark and make wordcount output nicer --- python/examples/wordcount.py | 2 +- python/pyspark/shell.py | 13 +++++++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/python/examples/wordcount.py b/python/examples/wordcount.py index a6de22766ab..b9139b9d765 100755 --- a/python/examples/wordcount.py +++ b/python/examples/wordcount.py @@ -32,4 +32,4 @@ .reduceByKey(add) output = counts.collect() for (word, count) in output: - print "%s : %i" % (word, count) + print "%s: %i" % (word, count) diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 9b4b4e78cb7..54823f80378 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -21,6 +21,7 @@ This file is designed to be launched as a PYTHONSTARTUP script. """ import os +import platform import pyspark from pyspark.context import SparkContext @@ -28,6 +29,18 @@ add_files = os.environ.get("ADD_FILES").split(',') if os.environ.get("ADD_FILES") != None else None sc = SparkContext(os.environ.get("MASTER", "local"), "PySparkShell", pyFiles=add_files) + +print """Welcome to + ____ __ + / __/__ ___ _____/ /__ + _\ \/ _ \/ _ `/ __/ '_/ + /__ / .__/\_,_/_/ /_/\_\ version 0.8.0 + /_/ +""" +print "Using Python version %s (%s, %s)" % ( + platform.python_version(), + platform.python_build()[0], + platform.python_build()[1]) print "Spark context avaiable as sc." if add_files != None: From d27cd03f3039999242739503071d8f73510db6a6 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 31 Aug 2013 22:38:50 -0700 Subject: [PATCH 0904/2521] Fix more URLs in docs --- docs/bagel-programming-guide.md | 8 ++++++-- docs/index.md | 2 +- docs/java-programming-guide.md | 9 ++++++--- docs/python-programming-guide.md | 7 +++++-- docs/scala-programming-guide.md | 6 +++++- docs/spark-debugger.md | 4 ++-- docs/streaming-programming-guide.md | 6 +++--- 7 files changed, 28 insertions(+), 14 deletions(-) diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md index 20b1e9b26db..f95627865a8 100644 --- a/docs/bagel-programming-guide.md +++ b/docs/bagel-programming-guide.md @@ -107,7 +107,7 @@ _Example_ ### Operations -Here are the actions and types in the Bagel API. See [Bagel.scala](https://github.com/mesos/spark/blob/master/bagel/src/main/scala/spark/bagel/Bagel.scala) for details. +Here are the actions and types in the Bagel API. See [Bagel.scala](https://github.com/apache/incubator-spark/blob/master/bagel/src/main/scala/spark/bagel/Bagel.scala) for details. #### Actions @@ -158,4 +158,8 @@ trait Message[K] { ## Where to Go from Here -Two example jobs, PageRank and shortest path, are included in `examples/src/main/scala/org/apache/spark/examples/bagel`. You can run them by passing the class name to the `run-example` script included in Spark -- for example, `./run-example org.apache.spark.examples.bagel.WikipediaPageRank`. Each example program prints usage help when run without any arguments. +Two example jobs, PageRank and shortest path, are included in `examples/src/main/scala/org/apache/spark/examples/bagel`. You can run them by passing the class name to the `run-example` script included in Spark; e.g.: + + ./run-example org.apache.spark.examples.bagel.WikipediaPageRank` + +Each example program prints usage help when run without any arguments. diff --git a/docs/index.md b/docs/index.md index 35a597d7bb1..3cf9cc1c641 100644 --- a/docs/index.md +++ b/docs/index.md @@ -95,7 +95,7 @@ In addition, if you wish to run Spark on [YARN](running-on-yarn.md), set exercises about Spark, Shark, Mesos, and more. [Videos](http://ampcamp.berkeley.edu/agenda-2012), [slides](http://ampcamp.berkeley.edu/agenda-2012) and [exercises](http://ampcamp.berkeley.edu/exercises-2012) are available online for free. -* [Code Examples](http://spark.incubator.apache.org/examples.html): more are also available in the [examples subfolder](https://github.com/mesos/spark/tree/master/examples/src/main/scala/) of Spark +* [Code Examples](http://spark.incubator.apache.org/examples.html): more are also available in the [examples subfolder](https://github.com/apache/incubator-spark/tree/master/examples/src/main/scala/) of Spark * [Paper Describing Spark](http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf) * [Paper Describing Spark Streaming](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md index 48bf36631ab..53085cc6719 100644 --- a/docs/java-programming-guide.md +++ b/docs/java-programming-guide.md @@ -189,7 +189,10 @@ We hope to generate documentation with Java-style syntax in the future. # Where to Go from Here Spark includes several sample programs using the Java API in -[`examples/src/main/java`](https://github.com/mesos/spark/tree/master/examples/src/main/java/org/apache/spark/examples). You can run them by passing the class name to the -`run-example` script included in Spark -- for example, `./run-example -org.apache.spark.examples.JavaWordCount`. Each example program prints usage help when run +[`examples/src/main/java`](https://github.com/apache/incubator-spark/tree/master/examples/src/main/java/org/apache/spark/examples). You can run them by passing the class name to the +`run-example` script included in Spark; for example: + + ./run-example org.apache.spark.examples.JavaWordCount + +Each example program prints usage help when run without any arguments. diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 27e0d10080d..8a539fe774f 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -135,8 +135,11 @@ Code dependencies can be added to an existing SparkContext using its `addPyFile( # Where to Go from Here -PySpark includes several sample programs in the [`python/examples` folder](https://github.com/mesos/spark/tree/master/python/examples). -You can run them by passing the files to the `pyspark` script -- for example `./pyspark python/examples/wordcount.py`. +PySpark includes several sample programs in the [`python/examples` folder](https://github.com/apache/incubator-spark/tree/master/python/examples). +You can run them by passing the files to the `pyspark` script; e.g.: + + ./pyspark python/examples/wordcount.py + Each program prints usage help when run without arguments. We currently provide [API documentation](api/pyspark/index.html) for the Python API as Epydoc. diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 5aa2b64d33f..2cf319a2632 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -364,7 +364,11 @@ res2: Int = 10 # Where to Go from Here You can see some [example Spark programs](http://www.spark-project.org/examples.html) on the Spark website. -In addition, Spark includes several sample programs in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `run-example` script included in Spark -- for example, `./run-example org.apache.spark.examples.SparkPi`. Each example program prints usage help when run without any arguments. +In addition, Spark includes several samples in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `run-example` script included in Spark; for example: + + ./run-example org.apache.spark.examples.SparkPi + +Each example program prints usage help when run without any arguments. For help on optimizing your program, the [configuration](configuration.html) and [tuning](tuning.html) guides provide information on best practices. They are especially important for diff --git a/docs/spark-debugger.md b/docs/spark-debugger.md index f6f0988858e..d6315d97f4d 100644 --- a/docs/spark-debugger.md +++ b/docs/spark-debugger.md @@ -2,7 +2,7 @@ layout: global title: The Spark Debugger --- -**Summary:** The Spark debugger provides replay debugging for deterministic (logic) errors in Spark programs. It's currently in development, but you can try it out in the [arthur branch](https://github.com/mesos/spark/tree/arthur). +**Summary:** The Spark debugger provides replay debugging for deterministic (logic) errors in Spark programs. It's currently in development, but you can try it out in the [arthur branch](https://github.com/apache/incubator-spark/tree/arthur). ## Introduction @@ -19,7 +19,7 @@ For deterministic errors, debugging a Spark program is now as easy as debugging ## Approach -As your Spark program runs, the slaves report key events back to the master -- for example, RDD creations, RDD contents, and uncaught exceptions. (A full list of event types is in [EventLogging.scala](https://github.com/mesos/spark/blob/arthur/core/src/main/scala/spark/EventLogging.scala).) The master logs those events, and you can load the event log into the debugger after your program is done running. +As your Spark program runs, the slaves report key events back to the master -- for example, RDD creations, RDD contents, and uncaught exceptions. (A full list of event types is in [EventLogging.scala](https://github.com/apache/incubator-spark/blob/arthur/core/src/main/scala/spark/EventLogging.scala).) The master logs those events, and you can load the event log into the debugger after your program is done running. _A note on nondeterminism:_ For fault recovery, Spark requires RDD transformations (for example, the function passed to `RDD.map`) to be deterministic. The Spark debugger also relies on this property, and it can also warn you if your transformation is nondeterministic. This works by checksumming the contents of each RDD and comparing the checksums from the original execution to the checksums after recomputing the RDD in the debugger. diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index e59d93df4b9..bc2f4f884f6 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -203,7 +203,7 @@ ssc.stop() {% endhighlight %} # Example -A simple example to start off is the [NetworkWordCount](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala). This example counts the words received from a network server every second. Given below is the relevant sections of the source code. You can find the full source code in `/streaming/src/main/scala/spark/streaming/examples/NetworkWordCount.scala` . +A simple example to start off is the [NetworkWordCount](https://github.com/apache/incubator-spark/tree/master/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala). This example counts the words received from a network server every second. Given below is the relevant sections of the source code. You can find the full source code in `/streaming/src/main/scala/spark/streaming/examples/NetworkWordCount.scala` . {% highlight scala %} import org.apache.spark.streaming.{Seconds, StreamingContext} @@ -513,8 +513,8 @@ JavaPairDStream wordCounts = words.map( {% endhighlight %} - # Where to Go from Here + * API docs - [Scala](api/streaming/index.html#org.apache.spark.streaming.package) and [Java](api/streaming/index.html#org.apache.spark.streaming.api.java.package) -* More examples - [Scala](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/streaming/examples) and [Java](https://github.com/mesos/spark/tree/master/examples/src/main/java/spark/streaming/examples) +* More examples - [Scala](https://github.com/apache/incubator-spark/tree/master/examples/src/main/scala/spark/streaming/examples) and [Java](https://github.com/apache/incubator-spark/tree/master/examples/src/main/java/spark/streaming/examples) * [Paper describing Spark Streaming](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) From debcf243892bcc7de3f41981e63b8daacb0f2e06 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 31 Aug 2013 22:41:50 -0700 Subject: [PATCH 0905/2521] Fix over-zealous find-and-replace in HTML --- docs/_layouts/global.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 349eb92a47f..84749fda4e6 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -100,7 +100,7 @@
  • Tuning Guide
  • Hardware Provisioning
  • Building Spark with Maven
  • -
  • Contributing to Spark
  • +
  • Contributing to Spark
  • From 5701eb92c7ac75176e0daebd3d551a07eea63cb5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 31 Aug 2013 22:45:53 -0700 Subject: [PATCH 0906/2521] Fix some URLs --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- docs/README.md | 2 +- docs/streaming-custom-receivers.md | 2 +- examples/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 4 ++-- project/SparkBuild.scala | 4 ++-- repl-bin/pom.xml | 2 +- repl/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 14 files changed, 16 insertions(+), 16 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index dc63811b76a..d19f44d292b 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-assembly Spark Project Assembly - http://spark-project.org/ + http://spark.incubator.apache.org/ diff --git a/bagel/pom.xml b/bagel/pom.xml index 9340991377a..51173c32b2c 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -29,7 +29,7 @@ spark-bagel jar Spark Project Bagel - http://spark-project.org/ + http://spark.incubator.apache.org/ diff --git a/core/pom.xml b/core/pom.xml index c803217f964..5738b7406f8 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -29,7 +29,7 @@ spark-core jar Spark Project Core - http://spark-project.org/ + http://spark.incubator.apache.org/ diff --git a/docs/README.md b/docs/README.md index c2b3497bb37..dfcf7535538 100644 --- a/docs/README.md +++ b/docs/README.md @@ -1,6 +1,6 @@ Welcome to the Spark documentation! -This readme will walk you through navigating and building the Spark documentation, which is included here with the Spark source code. You can also find documentation specific to release versions of Spark at http://spark-project.org/documentation.html. +This readme will walk you through navigating and building the Spark documentation, which is included here with the Spark source code. You can also find documentation specific to release versions of Spark at http://spark.incubator.apache.org/documentation.html. Read on to learn more about viewing documentation in plain text (i.e., markdown) or building the documentation yourself. Why build it yourself? So that you have the docs that corresponds to whichever version of Spark you currently have checked out of revision control. diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index dfa343bf94f..981cdfe0b2f 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -143,4 +143,4 @@ _A more comprehensive example is provided in the spark streaming examples_ ## References 1.[Akka Actor documentation](http://doc.akka.io/docs/akka/2.0.5/scala/actors.html) -2.[NetworkReceiver](http://spark-project.org/docs/latest/api/streaming/index.html#spark.streaming.dstream.NetworkReceiver) +2.[NetworkReceiver](http://spark.incubator.apache.org/docs/latest/api/streaming/index.html#spark.streaming.dstream.NetworkReceiver) diff --git a/examples/pom.xml b/examples/pom.xml index 13b55315113..224cf6c96c9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -29,7 +29,7 @@ spark-examples jar Spark Project Examples - http://spark-project.org/ + http://spark.incubator.apache.org/ diff --git a/mllib/pom.xml b/mllib/pom.xml index 2d5d3c00d10..966caf68355 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -29,7 +29,7 @@ spark-mllib jar Spark Project ML Library - http://spark-project.org/ + http://spark.incubator.apache.org/ diff --git a/pom.xml b/pom.xml index 9230611eaea..6a73e81da0b 100644 --- a/pom.xml +++ b/pom.xml @@ -41,8 +41,8 @@ Matei Zaharia matei.zaharia@gmail.com http://www.cs.berkeley.edu/~matei - U.C. Berkeley Computer Science - http://www.cs.berkeley.edu/ + Apache Software Foundation + http://spark.incubator.apache.org diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 18e86d2cae7..b1b99b37c4f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -121,8 +121,8 @@ object SparkBuild extends Build { Matei Zaharia matei.zaharia@gmail.com http://www.cs.berkeley.edu/~matei - U.C. Berkeley Computer Science - http://www.cs.berkeley.edu/ + Apache Software Foundation + http://spark.incubator.apache.org diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 6a1b09e8df7..d61b36a61a6 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -29,7 +29,7 @@ spark-repl-bin pom Spark Project REPL binary packaging - http://spark-project.org/ + http://spark.incubator.apache.org/ spark diff --git a/repl/pom.xml b/repl/pom.xml index f6276f18958..a1c87d76182 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -29,7 +29,7 @@ spark-repl jar Spark Project REPL - http://spark-project.org/ + http://spark.incubator.apache.org/ /usr/share/spark diff --git a/streaming/pom.xml b/streaming/pom.xml index c8946313ee9..7bea069b615 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -29,7 +29,7 @@ spark-streaming jar Spark Project Streaming - http://spark-project.org/ + http://spark.incubator.apache.org/ diff --git a/tools/pom.xml b/tools/pom.xml index 664bf7dcbdb..77646a68165 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -28,7 +28,7 @@ spark-tools jar Spark Project Tools - http://spark-project.org/ + http://spark.incubator.apache.org/ diff --git a/yarn/pom.xml b/yarn/pom.xml index a2afbeabff4..fcacdac9eb8 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -28,7 +28,7 @@ spark-yarn jar Spark Project YARN Support - http://spark-project.org/ + http://spark.incubator.apache.org/ target/scala-${scala.version}/classes From 5b4dea21439e86b61447bdb1613b2ddff9ffba9f Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 31 Aug 2013 23:01:50 -0700 Subject: [PATCH 0907/2521] More fixes --- docs/bagel-programming-guide.md | 27 ++++++++++--------- docs/streaming-custom-receivers.md | 40 ++++++++--------------------- docs/streaming-programming-guide.md | 12 +++++++-- docs/tuning.md | 13 +++++----- 4 files changed, 41 insertions(+), 51 deletions(-) diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md index f95627865a8..583684913d6 100644 --- a/docs/bagel-programming-guide.md +++ b/docs/bagel-programming-guide.md @@ -9,16 +9,15 @@ In the Pregel programming model, jobs run as a sequence of iterations called _su This guide shows the programming model and features of Bagel by walking through an example implementation of PageRank on Bagel. -## Linking with Bagel +# Linking with Bagel -To write a Bagel application, you will need to add Spark, its dependencies, and Bagel to your CLASSPATH: +To use Bagel in your program, add the following SBT or Maven dependency: -1. Run `sbt/sbt update` to fetch Spark's dependencies, if you haven't already done so. -2. Run `sbt/sbt assembly` to build Spark and its dependencies into one JAR (`core/target/spark-core-assembly-{{site.SPARK_VERSION}}.jar`) -3. Run `sbt/sbt package` build the Bagel JAR (`bagel/target/scala_{{site.SCALA_VERSION}}/spark-bagel_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}.jar`). -4. Add these two JARs to your CLASSPATH. + groupId = org.apache.spark + artifactId = spark-bagel_{{site.SCALA_VERSION}} + version = {{site.SPARK_VERSION}} -## Programming Model +# Programming Model Bagel operates on a graph represented as a [distributed dataset](scala-programming-guide.html) of (K, V) pairs, where keys are vertex IDs and values are vertices plus their associated state. In each superstep, Bagel runs a user-specified compute function on each vertex that takes as input the current vertex state and a list of messages sent to that vertex during the previous superstep, and returns the new vertex state and a list of outgoing messages. @@ -89,7 +88,7 @@ Finally, we print the results. println(result.map(v => "%s\t%s\n".format(v.id, v.rank)).collect.mkString) {% endhighlight %} -### Combiners +## Combiners Sending a message to another vertex generally involves expensive communication over the network. For certain algorithms, it's possible to reduce the amount of communication using _combiners_. For example, if the compute function receives integer messages and only uses their sum, it's possible for Bagel to combine multiple messages to the same vertex by summing them. @@ -97,7 +96,7 @@ For combiner support, Bagel can optionally take a set of combiner functions that _Example: PageRank with combiners_ -### Aggregators +## Aggregators Aggregators perform a reduce across all vertices after each superstep, and provide the result to each vertex in the next superstep. @@ -105,11 +104,11 @@ For aggregator support, Bagel can optionally take an aggregator function that re _Example_ -### Operations +## Operations Here are the actions and types in the Bagel API. See [Bagel.scala](https://github.com/apache/incubator-spark/blob/master/bagel/src/main/scala/spark/bagel/Bagel.scala) for details. -#### Actions +### Actions {% highlight scala %} /*** Full form ***/ @@ -133,7 +132,7 @@ Bagel.run(sc, vertices, messages, numSplits)(compute) // and returns (newVertex: V, outMessages: Array[M]) {% endhighlight %} -#### Types +### Types {% highlight scala %} trait Combiner[M, C] { @@ -156,10 +155,10 @@ trait Message[K] { } {% endhighlight %} -## Where to Go from Here +# Where to Go from Here Two example jobs, PageRank and shortest path, are included in `examples/src/main/scala/org/apache/spark/examples/bagel`. You can run them by passing the class name to the `run-example` script included in Spark; e.g.: - ./run-example org.apache.spark.examples.bagel.WikipediaPageRank` + ./run-example org.apache.spark.examples.bagel.WikipediaPageRank Each example program prints usage help when run without any arguments. diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index 981cdfe0b2f..4e27d6559cf 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -1,24 +1,22 @@ --- layout: global -title: Tutorial - Spark Streaming, Plugging in a custom receiver. +title: Spark Streaming Custom Receivers --- A "Spark Streaming" receiver can be a simple network stream, streams of messages from a message queue, files etc. A receiver can also assume roles more than just receiving data like filtering, preprocessing, to name a few of the possibilities. The api to plug-in any user defined custom receiver is thus provided to encourage development of receivers which may be well suited to ones specific need. This guide shows the programming model and features by walking through a simple sample receiver and corresponding Spark Streaming application. -### Write a simple receiver +### Writing a Simple Receiver -This starts with implementing [NetworkReceiver](#References) +This starts with implementing [NetworkReceiver](api/streaming/index.html#org.apache.spark.streaming.dstream.NetworkReceiver). -Following is a simple socket text-stream receiver. +The following is a simple socket text-stream receiver. {% highlight scala %} - - class SocketTextStreamReceiver(host: String, - port: Int - ) extends NetworkReceiver[String] { - + class SocketTextStreamReceiver(host: String, port: Int( + extends NetworkReceiver[String] + { protected lazy val blocksGenerator: BlockGenerator = new BlockGenerator(StorageLevel.MEMORY_ONLY_SER_2) @@ -36,23 +34,20 @@ Following is a simple socket text-stream receiver. protected def onStop() { blocksGenerator.stop() } - } - {% endhighlight %} All we did here is extended NetworkReceiver and called blockGenerator's API method (i.e. +=) to push our blocks of data. Please refer to scala-docs of NetworkReceiver for more details. -### An Actor as Receiver. +### An Actor as Receiver This starts with implementing [Actor](#References) Following is a simple socket text-stream receiver, which is appearently overly simplified using Akka's socket.io api. {% highlight scala %} - class SocketTextStreamReceiver (host:String, port:Int, bytesToString: ByteString => String) extends Actor with Receiver { @@ -64,52 +59,41 @@ Following is a simple socket text-stream receiver, which is appearently overly s } } - - {% endhighlight %} All we did here is mixed in trait Receiver and called pushBlock api method to push our blocks of data. Please refer to scala-docs of Receiver for more details. -### A sample spark application +### A Sample Spark Application * First create a Spark streaming context with master url and batchduration. {% highlight scala %} - val ssc = new StreamingContext(master, "WordCountCustomStreamSource", Seconds(batchDuration)) - {% endhighlight %} * Plug-in the custom receiver into the spark streaming context and create a DStream. {% highlight scala %} - val lines = ssc.networkStream[String](new SocketTextStreamReceiver( "localhost", 8445)) - {% endhighlight %} * OR Plug-in the actor as receiver into the spark streaming context and create a DStream. {% highlight scala %} - val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver( "localhost",8445, z => z.utf8String)),"SocketReceiver") - {% endhighlight %} * Process it. {% highlight scala %} - val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) wordCounts.print() ssc.start() - - {% endhighlight %} * After processing it, stream can be tested using the netcat utility. @@ -119,12 +103,11 @@ All we did here is mixed in trait Receiver and called pushBlock api method to pu hello hello -## Multiple homogeneous/heterogeneous receivers. +## Multiple Homogeneous/Heterogeneous Receivers. A DStream union operation is provided for taking union on multiple input streams. {% highlight scala %} - val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver( "localhost",8445, z => z.utf8String)),"SocketReceiver") @@ -133,7 +116,6 @@ A DStream union operation is provided for taking union on multiple input streams "localhost",8446, z => z.utf8String)),"SocketReceiver") val union = lines.union(lines2) - {% endhighlight %} Above stream can be easily process as described earlier. @@ -143,4 +125,4 @@ _A more comprehensive example is provided in the spark streaming examples_ ## References 1.[Akka Actor documentation](http://doc.akka.io/docs/akka/2.0.5/scala/actors.html) -2.[NetworkReceiver](http://spark.incubator.apache.org/docs/latest/api/streaming/index.html#spark.streaming.dstream.NetworkReceiver) +2.[NetworkReceiver](api/streaming/index.html#org.apache.spark.streaming.dstream.NetworkReceiver) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index bc2f4f884f6..c7df1720249 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -13,6 +13,14 @@ A Spark Streaming application is very similar to a Spark application; it consist This guide shows some how to start programming with DStreams. +# Linking with Spark Streaming + +Add the following SBT or Maven dependency to your project to use Spark Streaming: + + groupId = org.apache.spark + artifactId = spark-streaming_{{site.SCALA_VERSION}} + version = {{site.SPARK_VERSION}} + # Initializing Spark Streaming The first thing a Spark Streaming program must do is create a `StreamingContext` object, which tells Spark how to access a cluster. A `StreamingContext` can be created by using @@ -301,8 +309,8 @@ dstream.checkpoint(checkpointInterval) // checkpointInterval must be a multiple For DStreams that must be checkpointed (that is, DStreams created by `updateStateByKey` and `reduceByKeyAndWindow` with inverse function), the checkpoint interval of the DStream is by default set to a multiple of the DStream's sliding interval such that its at least 10 seconds. -## Customizing Receiver -Spark comes with a built in support for most common usage scenarios where input stream source can be either a network socket stream to support for a few message queues. Apart from that it is also possible to supply your own custom receiver via a convenient API. Find more details at [Custom Receiver Guide](streaming-custom-receivers.html) +## Custom Receivers +Spark comes with a built in support for most common usage scenarios where input stream source can be either a network socket stream to support for a few message queues. Apart from that it is also possible to supply your own custom receiver via a convenient API. Find more details at [Custom Receiver Guide](streaming-custom-receivers.html). # Performance Tuning Getting the best performance of a Spark Streaming application on a cluster requires a bit of tuning. This section explains a number of the parameters and configurations that can tuned to improve the performance of you application. At a high level, you need to consider two things: diff --git a/docs/tuning.md b/docs/tuning.md index 5ffca54481b..3563d110c9d 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -32,24 +32,25 @@ in your operations) and performance. It provides two serialization libraries: [`java.io.Externalizable`](http://docs.oracle.com/javase/6/docs/api/java/io/Externalizable.html). Java serialization is flexible but often quite slow, and leads to large serialized formats for many classes. -* [Kryo serialization](http://code.google.com/p/kryo/wiki/V1Documentation): Spark can also use +* [Kryo serialization](http://code.google.com/p/kryo/): Spark can also use the Kryo library (version 2) to serialize objects more quickly. Kryo is significantly faster and more compact than Java serialization (often as much as 10x), but does not support all `Serializable` types and requires you to *register* the classes you'll use in the program in advance for best performance. -You can switch to using Kryo by calling `System.setProperty("spark.serializer", "spark.KryoSerializer")` +You can switch to using Kryo by calling `System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer")` *before* creating your SparkContext. The only reason it is not the default is because of the custom registration requirement, but we recommend trying it in any network-intensive application. Finally, to register your classes with Kryo, create a public class that extends -[`spark.KryoRegistrator`](api/core/index.html#spark.KryoRegistrator) and set the +[`org.apache.spark.KryoRegistrator`](api/core/index.html#org.apache.spark.KryoRegistrator) and set the `spark.kryo.registrator` system property to point to it, as follows: {% highlight scala %} import com.esotericsoftware.kryo.Kryo +import org.apache.spark.KryoRegistrator -class MyRegistrator extends spark.KryoRegistrator { +class MyRegistrator extends KryoRegistrator { override def registerClasses(kryo: Kryo) { kryo.register(classOf[MyClass1]) kryo.register(classOf[MyClass2]) @@ -57,7 +58,7 @@ class MyRegistrator extends spark.KryoRegistrator { } // Make sure to set these properties *before* creating a SparkContext! -System.setProperty("spark.serializer", "spark.KryoSerializer") +System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer") System.setProperty("spark.kryo.registrator", "mypackage.MyRegistrator") val sc = new SparkContext(...) {% endhighlight %} @@ -216,7 +217,7 @@ enough. Spark automatically sets the number of "map" tasks to run on each file a (though you can control it through optional parameters to `SparkContext.textFile`, etc), and for distributed "reduce" operations, such as `groupByKey` and `reduceByKey`, it uses the largest parent RDD's number of partitions. You can pass the level of parallelism as a second argument -(see the [`spark.PairRDDFunctions`](api/core/index.html#spark.PairRDDFunctions) documentation), +(see the [`spark.PairRDDFunctions`](api/core/index.html#org.apache.spark.PairRDDFunctions) documentation), or set the system property `spark.default.parallelism` to change the default. In general, we recommend 2-3 tasks per CPU core in your cluster. From 0a8cc309211c62f8824d76618705c817edcf2424 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 1 Sep 2013 00:32:28 -0700 Subject: [PATCH 0908/2521] Move some classes to more appropriate packages: * RDD, *RDDFunctions -> org.apache.spark.rdd * Utils, ClosureCleaner, SizeEstimator -> org.apache.spark.util * JavaSerializer, KryoSerializer -> org.apache.spark.serializer --- .../scala/org/apache/spark/bagel/Bagel.scala | 2 +- .../scala/org/apache/spark/Accumulators.scala | 1 + .../scala/org/apache/spark/CacheManager.scala | 1 + .../scala/org/apache/spark/Dependency.scala | 2 + .../org/apache/spark/HttpFileServer.scala | 1 + .../scala/org/apache/spark/HttpServer.scala | 1 + .../org/apache/spark/MapOutputTracker.scala | 2 +- .../scala/org/apache/spark/Partitioner.scala | 3 + .../scala/org/apache/spark/SparkContext.scala | 8 +- .../scala/org/apache/spark/SparkEnv.scala | 6 +- .../apache/spark/api/java/JavaDoubleRDD.scala | 2 +- .../apache/spark/api/java/JavaPairRDD.scala | 2 +- .../org/apache/spark/api/java/JavaRDD.scala | 1 + .../apache/spark/api/java/JavaRDDLike.scala | 6 +- .../spark/api/java/JavaSparkContext.scala | 6 +- .../spark/api/python/PythonPartitioner.scala | 2 +- .../apache/spark/api/python/PythonRDD.scala | 2 + .../spark/broadcast/BitTorrentBroadcast.scala | 1 + .../spark/broadcast/HttpBroadcast.scala | 4 +- .../apache/spark/broadcast/MultiTracker.scala | 1 + .../spark/broadcast/TreeBroadcast.scala | 1 + .../apache/spark/deploy/DeployMessage.scala | 2 +- .../spark/deploy/LocalSparkCluster.scala | 4 +- .../spark/deploy/client/TestClient.scala | 4 +- .../apache/spark/deploy/master/Master.scala | 4 +- .../spark/deploy/master/MasterArguments.scala | 3 +- .../spark/deploy/master/WorkerInfo.scala | 2 +- .../deploy/master/ui/ApplicationPage.scala | 2 +- .../spark/deploy/master/ui/IndexPage.scala | 2 +- .../spark/deploy/master/ui/MasterWebUI.scala | 3 +- .../spark/deploy/worker/ExecutorRunner.scala | 3 +- .../apache/spark/deploy/worker/Worker.scala | 4 +- .../spark/deploy/worker/WorkerArguments.scala | 4 +- .../spark/deploy/worker/ui/IndexPage.scala | 2 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 3 +- .../org/apache/spark/executor/Executor.scala | 1 + .../spark/executor/MesosExecutorBackend.scala | 3 +- .../executor/StandaloneExecutorBackend.scala | 4 +- .../spark/network/ConnectionManager.scala | 1 + .../spark/network/ConnectionManagerId.scala | 2 +- .../main/scala/org/apache/spark/package.scala | 21 +-- .../partial/ApproximateActionListener.scala | 1 + .../scala/org/apache/spark/rdd/BlockRDD.scala | 2 +- .../org/apache/spark/rdd/CoGroupedRDD.scala | 2 +- .../spark/{ => rdd}/DoubleRDDFunctions.scala | 5 +- .../scala/org/apache/spark/rdd/EmptyRDD.scala | 2 +- .../org/apache/spark/rdd/FilteredRDD.scala | 2 +- .../org/apache/spark/rdd/FlatMappedRDD.scala | 2 +- .../spark/rdd/FlatMappedValuesRDD.scala | 2 +- .../org/apache/spark/rdd/GlommedRDD.scala | 2 +- .../org/apache/spark/rdd/HadoopRDD.scala | 8 +- .../scala/org/apache/spark/rdd/JdbcRDD.scala | 2 +- .../apache/spark/rdd/MapPartitionsRDD.scala | 2 +- .../spark/rdd/MapPartitionsWithIndexRDD.scala | 2 +- .../org/apache/spark/rdd/MappedRDD.scala | 2 +- .../apache/spark/rdd/MappedValuesRDD.scala | 2 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 2 +- .../spark/rdd/OrderedRDDFunctions.scala | 7 +- .../spark/{ => rdd}/PairRDDFunctions.scala | 31 +++-- .../spark/rdd/ParallelCollectionRDD.scala | 2 + .../spark/rdd/PartitionPruningRDD.scala | 2 +- .../scala/org/apache/spark/rdd/PipedRDD.scala | 2 +- .../org/apache/spark/{ => rdd}/RDD.scala | 35 ++--- .../spark/{ => rdd}/RDDCheckpointData.scala | 7 +- .../org/apache/spark/rdd/SampledRDD.scala | 2 +- .../{ => rdd}/SequenceFileRDDFunctions.scala | 24 +--- .../org/apache/spark/rdd/ShuffledRDD.scala | 2 +- .../org/apache/spark/rdd/SubtractedRDD.scala | 1 - .../scala/org/apache/spark/rdd/UnionRDD.scala | 2 +- .../spark/rdd/ZippedPartitionsRDD.scala | 2 +- .../org/apache/spark/rdd/ZippedRDD.scala | 2 +- .../apache/spark/scheduler/DAGScheduler.scala | 1 + .../spark/scheduler/DAGSchedulerEvent.scala | 1 + .../apache/spark/scheduler/JobLogger.scala | 1 + .../apache/spark/scheduler/ResultTask.scala | 7 +- .../spark/scheduler/ShuffleMapTask.scala | 2 + .../spark/scheduler/SparkListener.scala | 4 +- .../org/apache/spark/scheduler/Stage.scala | 3 +- .../apache/spark/scheduler/TaskResult.scala | 3 +- .../cluster/ClusterTaskSetManager.scala | 2 +- .../scheduler/cluster/SchedulerBackend.scala | 2 +- .../cluster/SparkDeploySchedulerBackend.scala | 3 +- .../cluster/StandaloneClusterMessage.scala | 3 +- .../cluster/StandaloneSchedulerBackend.scala | 3 +- .../spark/scheduler/cluster/TaskInfo.scala | 2 +- .../scheduler/local/LocalScheduler.scala | 1 + .../mesos/CoarseMesosSchedulerBackend.scala | 2 +- .../mesos/MesosSchedulerBackend.scala | 3 +- .../{ => serializer}/JavaSerializer.scala | 3 +- .../{ => serializer}/KryoSerializer.scala | 125 +++++++++--------- .../spark/storage/BlockFetcherIterator.scala | 2 +- .../apache/spark/storage/BlockManager.scala | 4 +- .../apache/spark/storage/BlockManagerId.scala | 2 +- .../storage/BlockManagerMasterActor.scala | 3 +- .../spark/storage/BlockManagerWorker.scala | 3 +- .../org/apache/spark/storage/DiskStore.scala | 2 +- .../apache/spark/storage/MemoryStore.scala | 2 +- .../apache/spark/storage/StorageUtils.scala | 3 +- .../apache/spark/storage/ThreadingTest.scala | 2 +- .../scala/org/apache/spark/ui/SparkUI.scala | 3 +- .../apache/spark/ui/exec/ExecutorsUI.scala | 3 +- .../spark/ui/jobs/JobProgressListener.scala | 2 +- .../apache/spark/ui/jobs/JobProgressUI.scala | 3 +- .../org/apache/spark/ui/jobs/StagePage.scala | 4 +- .../org/apache/spark/ui/jobs/StageTable.scala | 2 +- .../apache/spark/ui/storage/IndexPage.scala | 2 +- .../org/apache/spark/ui/storage/RDDPage.scala | 2 +- .../spark/{ => util}/ClosureCleaner.scala | 3 +- .../org/apache/spark/util/MemoryParam.scala | 2 - .../spark/{ => util}/SizeEstimator.scala | 3 +- .../org/apache/spark/{ => util}/Utils.scala | 5 +- .../org/apache/spark/CheckpointSuite.scala | 1 + .../scala/org/apache/spark/DriverSuite.scala | 1 + .../scala/org/apache/spark/FailureSuite.scala | 1 + .../spark/PartitionPruningRDDSuite.scala | 2 +- .../org/apache/spark/PartitioningSuite.scala | 9 +- .../scala/org/apache/spark/ShuffleSuite.scala | 3 +- .../{ => rdd}/PairRDDFunctionsSuite.scala | 3 +- .../org/apache/spark/{ => rdd}/RDDSuite.scala | 4 +- .../apache/spark/{ => rdd}/SortingSuite.scala | 6 +- .../spark/scheduler/DAGSchedulerSuite.scala | 2 +- .../spark/scheduler/JobLoggerSuite.scala | 6 +- .../spark/scheduler/TaskContextSuite.scala | 2 +- .../cluster/ClusterTaskSetManagerSuite.scala | 2 +- .../KryoSerializerSuite.scala | 12 +- .../spark/storage/BlockManagerSuite.scala | 8 +- .../{ => util}/ClosureCleanerSuite.scala | 5 +- .../spark/{ => util}/SizeEstimatorSuite.scala | 2 +- .../apache/spark/{ => util}/UtilsSuite.scala | 2 +- docs/configuration.md | 10 +- docs/quick-start.md | 2 +- docs/scala-programming-guide.md | 2 +- docs/tuning.md | 10 +- .../spark/examples/bagel/PageRankUtils.scala | 1 + .../examples/bagel/WikipediaPageRank.scala | 2 +- .../bagel/WikipediaPageRankStandalone.scala | 17 +-- .../streaming/examples/QueueStream.scala | 2 +- .../streaming/examples/RawNetworkGrep.scala | 2 +- .../classification/ClassificationModel.scala | 2 +- .../classification/LogisticRegression.scala | 3 +- .../spark/mllib/classification/SVM.scala | 3 +- .../spark/mllib/clustering/KMeans.scala | 3 +- .../spark/mllib/clustering/KMeansModel.scala | 2 +- .../mllib/optimization/GradientDescent.scala | 8 +- .../spark/mllib/optimization/Optimizer.scala | 2 +- .../spark/mllib/recommendation/ALS.scala | 7 +- .../MatrixFactorizationModel.scala | 2 +- .../GeneralizedLinearAlgorithm.scala | 3 +- .../apache/spark/mllib/regression/Lasso.scala | 3 +- .../mllib/regression/LinearRegression.scala | 3 +- .../mllib/regression/RegressionModel.scala | 2 +- .../mllib/regression/RidgeRegression.scala | 3 +- .../spark/mllib/util/DataValidators.scala | 3 +- .../mllib/util/KMeansDataGenerator.scala | 3 +- .../mllib/util/LinearDataGenerator.scala | 3 +- .../LogisticRegressionDataGenerator.scala | 3 +- .../spark/mllib/util/MFDataGenerator.scala | 3 +- .../org/apache/spark/mllib/util/MLUtils.scala | 3 +- .../spark/mllib/util/SVMDataGenerator.scala | 5 +- python/pyspark/context.py | 4 +- .../org/apache/spark/repl/SparkIMain.scala | 2 +- .../org/apache/spark/streaming/DStream.scala | 3 +- .../org/apache/spark/streaming/Duration.scala | 2 +- .../streaming/PairDStreamFunctions.scala | 9 +- .../spark/streaming/StreamingContext.scala | 1 + .../streaming/api/java/JavaDStream.scala | 2 +- .../streaming/api/java/JavaDStreamLike.scala | 2 +- .../streaming/api/java/JavaPairDStream.scala | 7 +- .../api/java/JavaStreamingContext.scala | 25 ++-- .../streaming/dstream/CoGroupedDStream.scala | 3 +- .../dstream/ConstantInputDStream.scala | 2 +- .../streaming/dstream/FileInputDStream.scala | 2 +- .../streaming/dstream/FilteredDStream.scala | 2 +- .../dstream/FlatMapValuedDStream.scala | 2 +- .../streaming/dstream/FlatMappedDStream.scala | 2 +- .../streaming/dstream/FlumeInputDStream.scala | 15 +-- .../streaming/dstream/ForEachDStream.scala | 2 +- .../streaming/dstream/GlommedDStream.scala | 2 +- .../dstream/MapPartitionedDStream.scala | 2 +- .../streaming/dstream/MapValuedDStream.scala | 2 +- .../streaming/dstream/MappedDStream.scala | 2 +- .../dstream/NetworkInputDStream.scala | 15 +-- .../streaming/dstream/QueueInputDStream.scala | 2 +- .../dstream/ReducedWindowedDStream.scala | 2 +- .../streaming/dstream/ShuffledDStream.scala | 3 +- .../streaming/dstream/StateDStream.scala | 2 +- .../dstream/TransformedDStream.scala | 2 +- .../streaming/dstream/UnionDStream.scala | 2 +- .../streaming/dstream/WindowedDStream.scala | 2 +- .../streaming/util/MasterFailureTest.scala | 3 +- .../spark/streaming/util/RawTextSender.scala | 3 +- .../spark/streaming/TestSuiteBase.scala | 5 +- .../tools/JavaAPICompletenessChecker.scala | 38 +++--- 193 files changed, 462 insertions(+), 418 deletions(-) rename core/src/main/scala/org/apache/spark/{ => rdd}/DoubleRDDFunctions.scala (94%) rename core/src/main/scala/org/apache/spark/{ => rdd}/PairRDDFunctions.scala (97%) rename core/src/main/scala/org/apache/spark/{ => rdd}/RDD.scala (96%) rename core/src/main/scala/org/apache/spark/{ => rdd}/RDDCheckpointData.scala (96%) rename core/src/main/scala/org/apache/spark/{ => rdd}/SequenceFileRDDFunctions.scala (84%) rename core/src/main/scala/org/apache/spark/{ => serializer}/JavaSerializer.scala (95%) rename core/src/main/scala/org/apache/spark/{ => serializer}/KryoSerializer.scala (93%) rename core/src/main/scala/org/apache/spark/{ => util}/ClosureCleaner.scala (99%) rename core/src/main/scala/org/apache/spark/{ => util}/SizeEstimator.scala (99%) rename core/src/main/scala/org/apache/spark/{ => util}/Utils.scala (99%) rename core/src/test/scala/org/apache/spark/{ => rdd}/PairRDDFunctionsSuite.scala (99%) rename core/src/test/scala/org/apache/spark/{ => rdd}/RDDSuite.scala (99%) rename core/src/test/scala/org/apache/spark/{ => rdd}/SortingSuite.scala (97%) rename core/src/test/scala/org/apache/spark/{ => serializer}/KryoSerializerSuite.scala (96%) rename core/src/test/scala/org/apache/spark/{ => util}/ClosureCleanerSuite.scala (98%) rename core/src/test/scala/org/apache/spark/{ => util}/SizeEstimatorSuite.scala (99%) rename core/src/test/scala/org/apache/spark/{ => util}/UtilsSuite.scala (99%) diff --git a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala index fec8737fcdf..44e26bbb9e0 100644 --- a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala @@ -19,7 +19,7 @@ package org.apache.spark.bagel import org.apache.spark._ import org.apache.spark.SparkContext._ - +import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel object Bagel extends Logging { diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 5177ee58faa..6e922a612a0 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -21,6 +21,7 @@ import java.io._ import scala.collection.mutable.Map import scala.collection.generic.Growable +import org.apache.spark.serializer.JavaSerializer /** * A datatype that can be accumulated, i.e. has an commutative and associative "add" operation, diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 42e465b9d86..e299a106ee6 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -19,6 +19,7 @@ package org.apache.spark import scala.collection.mutable.{ArrayBuffer, HashSet} import org.apache.spark.storage.{BlockManager, StorageLevel} +import org.apache.spark.rdd.RDD /** Spark class responsible for passing RDDs split contents to the BlockManager and making diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index cc3c2474a6d..cc30105940d 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -17,6 +17,8 @@ package org.apache.spark +import org.apache.spark.rdd.RDD + /** * Base class for dependencies. */ diff --git a/core/src/main/scala/org/apache/spark/HttpFileServer.scala b/core/src/main/scala/org/apache/spark/HttpFileServer.scala index 9b3a8966489..ad1ee20045f 100644 --- a/core/src/main/scala/org/apache/spark/HttpFileServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpFileServer.scala @@ -19,6 +19,7 @@ package org.apache.spark import java.io.{File} import com.google.common.io.Files +import org.apache.spark.util.Utils private[spark] class HttpFileServer extends Logging { diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index db36c7c9dd1..cdfc9dd54e0 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -26,6 +26,7 @@ import org.eclipse.jetty.server.handler.DefaultHandler import org.eclipse.jetty.server.handler.HandlerList import org.eclipse.jetty.server.handler.ResourceHandler import org.eclipse.jetty.util.thread.QueuedThreadPool +import org.apache.spark.util.Utils /** * Exception type thrown by HttpServer when it is in the wrong state for an operation. diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 0f422d910a1..ae7cf2a8930 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -32,7 +32,7 @@ import akka.util.Duration import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap} +import org.apache.spark.util.{Utils, MetadataCleaner, TimeStampedHashMap} private[spark] sealed trait MapOutputTrackerMessage diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 4dce2607b0a..0e2c987a598 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -17,6 +17,9 @@ package org.apache.spark +import org.apache.spark.util.Utils +import org.apache.spark.rdd.RDD + /** * An object that defines how the elements in a key-value pair RDD are partitioned by key. * Maps each key to a partition ID, from 0 to `numPartitions - 1`. diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 1207b242bc0..faf0c2362a2 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -54,17 +54,15 @@ import org.apache.mesos.MesosNativeLibrary import org.apache.spark.deploy.LocalSparkCluster import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} -import org.apache.spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD, - OrderedRDDFunctions} +import org.apache.spark.rdd._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend, ClusterScheduler, Schedulable, SchedulingMode} import org.apache.spark.scheduler.local.LocalScheduler import org.apache.spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} -import org.apache.spark.storage.{StorageStatus, StorageUtils, RDDInfo, BlockManagerSource} +import org.apache.spark.storage.{StorageUtils, BlockManagerSource} import org.apache.spark.ui.SparkUI -import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap} -import scala.Some +import org.apache.spark.util.{ClosureCleaner, Utils, MetadataCleaner, TimeStampedHashMap} import org.apache.spark.scheduler.StageInfo import org.apache.spark.storage.RDDInfo import org.apache.spark.storage.StorageStatus diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 6e6fe5df6b6..478e5a0aaf2 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -29,7 +29,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.storage.{BlockManagerMasterActor, BlockManager, BlockManagerMaster} import org.apache.spark.network.ConnectionManager import org.apache.spark.serializer.{Serializer, SerializerManager} -import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.api.python.PythonWorkerFactory @@ -155,10 +155,10 @@ object SparkEnv extends Logging { val serializerManager = new SerializerManager val serializer = serializerManager.setDefault( - System.getProperty("spark.serializer", "org.apache.spark.JavaSerializer")) + System.getProperty("spark.serializer", "org.apache.spark.serializer.JavaSerializer")) val closureSerializer = serializerManager.get( - System.getProperty("spark.closure.serializer", "org.apache.spark.JavaSerializer")) + System.getProperty("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer")) def registerOrLookup(name: String, newActor: => Actor): ActorRef = { if (isDriver) { diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index cb25ff728ee..5fd1fab5809 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.api.java -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext.doubleRDDToDoubleRDDFunctions import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.util.StatCounter diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 09da35aee66..a6518abf456 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -33,12 +33,12 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.HashPartitioner import org.apache.spark.Partitioner import org.apache.spark.Partitioner._ -import org.apache.spark.RDD import org.apache.spark.SparkContext.rddToPairRDDFunctions import org.apache.spark.api.java.function.{Function2 => JFunction2} import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.PartialResult +import org.apache.spark.rdd.RDD import org.apache.spark.rdd.OrderedRDDFunctions import org.apache.spark.storage.StorageLevel diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 68cfcf59994..eec58abdd60 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -18,6 +18,7 @@ package org.apache.spark.api.java import org.apache.spark._ +import org.apache.spark.rdd.RDD import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.storage.StorageLevel diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 1ad8514980f..7e6e691f11c 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -21,13 +21,15 @@ import java.util.{List => JList, Comparator} import scala.Tuple2 import scala.collection.JavaConversions._ +import com.google.common.base.Optional import org.apache.hadoop.io.compress.CompressionCodec -import org.apache.spark.{SparkContext, Partition, RDD, TaskContext} + +import org.apache.spark.{SparkContext, Partition, TaskContext} +import org.apache.spark.rdd.RDD import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _} import org.apache.spark.partial.{PartialResult, BoundedDouble} import org.apache.spark.storage.StorageLevel -import com.google.common.base.Optional trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 618a7b3bf7f..8869e072bf1 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -26,13 +26,13 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} +import com.google.common.base.Optional -import org.apache.spark.{Accumulable, AccumulableParam, Accumulator, AccumulatorParam, RDD, SparkContext} +import org.apache.spark.{Accumulable, AccumulableParam, Accumulator, AccumulatorParam, SparkContext} import org.apache.spark.SparkContext.IntAccumulatorParam import org.apache.spark.SparkContext.DoubleAccumulatorParam import org.apache.spark.broadcast.Broadcast - -import com.google.common.base.Optional +import org.apache.spark.rdd.RDD /** * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns [[org.apache.spark.api.java.JavaRDD]]s and diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala index eea63d5a4e4..b090c6edf3c 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala @@ -18,8 +18,8 @@ package org.apache.spark.api.python import org.apache.spark.Partitioner -import org.apache.spark.Utils import java.util.Arrays +import org.apache.spark.util.Utils /** * A [[org.apache.spark.Partitioner]] that performs handling of byte arrays, for use by the Python API. diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 621f0fe8ee6..ccd38339646 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -26,7 +26,9 @@ import scala.collection.JavaConversions._ import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import org.apache.spark.broadcast.Broadcast import org.apache.spark._ +import org.apache.spark.rdd.RDD import org.apache.spark.rdd.PipedRDD +import org.apache.spark.util.Utils private[spark] class PythonRDD[T: ClassManifest]( diff --git a/core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala index 99e86237fc0..93e7815ab51 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala @@ -27,6 +27,7 @@ import scala.math import org.apache.spark._ import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) extends Broadcast[T](id) diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 7a52ff0769c..9db26ae6de6 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -23,10 +23,10 @@ import java.net.URL import it.unimi.dsi.fastutil.io.FastBufferedInputStream import it.unimi.dsi.fastutil.io.FastBufferedOutputStream -import org.apache.spark.{HttpServer, Logging, SparkEnv, Utils} +import org.apache.spark.{HttpServer, Logging, SparkEnv} import org.apache.spark.io.CompressionCodec import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{MetadataCleaner, TimeStampedHashSet} +import org.apache.spark.util.{Utils, MetadataCleaner, TimeStampedHashSet} private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) diff --git a/core/src/main/scala/org/apache/spark/broadcast/MultiTracker.scala b/core/src/main/scala/org/apache/spark/broadcast/MultiTracker.scala index 10b910df87e..21ec94659e6 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/MultiTracker.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/MultiTracker.scala @@ -24,6 +24,7 @@ import java.util.Random import scala.collection.mutable.Map import org.apache.spark._ +import org.apache.spark.util.Utils private object MultiTracker extends Logging { diff --git a/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala index b5a4ccc0eee..80c97ca073e 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala @@ -26,6 +26,7 @@ import scala.math import org.apache.spark._ import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils private[spark] class TreeBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) extends Broadcast[T](id) with Logging with Serializable { diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index 4dc6ada2d16..c31619db279 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -19,10 +19,10 @@ package org.apache.spark.deploy import scala.collection.immutable.List -import org.apache.spark.Utils import org.apache.spark.deploy.ExecutorState.ExecutorState import org.apache.spark.deploy.master.{WorkerInfo, ApplicationInfo} import org.apache.spark.deploy.worker.ExecutorRunner +import org.apache.spark.util.Utils private[deploy] sealed trait DeployMessage extends Serializable diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index af5a4110b07..78e3747ad81 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -21,8 +21,8 @@ import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} import org.apache.spark.deploy.worker.Worker import org.apache.spark.deploy.master.Master -import org.apache.spark.util.AkkaUtils -import org.apache.spark.{Logging, Utils} +import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.{Logging} import scala.collection.mutable.ArrayBuffer diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index 0322029fbd0..d5e9a0e0957 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -17,8 +17,8 @@ package org.apache.spark.deploy.client -import org.apache.spark.util.AkkaUtils -import org.apache.spark.{Logging, Utils} +import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.{Logging} import org.apache.spark.deploy.{Command, ApplicationDescription} private[spark] object TestClient { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 869b2b2646d..7cf0a7754f7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -27,12 +27,12 @@ import akka.actor.Terminated import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown} import akka.util.duration._ -import org.apache.spark.{Logging, SparkException, Utils} +import org.apache.spark.{Logging, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.{Utils, AkkaUtils} private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index c86cca278d0..9d89b455fb9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -17,8 +17,7 @@ package org.apache.spark.deploy.master -import org.apache.spark.util.IntParam -import org.apache.spark.Utils +import org.apache.spark.util.{Utils, IntParam} /** * Command-line parser for the master. diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala index 285e07a8232..6219f11f2a2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.master import akka.actor.ActorRef import scala.collection.mutable -import org.apache.spark.Utils +import org.apache.spark.util.Utils private[spark] class WorkerInfo( val id: String, diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 6435c7f917c..f4e574d15db 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -31,7 +31,7 @@ import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMaste import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.master.ExecutorInfo import org.apache.spark.ui.UIUtils -import org.apache.spark.Utils +import org.apache.spark.util.Utils private[spark] class ApplicationPage(parent: MasterWebUI) { val master = parent.masterActorRef diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala index 58d3863009e..d7a57229b00 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala @@ -27,12 +27,12 @@ import akka.util.duration._ import net.liftweb.json.JsonAST.JValue -import org.apache.spark.Utils import org.apache.spark.deploy.DeployWebUI import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.master.{ApplicationInfo, WorkerInfo} import org.apache.spark.ui.UIUtils +import org.apache.spark.util.Utils private[spark] class IndexPage(parent: MasterWebUI) { val master = parent.masterActorRef diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 47b1e521f5c..f4df729e871 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -23,10 +23,11 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} -import org.apache.spark.{Logging, Utils} +import org.apache.spark.{Logging} import org.apache.spark.deploy.master.Master import org.apache.spark.ui.JettyUtils import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.util.Utils /** * Web UI server for the standalone master. diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 01ce4a6deab..e3dc30eefc5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -25,9 +25,10 @@ import akka.actor.ActorRef import com.google.common.base.Charsets import com.google.common.io.Files -import org.apache.spark.{Utils, Logging} +import org.apache.spark.{Logging} import org.apache.spark.deploy.{ExecutorState, ApplicationDescription} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged +import org.apache.spark.util.Utils /** * Manages the execution of one executor process. diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 86e8e7543ba..09530beb3be 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -27,13 +27,13 @@ import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} import akka.util.duration._ -import org.apache.spark.{Logging, Utils} +import org.apache.spark.{Logging} import org.apache.spark.deploy.ExecutorState import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.{Utils, AkkaUtils} private[spark] class Worker( diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index 6d91223413e..0ae89a864fd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -17,9 +17,7 @@ package org.apache.spark.deploy.worker -import org.apache.spark.util.IntParam -import org.apache.spark.util.MemoryParam -import org.apache.spark.Utils +import org.apache.spark.util.{Utils, IntParam, MemoryParam} import java.lang.management.ManagementFactory /** diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala index 6192c2324bd..d2d36174985 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala @@ -27,11 +27,11 @@ import akka.util.duration._ import net.liftweb.json.JsonAST.JValue -import org.apache.spark.Utils import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} import org.apache.spark.deploy.worker.ExecutorRunner import org.apache.spark.ui.UIUtils +import org.apache.spark.util.Utils private[spark] class IndexPage(parent: WorkerWebUI) { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index bb8165ac098..95d6007f3b3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -26,10 +26,11 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} import org.apache.spark.deploy.worker.Worker -import org.apache.spark.{Utils, Logging} +import org.apache.spark.{Logging} import org.apache.spark.ui.JettyUtils import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.UIUtils +import org.apache.spark.util.Utils /** * Web UI server for the standalone worker. diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 5446a3fca9f..d3658049945 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -27,6 +27,7 @@ import scala.collection.mutable.HashMap import org.apache.spark.scheduler._ import org.apache.spark._ +import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 410a94df6bf..da620919801 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -22,8 +22,9 @@ import org.apache.mesos.{Executor => MesosExecutor, MesosExecutorDriver, MesosNa import org.apache.mesos.Protos.{TaskState => MesosTaskState, TaskStatus => MesosTaskStatus, _} import org.apache.spark.TaskState.TaskState import com.google.protobuf.ByteString -import org.apache.spark.{Utils, Logging} +import org.apache.spark.{Logging} import org.apache.spark.TaskState +import org.apache.spark.util.Utils private[spark] class MesosExecutorBackend extends MesosExecutor diff --git a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala index 65801f75b7d..78390238681 100644 --- a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala @@ -22,10 +22,10 @@ import java.nio.ByteBuffer import akka.actor.{ActorRef, Actor, Props, Terminated} import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} -import org.apache.spark.{Logging, Utils, SparkEnv} +import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.cluster.StandaloneClusterMessages._ -import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.{Utils, AkkaUtils} private[spark] class StandaloneExecutorBackend( diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 9e2233c07b5..e15a839c4e7 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -34,6 +34,7 @@ import scala.collection.mutable.ArrayBuffer import akka.dispatch.{Await, Promise, ExecutionContext, Future} import akka.util.Duration import akka.util.duration._ +import org.apache.spark.util.Utils private[spark] class ConnectionManager(port: Int) extends Logging { diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala index 0839c011b8f..50dd9bc2d10 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala @@ -19,7 +19,7 @@ package org.apache.spark.network import java.net.InetSocketAddress -import org.apache.spark.Utils +import org.apache.spark.util.Utils private[spark] case class ConnectionManagerId(host: String, port: Int) { diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 11264806890..c0ec527339b 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -1,3 +1,5 @@ +import org.apache.spark.rdd.{SequenceFileRDDFunctions, DoubleRDDFunctions, PairRDDFunctions} + /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -16,16 +18,17 @@ */ /** - * Core Spark functionality. [[org.apache.spark.SparkContext]] serves as the main entry point to Spark, while - * [[org.apache.spark.RDD]] is the data type representing a distributed collection, and provides most - * parallel operations. + * Core Spark functionality. [[org.apache.spark.SparkContext]] serves as the main entry point to + * Spark, while [[org.apache.spark.rdd.RDD]] is the data type representing a distributed collection, + * and provides most parallel operations. * - * In addition, [[org.apache.spark.PairRDDFunctions]] contains operations available only on RDDs of key-value - * pairs, such as `groupByKey` and `join`; [[org.apache.spark.DoubleRDDFunctions]] contains operations - * available only on RDDs of Doubles; and [[org.apache.spark.SequenceFileRDDFunctions]] contains operations - * available on RDDs that can be saved as SequenceFiles. These operations are automatically - * available on any RDD of the right type (e.g. RDD[(Int, Int)] through implicit conversions when - * you `import org.apache.spark.SparkContext._`. + * In addition, [[org.apache.spark.rdd.PairRDDFunctions]] contains operations available only on RDDs + * of key-value pairs, such as `groupByKey` and `join`; [[org.apache.spark.rdd.DoubleRDDFunctions]] + * contains operations available only on RDDs of Doubles; and + * [[org.apache.spark.rdd.SequenceFileRDDFunctions]] contains operations available on RDDs that can + * be saved as SequenceFiles. These operations are automatically available on any RDD of the right + * type (e.g. RDD[(Int, Int)] through implicit conversions when you + * `import org.apache.spark.SparkContext._`. */ package object spark { // For package docs only diff --git a/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala b/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala index c5d51bee507..d71069444a7 100644 --- a/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala +++ b/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala @@ -19,6 +19,7 @@ package org.apache.spark.partial import org.apache.spark._ import org.apache.spark.scheduler.JobListener +import org.apache.spark.rdd.RDD /** * A JobListener for an approximate single-result action, such as count() or non-parallel reduce(). diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala index 4bb01efa865..bca6956a182 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} +import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext} import org.apache.spark.storage.BlockManager private[spark] class BlockRDDPartition(val blockId: String, idx: Int) extends Partition { diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index dcc35e8d0e9..0187256a8e3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -23,7 +23,7 @@ import java.util.{HashMap => JHashMap} import scala.collection.JavaConversions import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{Partition, Partitioner, RDD, SparkEnv, TaskContext} +import org.apache.spark.{Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} diff --git a/core/src/main/scala/org/apache/spark/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala similarity index 94% rename from core/src/main/scala/org/apache/spark/DoubleRDDFunctions.scala rename to core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index dd344491b83..a4bec417529 100644 --- a/core/src/main/scala/org/apache/spark/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -15,17 +15,18 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.rdd import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.MeanEvaluator import org.apache.spark.partial.PartialResult import org.apache.spark.partial.SumEvaluator import org.apache.spark.util.StatCounter +import org.apache.spark.{TaskContext, Logging} /** * Extra functions available on RDDs of Doubles through an implicit conversion. - * Import `spark.SparkContext._` at the top of your program to use these functions. + * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { /** Add up the elements in this RDD. */ diff --git a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala index 24ce4abbc47..c8900d1a934 100644 --- a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext} +import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext} /** diff --git a/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala index 4df8ceb58be..5312dc0b593 100644 --- a/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{OneToOneDependency, RDD, Partition, TaskContext} +import org.apache.spark.{OneToOneDependency, Partition, TaskContext} private[spark] class FilteredRDD[T: ClassManifest]( prev: RDD[T], diff --git a/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala index 2bf7653af1b..cbdf6d84c07 100644 --- a/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{RDD, Partition, TaskContext} +import org.apache.spark.{Partition, TaskContext} private[spark] diff --git a/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala index e544720b054..82000bac092 100644 --- a/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{TaskContext, Partition, RDD} +import org.apache.spark.{TaskContext, Partition} private[spark] diff --git a/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala index 2ce94199f2e..829545d7b0a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{RDD, Partition, TaskContext} +import org.apache.spark.{Partition, TaskContext} private[spark] class GlommedRDD[T: ClassManifest](prev: RDD[T]) extends RDD[Array[T]](prev) { diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 08e6154bb9f..2cb6734e419 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -18,21 +18,15 @@ package org.apache.spark.rdd import java.io.EOFException -import java.util.NoSuchElementException -import org.apache.hadoop.io.LongWritable -import org.apache.hadoop.io.NullWritable -import org.apache.hadoop.io.Text -import org.apache.hadoop.mapred.FileInputFormat import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.mapred.InputSplit import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapred.TextInputFormat import org.apache.hadoop.mapred.RecordReader import org.apache.hadoop.mapred.Reporter import org.apache.hadoop.util.ReflectionUtils -import org.apache.spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, SparkEnv, TaskContext} +import org.apache.spark.{Logging, Partition, SerializableWritable, SparkContext, SparkEnv, TaskContext} import org.apache.spark.util.NextIterator import org.apache.hadoop.conf.{Configuration, Configurable} diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index 3db460b3ced..aca01468842 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -19,7 +19,7 @@ package org.apache.spark.rdd import java.sql.{Connection, ResultSet} -import org.apache.spark.{Logging, Partition, RDD, SparkContext, TaskContext} +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} import org.apache.spark.util.NextIterator private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala index 13009d3e172..203179c4ea8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{RDD, Partition, TaskContext} +import org.apache.spark.{Partition, TaskContext} private[spark] diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala index 1683050b865..3ed83390107 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{RDD, Partition, TaskContext} +import org.apache.spark.{Partition, TaskContext} /** diff --git a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala index 26d4806edb0..e8be1c4816e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{RDD, Partition, TaskContext} +import org.apache.spark.{Partition, TaskContext} private[spark] class MappedRDD[U: ClassManifest, T: ClassManifest](prev: RDD[T], f: T => U) diff --git a/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala index a405e9acddb..d33c1af5815 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala @@ -18,7 +18,7 @@ package org.apache.spark.rdd -import org.apache.spark.{TaskContext, Partition, RDD} +import org.apache.spark.{TaskContext, Partition} private[spark] class MappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => U) diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 114b5044861..7b3a89f7e00 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ -import org.apache.spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext} +import org.apache.spark.{Dependency, Logging, Partition, SerializableWritable, SparkContext, TaskContext} private[spark] diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index 4c3df0eaf46..697be8b997b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -17,12 +17,13 @@ package org.apache.spark.rdd -import org.apache.spark.{RangePartitioner, Logging, RDD} +import org.apache.spark.{RangePartitioner, Logging} /** * Extra functions available on RDDs of (key, value) pairs where the key is sortable through - * an implicit conversion. Import `spark.SparkContext._` at the top of your program to use these - * functions. They will work with any key type that has a `scala.math.Ordered` implementation. + * an implicit conversion. Import `org.apache.spark.SparkContext._` at the top of your program to + * use these functions. They will work with any key type that has a `scala.math.Ordered` + * implementation. */ class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest, diff --git a/core/src/main/scala/org/apache/spark/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/PairRDDFunctions.scala rename to core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index d046e7c1a4f..a47c5122759 100644 --- a/core/src/main/scala/org/apache/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -15,40 +15,40 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.rdd import java.nio.ByteBuffer -import java.util.{Date, HashMap => JHashMap} +import java.util.Date import java.text.SimpleDateFormat +import java.util.{HashMap => JHashMap} import scala.collection.{mutable, Map} import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ +import org.apache.hadoop.mapred._ +import org.apache.hadoop.io.compress.CompressionCodec 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.SparkHadoopWriter -import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.OutputFormat - +import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} -import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, - RecordWriter => NewRecordWriter, Job => NewAPIHadoopJob, SparkHadoopMapReduceUtil} -import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.mapreduce.SparkHadoopMapReduceUtil +import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob} +import org.apache.hadoop.mapreduce.{RecordWriter => NewRecordWriter} -import org.apache.spark.partial.BoundedDouble -import org.apache.spark.partial.PartialResult -import org.apache.spark.rdd._ +import org.apache.spark._ import org.apache.spark.SparkContext._ -import org.apache.spark.Partitioner._ +import org.apache.spark.partial.{BoundedDouble, PartialResult} +import org.apache.spark.Aggregator +import org.apache.spark.Partitioner +import org.apache.spark.Partitioner.defaultPartitioner /** * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. - * Import `spark.SparkContext._` at the top of your program to use these functions. + * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) extends Logging @@ -697,7 +697,6 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) private[spark] def getValueClass() = implicitly[ClassManifest[V]].erasure } - private[spark] object Manifests { val seqSeqManifest = classManifest[Seq[Seq[_]]] } diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala index 8db3611054d..6dbd4309aae 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -23,6 +23,8 @@ import scala.collection.Map import org.apache.spark._ import java.io._ import scala.Serializable +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.util.Utils private[spark] class ParallelCollectionPartition[T: ClassManifest]( var rddId: Long, diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala index 8e79a5c874e..165cd412fcf 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{NarrowDependency, RDD, SparkEnv, Partition, TaskContext} +import org.apache.spark.{NarrowDependency, SparkEnv, Partition, TaskContext} class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends Partition { diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 98498d5ddfe..d5304ab0aed 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -25,7 +25,7 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer import scala.io.Source -import org.apache.spark.{RDD, SparkEnv, Partition, TaskContext} +import org.apache.spark.{SparkEnv, Partition, TaskContext} import org.apache.spark.broadcast.Broadcast diff --git a/core/src/main/scala/org/apache/spark/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/RDD.scala rename to core/src/main/scala/org/apache/spark/rdd/RDD.scala index 0d1f07f76cb..e143ecd0961 100644 --- a/core/src/main/scala/org/apache/spark/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.rdd import java.util.Random @@ -37,37 +37,22 @@ import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator import org.apache.spark.partial.GroupedCountEvaluator import org.apache.spark.partial.PartialResult -import org.apache.spark.rdd.CoalescedRDD -import org.apache.spark.rdd.CartesianRDD -import org.apache.spark.rdd.FilteredRDD -import org.apache.spark.rdd.FlatMappedRDD -import org.apache.spark.rdd.GlommedRDD -import org.apache.spark.rdd.MappedRDD -import org.apache.spark.rdd.MapPartitionsRDD -import org.apache.spark.rdd.MapPartitionsWithIndexRDD -import org.apache.spark.rdd.PipedRDD -import org.apache.spark.rdd.SampledRDD -import org.apache.spark.rdd.ShuffledRDD -import org.apache.spark.rdd.UnionRDD -import org.apache.spark.rdd.ZippedRDD -import org.apache.spark.rdd.ZippedPartitionsRDD2 -import org.apache.spark.rdd.ZippedPartitionsRDD3 -import org.apache.spark.rdd.ZippedPartitionsRDD4 import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.BoundedPriorityQueue +import org.apache.spark.util.{Utils, BoundedPriorityQueue} -import SparkContext._ +import org.apache.spark.SparkContext._ +import org.apache.spark._ /** * A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. Represents an immutable, * partitioned collection of elements that can be operated on in parallel. This class contains the * basic operations available on all RDDs, such as `map`, `filter`, and `persist`. In addition, - * [[org.apache.spark.PairRDDFunctions]] contains operations available only on RDDs of key-value pairs, such - * as `groupByKey` and `join`; [[org.apache.spark.DoubleRDDFunctions]] contains operations available only on - * RDDs of Doubles; and [[org.apache.spark.SequenceFileRDDFunctions]] contains operations available on RDDs - * that can be saved as SequenceFiles. These operations are automatically available on any RDD of - * the right type (e.g. RDD[(Int, Int)] through implicit conversions when you - * `import org.apache.spark.SparkContext._`. + * [[org.apache.spark.rdd.PairRDDFunctions]] contains operations available only on RDDs of key-value + * pairs, such as `groupByKey` and `join`; [[org.apache.spark.rdd.DoubleRDDFunctions]] contains + * operations available only on RDDs of Doubles; and [[org.apache.spark.rdd.SequenceFileRDDFunctions]] + * contains operations available on RDDs that can be saved as SequenceFiles. These operations are + * automatically available on any RDD of the right type (e.g. RDD[(Int, Int)] through implicit + * conversions when you `import org.apache.spark.SparkContext._`. * * Internally, each RDD is characterized by five main properties: * diff --git a/core/src/main/scala/org/apache/spark/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/RDDCheckpointData.scala rename to core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index 0334de6924c..6009a41570e 100644 --- a/core/src/main/scala/org/apache/spark/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -15,12 +15,13 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.rdd import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration -import rdd.{CheckpointRDD, CoalescedRDD} -import scheduler.{ResultTask, ShuffleMapTask} + +import org.apache.spark.{Partition, SparkException, Logging} +import org.apache.spark.scheduler.{ResultTask, ShuffleMapTask} /** * Enumeration to manage state transitions of an RDD through checkpointing diff --git a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala index 1e8d89e9120..2c5253ae301 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala @@ -22,7 +22,7 @@ import java.util.Random import cern.jet.random.Poisson import cern.jet.random.engine.DRand -import org.apache.spark.{RDD, Partition, TaskContext} +import org.apache.spark.{Partition, TaskContext} private[spark] class SampledRDDPartition(val prev: Partition, val seed: Int) extends Partition with Serializable { diff --git a/core/src/main/scala/org/apache/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala similarity index 84% rename from core/src/main/scala/org/apache/spark/SequenceFileRDDFunctions.scala rename to core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala index d58fb4e4bcb..5fe4676029d 100644 --- a/core/src/main/scala/org/apache/spark/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala @@ -15,40 +15,22 @@ * limitations under the License. */ -package org.apache.spark - -import java.io.EOFException -import java.net.URL -import java.io.ObjectInputStream -import java.util.concurrent.atomic.AtomicLong -import java.util.HashSet -import java.util.Random -import java.util.Date - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.Map -import scala.collection.mutable.HashMap +package org.apache.spark.rdd import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapred.OutputFormat -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 -import org.apache.hadoop.io.Text import org.apache.spark.SparkContext._ +import org.apache.spark.Logging /** * Extra functions available on RDDs of (key, value) pairs to create a Hadoop SequenceFile, * through an implicit conversion. Note that this can't be part of PairRDDFunctions because * we need more implicit parameters to convert our keys and values to Writable. * - * Users should import `spark.SparkContext._` at the top of their program to use these functions. + * Import `org.apache.spark.SparkContext._` at the top of their program to use these functions. */ class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : ClassManifest]( self: RDD[(K, V)]) diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index f0e9ab8b801..95371523350 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{Dependency, Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext} +import org.apache.spark.{Dependency, Partitioner, SparkEnv, ShuffleDependency, Partition, TaskContext} private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala index 7369dfaa74b..8c1a29dfff0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala @@ -20,7 +20,6 @@ package org.apache.spark.rdd import java.util.{HashMap => JHashMap} import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer -import org.apache.spark.RDD import org.apache.spark.Partitioner import org.apache.spark.Dependency import org.apache.spark.TaskContext diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index fd02476b627..ae8a9f36a60 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -18,7 +18,7 @@ package org.apache.spark.rdd import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{Dependency, RangeDependency, RDD, SparkContext, Partition, TaskContext} +import org.apache.spark.{Dependency, RangeDependency, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} private[spark] class UnionPartition[T: ClassManifest](idx: Int, rdd: RDD[T], splitIndex: Int) diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index 5ae1db3e676..31e6fd519d0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} +import org.apache.spark.{OneToOneDependency, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} private[spark] class ZippedPartitionsPartition( diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala index 3bd00d291b1..567b67dfee0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import org.apache.spark.{Utils, OneToOneDependency, RDD, SparkContext, Partition, TaskContext} +import org.apache.spark.{OneToOneDependency, SparkContext, Partition, TaskContext} import java.io.{ObjectOutputStream, IOException} diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 5ac700bbf44..92add5b073a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -25,6 +25,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} import org.apache.spark._ +import org.apache.spark.rdd.RDD import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.scheduler.cluster.TaskInfo diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 5b07933eedc..0d996706489 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -23,6 +23,7 @@ import org.apache.spark.scheduler.cluster.TaskInfo import scala.collection.mutable.Map import org.apache.spark._ +import org.apache.spark.rdd.RDD import org.apache.spark.executor.TaskMetrics /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 98ef4d1e634..c8b78bf00a8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -28,6 +28,7 @@ import scala.collection.mutable.{Map, HashMap, ListBuffer} import scala.io.Source import org.apache.spark._ +import org.apache.spark.rdd.RDD import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.cluster.TaskInfo diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 2f157ccdd25..2b007cbe824 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -17,11 +17,14 @@ package org.apache.spark.scheduler -import org.apache.spark._ import java.io._ -import util.{MetadataCleaner, TimeStampedHashMap} import java.util.zip.{GZIPInputStream, GZIPOutputStream} +import org.apache.spark._ +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.RDDCheckpointData +import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap} + private[spark] object ResultTask { // A simple map between the stage id to the serialized byte array of a task. diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index ca716b44e88..764775fedea 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -26,6 +26,8 @@ import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.storage._ import org.apache.spark.util.{TimeStampedHashMap, MetadataCleaner} +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.RDDCheckpointData private[spark] object ShuffleMapTask { diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 3504424fa94..c3cf4b8907e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -19,8 +19,8 @@ package org.apache.spark.scheduler import java.util.Properties import org.apache.spark.scheduler.cluster.TaskInfo -import org.apache.spark.util.Distribution -import org.apache.spark.{Logging, SparkContext, TaskEndReason, Utils} +import org.apache.spark.util.{Utils, Distribution} +import org.apache.spark.{Logging, SparkContext, TaskEndReason} import org.apache.spark.executor.TaskMetrics sealed trait SparkListenerEvents diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 87b1fe4e0c9..aa293dc6b31 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -17,9 +17,8 @@ package org.apache.spark.scheduler -import java.net.URI - import org.apache.spark._ +import org.apache.spark.rdd.RDD import org.apache.spark.storage.BlockManagerId /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index 776675d28c9..5c7e5bb9775 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -21,8 +21,9 @@ import java.io._ import scala.collection.mutable.Map import org.apache.spark.executor.TaskMetrics -import org.apache.spark.{Utils, SparkEnv} +import org.apache.spark.{SparkEnv} import java.nio.ByteBuffer +import org.apache.spark.util.Utils // Task result. Also contains updates to accumulator variables. // TODO: Use of distributed cache to return result is a hack to get around diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala index a33307b83ae..1b31c8c57ec 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -26,7 +26,7 @@ import scala.collection.mutable.HashSet import scala.math.max import scala.math.min -import org.apache.spark.{FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskState, Utils} +import org.apache.spark.{FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskState} import org.apache.spark.{ExceptionFailure, SparkException, TaskResultTooBigFailure} import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala index bde2f73df41..d57eb3276f7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler.cluster -import org.apache.spark.{SparkContext, Utils} +import org.apache.spark.{SparkContext} /** * A backend interface for cluster scheduling systems that allows plugging in different ones under diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index ac6dc7d8793..d003bf1bba9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -17,10 +17,11 @@ package org.apache.spark.scheduler.cluster -import org.apache.spark.{Utils, Logging, SparkContext} +import org.apache.spark.{Logging, SparkContext} import org.apache.spark.deploy.client.{Client, ClientListener} import org.apache.spark.deploy.{Command, ApplicationDescription} import scala.collection.mutable.HashMap +import org.apache.spark.util.Utils private[spark] class SparkDeploySchedulerBackend( scheduler: ClusterScheduler, diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala index 1cc5daf6739..9c36d221f69 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala @@ -20,8 +20,7 @@ package org.apache.spark.scheduler.cluster import java.nio.ByteBuffer import org.apache.spark.TaskState.TaskState -import org.apache.spark.Utils -import org.apache.spark.util.SerializableBuffer +import org.apache.spark.util.{Utils, SerializableBuffer} private[spark] sealed trait StandaloneClusterMessage extends Serializable diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 3677a827e0c..b4ea0be415e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -28,8 +28,9 @@ import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClient import akka.util.Duration import akka.util.duration._ -import org.apache.spark.{Utils, SparkException, Logging, TaskState} +import org.apache.spark.{SparkException, Logging, TaskState} import org.apache.spark.scheduler.cluster.StandaloneClusterMessages._ +import org.apache.spark.util.Utils /** * A standalone scheduler backend, which waits for standalone executors to connect to it through diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala index 7ce14be7fb9..9685fb1a67b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler.cluster -import org.apache.spark.Utils +import org.apache.spark.util.Utils /** * Information about a running task attempt inside a TaskSet. diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala index f0ebe66d823..e8fa5e2f170 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala @@ -34,6 +34,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster._ import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode import akka.actor._ +import org.apache.spark.util.Utils /** * A FIFO or Fair TaskScheduler implementation that runs tasks locally in a thread pool. Optionally diff --git a/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index f6a2feab289..3dbe61d7067 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -23,7 +23,7 @@ import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import org.apache.spark.{SparkException, Utils, Logging, SparkContext} +import org.apache.spark.{SparkException, Logging, SparkContext} import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.collection.JavaConversions._ import java.io.File diff --git a/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala index e002af17422..541f86e3381 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala @@ -23,7 +23,7 @@ import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import org.apache.spark.{SparkException, Utils, Logging, SparkContext} +import org.apache.spark.{SparkException, Logging, SparkContext} import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.collection.JavaConversions._ import java.io.File @@ -31,6 +31,7 @@ import org.apache.spark.scheduler.cluster._ import java.util.{ArrayList => JArrayList, List => JList} import java.util.Collections import org.apache.spark.TaskState +import org.apache.spark.util.Utils /** * A SchedulerBackend for running fine-grained tasks on Mesos. Each Spark task is mapped to a diff --git a/core/src/main/scala/org/apache/spark/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala similarity index 95% rename from core/src/main/scala/org/apache/spark/JavaSerializer.scala rename to core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index f43396cb6b0..4de81617b1d 100644 --- a/core/src/main/scala/org/apache/spark/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.serializer import java.io._ import java.nio.ByteBuffer -import serializer.{Serializer, SerializerInstance, DeserializationStream, SerializationStream} import org.apache.spark.util.ByteBufferInputStream private[spark] class JavaSerializationStream(out: OutputStream) extends SerializationStream { diff --git a/core/src/main/scala/org/apache/spark/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala similarity index 93% rename from core/src/main/scala/org/apache/spark/KryoSerializer.scala rename to core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index db86e6db439..24ef204aa19 100644 --- a/core/src/main/scala/org/apache/spark/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -15,17 +15,74 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.serializer -import java.io._ import java.nio.ByteBuffer -import com.esotericsoftware.kryo.{Kryo, KryoException} -import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} +import java.io.{EOFException, InputStream, OutputStream} + import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} +import com.esotericsoftware.kryo.{KryoException, Kryo} +import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} import com.twitter.chill.ScalaKryoInstantiator -import serializer.{SerializerInstance, DeserializationStream, SerializationStream} -import org.apache.spark.broadcast._ -import org.apache.spark.storage._ + +import org.apache.spark.{SerializableWritable, Logging} +import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock, StorageLevel} + +import org.apache.spark.broadcast.HttpBroadcast + +/** + * A Spark serializer that uses the [[http://code.google.com/p/kryo/wiki/V1Documentation Kryo 1.x library]]. + */ +class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging { + private val bufferSize = System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 + + def newKryoOutput() = new KryoOutput(bufferSize) + + def newKryoInput() = new KryoInput(bufferSize) + + 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( + 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) + + // Allow sending SerializableWritable + kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer()) + kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer()) + + // 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.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) + + kryo + } + + def newInstance(): SerializerInstance = { + new KryoSerializerInstance(this) + } +} private[spark] class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { @@ -100,57 +157,3 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ trait KryoRegistrator { 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 org.apache.spark.serializer.Serializer with Logging { - private val bufferSize = System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024 - - def newKryoOutput() = new KryoOutput(bufferSize) - - def newKryoInput() = new KryoInput(bufferSize) - - 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( - 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) - - // Allow sending SerializableWritable - kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer()) - kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer()) - - // 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.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) - - kryo - } - - def newInstance(): SerializerInstance = { - new KryoSerializerInstance(this) - } -} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index c91f0fc1ad8..3aeda3879d1 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -27,12 +27,12 @@ import scala.collection.mutable.Queue import io.netty.buffer.ByteBuf import org.apache.spark.Logging -import org.apache.spark.Utils import org.apache.spark.SparkException import org.apache.spark.network.BufferMessage import org.apache.spark.network.ConnectionManagerId import org.apache.spark.network.netty.ShuffleCopier import org.apache.spark.serializer.Serializer +import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 3299ac98d55..60fdc5f2ee2 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -29,11 +29,11 @@ import akka.util.duration._ import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream -import org.apache.spark.{Logging, SparkEnv, SparkException, Utils} +import org.apache.spark.{Logging, SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer -import org.apache.spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStampedHashMap} +import org.apache.spark.util._ import sun.nio.ch.DirectBuffer diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index a22a80decc9..74207f59af1 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -19,7 +19,7 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import java.util.concurrent.ConcurrentHashMap -import org.apache.spark.Utils +import org.apache.spark.util.Utils /** * This class represent an unique identifier for a BlockManager. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index baa4a1da502..c7b23ab094a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -28,8 +28,9 @@ import akka.pattern.ask import akka.util.Duration import akka.util.duration._ -import org.apache.spark.{Logging, Utils, SparkException} +import org.apache.spark.{Logging, SparkException} import org.apache.spark.storage.BlockManagerMessages._ +import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala index f4856020e55..678c38203c7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala @@ -19,8 +19,9 @@ package org.apache.spark.storage import java.nio.ByteBuffer -import org.apache.spark.{Logging, Utils} +import org.apache.spark.{Logging} import org.apache.spark.network._ +import org.apache.spark.util.Utils /** * A network interface for BlockManager. Each slave should have one diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index fd945e065cc..fc25ef0fae2 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -28,12 +28,12 @@ import scala.collection.mutable.ArrayBuffer import it.unimi.dsi.fastutil.io.FastBufferedOutputStream -import org.apache.spark.Utils import org.apache.spark.executor.ExecutorExitCode import org.apache.spark.serializer.{Serializer, SerializationStream} import org.apache.spark.Logging import org.apache.spark.network.netty.ShuffleSender import org.apache.spark.network.netty.PathResolver +import org.apache.spark.util.Utils /** diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 828dc0f22da..3b3b2342fa2 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -19,9 +19,9 @@ package org.apache.spark.storage import java.util.LinkedHashMap import java.util.concurrent.ArrayBlockingQueue -import org.apache.spark.{SizeEstimator, Utils} import java.nio.ByteBuffer import collection.mutable.ArrayBuffer +import org.apache.spark.util.{SizeEstimator, Utils} /** * Stores blocks in memory, either as ArrayBuffers of deserialized Java objects or as diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 0bba1dac540..2bb7715696c 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -17,8 +17,9 @@ package org.apache.spark.storage -import org.apache.spark.{Utils, SparkContext} +import org.apache.spark.{SparkContext} import BlockManagerMasterActor.BlockStatus +import org.apache.spark.util.Utils private[spark] case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index 1d5afe9b085..f2ae8dd97da 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -19,9 +19,9 @@ package org.apache.spark.storage import akka.actor._ -import org.apache.spark.KryoSerializer import java.util.concurrent.ArrayBlockingQueue import util.Random +import org.apache.spark.serializer.KryoSerializer /** * This class tests the BlockManager and MemoryStore for thread safety and diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 4688effe0ad..ad456ea5653 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -21,12 +21,13 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.server.{Handler, Server} -import org.apache.spark.{Logging, SparkContext, SparkEnv, Utils} +import org.apache.spark.{Logging, SparkContext, SparkEnv} import org.apache.spark.ui.env.EnvironmentUI import org.apache.spark.ui.exec.ExecutorsUI import org.apache.spark.ui.storage.BlockManagerUI import org.apache.spark.ui.jobs.JobProgressUI import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.util.Utils /** Top level user interface for Spark */ private[spark] class SparkUI(sc: SparkContext) extends Logging { diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index efe6b474e00..6e56c22d04a 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -7,13 +7,14 @@ import scala.xml.Node import org.eclipse.jetty.server.Handler -import org.apache.spark.{ExceptionFailure, Logging, Utils, SparkContext} +import org.apache.spark.{ExceptionFailure, Logging, SparkContext} import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.cluster.TaskInfo import org.apache.spark.scheduler.{SparkListenerTaskStart, SparkListenerTaskEnd, SparkListener} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Executors import org.apache.spark.ui.UIUtils +import org.apache.spark.util.Utils private[spark] class ExecutorsUI(val sc: SparkContext) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index ae022263000..86e0af0399f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -3,7 +3,7 @@ package org.apache.spark.ui.jobs import scala.Seq import scala.collection.mutable.{ListBuffer, HashMap, HashSet} -import org.apache.spark.{ExceptionFailure, SparkContext, Success, Utils} +import org.apache.spark.{ExceptionFailure, SparkContext, Success} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.TaskInfo import org.apache.spark.executor.TaskMetrics diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index 1bb7638bd9d..6aecef51204 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -29,11 +29,12 @@ import scala.Seq import scala.collection.mutable.{HashSet, ListBuffer, HashMap, ArrayBuffer} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.{ExceptionFailure, SparkContext, Success, Utils} +import org.apache.spark.{ExceptionFailure, SparkContext, Success} import org.apache.spark.scheduler._ import collection.mutable import org.apache.spark.scheduler.cluster.SchedulingMode import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode +import org.apache.spark.util.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ private[spark] class JobProgressUI(val sc: SparkContext) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 2fe85bc0cf3..a9969ab1c08 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -25,8 +25,8 @@ import scala.xml.Node import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ -import org.apache.spark.util.Distribution -import org.apache.spark.{ExceptionFailure, Utils} +import org.apache.spark.util.{Utils, Distribution} +import org.apache.spark.{ExceptionFailure} import org.apache.spark.scheduler.cluster.TaskInfo import org.apache.spark.executor.TaskMetrics diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index beb05745486..71e58a977ef 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -5,9 +5,9 @@ import java.util.Date import scala.xml.Node import scala.collection.mutable.HashSet -import org.apache.spark.Utils import org.apache.spark.scheduler.cluster.{SchedulingMode, TaskInfo} import org.apache.spark.scheduler.Stage +import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index 1eb4a7a85ea..c3ec9073704 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -22,9 +22,9 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.storage.{RDDInfo, StorageUtils} -import org.apache.spark.Utils import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ +import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ private[spark] class IndexPage(parent: BlockManagerUI) { diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 37baf17f7a5..43c12576770 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -21,11 +21,11 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.Utils import org.apache.spark.storage.{StorageStatus, StorageUtils} import org.apache.spark.storage.BlockManagerMasterActor.BlockStatus import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ +import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ diff --git a/core/src/main/scala/org/apache/spark/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala similarity index 99% rename from core/src/main/scala/org/apache/spark/ClosureCleaner.scala rename to core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index 71d9e62d4f0..7108595e3e4 100644 --- a/core/src/main/scala/org/apache/spark/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.util import java.lang.reflect.Field @@ -25,6 +25,7 @@ import scala.collection.mutable.Set import org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} import org.objectweb.asm.Opcodes._ import java.io.{InputStream, IOException, ByteArrayOutputStream, ByteArrayInputStream, BufferedInputStream} +import org.apache.spark.Logging private[spark] object ClosureCleaner extends Logging { // Get an ASM class reader for a given class from the JAR that loaded it diff --git a/core/src/main/scala/org/apache/spark/util/MemoryParam.scala b/core/src/main/scala/org/apache/spark/util/MemoryParam.scala index 0ee6707826d..4869c9897a4 100644 --- a/core/src/main/scala/org/apache/spark/util/MemoryParam.scala +++ b/core/src/main/scala/org/apache/spark/util/MemoryParam.scala @@ -17,8 +17,6 @@ package org.apache.spark.util -import org.apache.spark.Utils - /** * An extractor object for parsing JVM memory strings, such as "10g", into an Int representing * the number of megabytes. Supports the same formats as Utils.memoryStringToMb. diff --git a/core/src/main/scala/org/apache/spark/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala similarity index 99% rename from core/src/main/scala/org/apache/spark/SizeEstimator.scala rename to core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index 4bfc8377103..a25b37a2a96 100644 --- a/core/src/main/scala/org/apache/spark/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.util import java.lang.reflect.Field import java.lang.reflect.Modifier @@ -30,6 +30,7 @@ import java.lang.management.ManagementFactory import scala.collection.mutable.ArrayBuffer import it.unimi.dsi.fastutil.ints.IntOpenHashSet +import org.apache.spark.Logging /** * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in diff --git a/core/src/main/scala/org/apache/spark/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala similarity index 99% rename from core/src/main/scala/org/apache/spark/Utils.scala rename to core/src/main/scala/org/apache/spark/util/Utils.scala index 1e17deb010d..bb47fc0a2ca 100644 --- a/core/src/main/scala/org/apache/spark/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.util import java.io._ import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address, ServerSocket} @@ -36,12 +36,13 @@ import org.apache.hadoop.fs.{Path, FileSystem, FileUtil} import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} import org.apache.spark.deploy.SparkHadoopUtil import java.nio.ByteBuffer +import org.apache.spark.{SparkEnv, SparkException, Logging} /** * Various utility methods used by Spark. */ -private object Utils extends Logging { +private[spark] object Utils extends Logging { /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index 23b14f4245e..d9103aebb7f 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -22,6 +22,7 @@ import java.io.File import org.apache.spark.rdd._ import org.apache.spark.SparkContext._ import storage.StorageLevel +import org.apache.spark.util.Utils class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { initLogging() diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index b08aad1a6fd..01a72d84016 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -26,6 +26,7 @@ import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts import org.scalatest.prop.TableDrivenPropertyChecks._ import org.scalatest.time.SpanSugar._ +import org.apache.spark.util.Utils class DriverSuite extends FunSuite with Timeouts { test("driver should exit after finishing") { diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index ee89a7a3872..af448fcb37a 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark import org.scalatest.FunSuite import SparkContext._ +import org.apache.spark.util.NonSerializable // Common state shared by FailureSuite-launched tasks. We use a global object // for this because any local variables used in the task closures will rightfully diff --git a/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala b/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala index adbe805916b..5a18dd13ff5 100644 --- a/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala @@ -2,7 +2,7 @@ package org.apache.spark import org.scalatest.FunSuite import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.PartitionPruningRDD +import org.apache.spark.rdd.{RDD, PartitionPruningRDD} class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index 7669cf6fb14..7d938917f26 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -17,11 +17,14 @@ package org.apache.spark -import org.scalatest.FunSuite +import scala.math.abs import scala.collection.mutable.ArrayBuffer -import SparkContext._ + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext._ import org.apache.spark.util.StatCounter -import scala.math.abs +import org.apache.spark.rdd.RDD class PartitioningSuite extends FunSuite with SharedSparkContext { diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 357175e89ee..db717865db5 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -22,8 +22,9 @@ import org.scalatest.matchers.ShouldMatchers import org.apache.spark.SparkContext._ import org.apache.spark.ShuffleSuite.NonJavaSerializableClass -import org.apache.spark.rdd.{SubtractedRDD, CoGroupedRDD, OrderedRDDFunctions, ShuffledRDD} +import org.apache.spark.rdd.{RDD, SubtractedRDD, CoGroupedRDD, OrderedRDDFunctions, ShuffledRDD} import org.apache.spark.util.MutablePair +import org.apache.spark.serializer.KryoSerializer class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala similarity index 99% rename from core/src/test/scala/org/apache/spark/PairRDDFunctionsSuite.scala rename to core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index f79752b34ef..31f97fc1391 100644 --- a/core/src/test/scala/org/apache/spark/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.rdd import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashSet @@ -24,6 +24,7 @@ import org.scalatest.FunSuite import com.google.common.io.Files import org.apache.spark.SparkContext._ +import org.apache.spark.{Partitioner, SharedSparkContext} class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { diff --git a/core/src/test/scala/org/apache/spark/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala similarity index 99% rename from core/src/test/scala/org/apache/spark/RDDSuite.scala rename to core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 342ba8adb20..adc971050e1 100644 --- a/core/src/test/scala/org/apache/spark/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.rdd import scala.collection.mutable.HashMap import org.scalatest.FunSuite @@ -24,6 +24,8 @@ import org.scalatest.time.{Span, Millis} import org.apache.spark.SparkContext._ import org.apache.spark.rdd._ import scala.collection.parallel.mutable +import org.apache.spark._ +import org.apache.spark.rdd.CoalescedRDDPartition class RDDSuite extends FunSuite with SharedSparkContext { diff --git a/core/src/test/scala/org/apache/spark/SortingSuite.scala b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala similarity index 97% rename from core/src/test/scala/org/apache/spark/SortingSuite.scala rename to core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala index f4fa9511dda..2f7bd370fc4 100644 --- a/core/src/test/scala/org/apache/spark/SortingSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala @@ -15,12 +15,14 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.rdd import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter import org.scalatest.matchers.ShouldMatchers -import SparkContext._ + +import org.apache.spark.{Logging, SharedSparkContext} +import org.apache.spark.SparkContext._ class SortingSuite extends FunSuite with SharedSparkContext with ShouldMatchers with Logging { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 94df282b284..94f66c94c68 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -24,7 +24,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.LocalSparkContext import org.apache.spark.MapOutputTracker -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext import org.apache.spark.Partition import org.apache.spark.TaskContext diff --git a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala index f5b3e972227..cece60dda72 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala @@ -19,11 +19,15 @@ package org.apache.spark.scheduler import java.util.Properties import java.util.concurrent.LinkedBlockingQueue + +import scala.collection.mutable + import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers -import scala.collection.mutable + import org.apache.spark._ import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.RDD class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index 0347cc02d7a..e31a116a75b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter import org.apache.spark.TaskContext -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext import org.apache.spark.Partition import org.apache.spark.LocalSparkContext diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala index a4f63baf3d8..ff70a2cdf07 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark._ import org.apache.spark.scheduler._ import org.apache.spark.executor.TaskMetrics import java.nio.ByteBuffer -import org.apache.spark.util.FakeClock +import org.apache.spark.util.{Utils, FakeClock} /** * A mock ClusterScheduler implementation that just remembers information about tasks started and diff --git a/core/src/test/scala/org/apache/spark/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala similarity index 96% rename from core/src/test/scala/org/apache/spark/KryoSerializerSuite.scala rename to core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index d7b23c93fec..0164dda0ba5 100644 --- a/core/src/test/scala/org/apache/spark/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -15,14 +15,15 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.serializer import scala.collection.mutable -import org.scalatest.FunSuite -import com.esotericsoftware.kryo._ +import com.esotericsoftware.kryo.Kryo -import KryoTest._ +import org.scalatest.FunSuite +import org.apache.spark.SharedSparkContext +import org.apache.spark.serializer.KryoTest._ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("basic types") { @@ -103,7 +104,6 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { } test("custom registrator") { - import KryoTest._ System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName) val ser = (new KryoSerializer).newInstance() @@ -167,7 +167,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { } override def beforeAll() { - System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer") + System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName) super.beforeAll() } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 88ba10f2f2d..038a9acb859 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -29,12 +29,8 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.matchers.ShouldMatchers._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.JavaSerializer -import org.apache.spark.KryoSerializer -import org.apache.spark.SizeEstimator -import org.apache.spark.Utils -import org.apache.spark.util.AkkaUtils -import org.apache.spark.util.ByteBufferInputStream +import org.apache.spark.util.{SizeEstimator, Utils, AkkaUtils, ByteBufferInputStream} +import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester { diff --git a/core/src/test/scala/org/apache/spark/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala similarity index 98% rename from core/src/test/scala/org/apache/spark/ClosureCleanerSuite.scala rename to core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala index 8494899b984..0ed366fb707 100644 --- a/core/src/test/scala/org/apache/spark/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -15,13 +15,14 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.util import java.io.NotSerializableException import org.scalatest.FunSuite + +import org.apache.spark.SparkContext import org.apache.spark.LocalSparkContext._ -import SparkContext._ class ClosureCleanerSuite extends FunSuite { test("closures inside an object") { diff --git a/core/src/test/scala/org/apache/spark/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala similarity index 99% rename from core/src/test/scala/org/apache/spark/SizeEstimatorSuite.scala rename to core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala index 214ac74898f..4e40dcbdeeb 100644 --- a/core/src/test/scala/org/apache/spark/SizeEstimatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.util import org.scalatest.FunSuite import org.scalatest.BeforeAndAfterAll diff --git a/core/src/test/scala/org/apache/spark/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala similarity index 99% rename from core/src/test/scala/org/apache/spark/UtilsSuite.scala rename to core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 3a908720a85..e2859caf58d 100644 --- a/core/src/test/scala/org/apache/spark/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.util import com.google.common.base.Charsets import com.google.common.io.Files diff --git a/docs/configuration.md b/docs/configuration.md index 55df18b6fb8..58e9434bdc0 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -36,13 +36,13 @@ there are at least five properties that you will commonly want to control: spark.serializer - org.apache.spark.JavaSerializer + org.apache.spark.serializer.
    JavaSerializer Class to use for serializing objects that will be sent over the network or need to be cached in serialized form. The default of Java serialization works with any Serializable Java object but is - quite slow, so we recommend using org.apache.spark.KryoSerializer + quite slow, so we recommend using org.apache.spark.serializer.KryoSerializer and configuring Kryo serialization when speed is necessary. Can be any subclass of - org.apache.spark.Serializer. + org.apache.spark.Serializer. @@ -51,7 +51,7 @@ there are at least five properties that you will commonly want to control: If you use Kryo serialization, set this class to register your custom classes with Kryo. It should be set to a class that extends - KryoRegistrator. + KryoRegistrator. See the tuning guide for more details. @@ -171,7 +171,7 @@ Apart from these, the following properties are also available, and may be useful spark.closure.serializer - org.apache.spark.JavaSerializer + org.apache.spark.serializer.
    JavaSerializer Serializer class to use for closures. Generally Java is fine unless your distributed functions (e.g. map functions) reference large objects in the driver program. diff --git a/docs/quick-start.md b/docs/quick-start.md index 8cbc55bed28..70c3df8095d 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -109,7 +109,7 @@ We'll create a very simple Spark job in Scala. So simple, in fact, that it's nam {% highlight scala %} /*** SimpleJob.scala ***/ import org.apache.spark.SparkContext -import SparkContext._ +import org.apache.spark.SparkContext._ object SimpleJob { def main(args: Array[String]) { diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 2cf319a2632..f7768e55fc6 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -37,7 +37,7 @@ Finally, you need to import some Spark classes and implicit conversions into you {% highlight scala %} import org.apache.spark.SparkContext -import SparkContext._ +import org.apache.spark.SparkContext._ {% endhighlight %} # Initializing Spark diff --git a/docs/tuning.md b/docs/tuning.md index 3563d110c9d..28d88a26592 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -38,17 +38,17 @@ in your operations) and performance. It provides two serialization libraries: `Serializable` types and requires you to *register* the classes you'll use in the program in advance for best performance. -You can switch to using Kryo by calling `System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer")` +You can switch to using Kryo by calling `System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer")` *before* creating your SparkContext. The only reason it is not the default is because of the custom registration requirement, but we recommend trying it in any network-intensive application. Finally, to register your classes with Kryo, create a public class that extends -[`org.apache.spark.KryoRegistrator`](api/core/index.html#org.apache.spark.KryoRegistrator) and set the +[`org.apache.spark.serializer.KryoRegistrator`](api/core/index.html#org.apache.spark.serializer.KryoRegistrator) and set the `spark.kryo.registrator` system property to point to it, as follows: {% highlight scala %} import com.esotericsoftware.kryo.Kryo -import org.apache.spark.KryoRegistrator +import org.apache.spark.serializer.KryoRegistrator class MyRegistrator extends KryoRegistrator { override def registerClasses(kryo: Kryo) { @@ -58,7 +58,7 @@ class MyRegistrator extends KryoRegistrator { } // Make sure to set these properties *before* creating a SparkContext! -System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer") +System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") System.setProperty("spark.kryo.registrator", "mypackage.MyRegistrator") val sc = new SparkContext(...) {% endhighlight %} @@ -217,7 +217,7 @@ enough. Spark automatically sets the number of "map" tasks to run on each file a (though you can control it through optional parameters to `SparkContext.textFile`, etc), and for distributed "reduce" operations, such as `groupByKey` and `reduceByKey`, it uses the largest parent RDD's number of partitions. You can pass the level of parallelism as a second argument -(see the [`spark.PairRDDFunctions`](api/core/index.html#org.apache.spark.PairRDDFunctions) documentation), +(see the [`spark.PairRDDFunctions`](api/core/index.html#org.apache.spark.rdd.PairRDDFunctions) documentation), or set the system property `spark.default.parallelism` to change the default. In general, we recommend 2-3 tasks per CPU core in your cluster. diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala index b190e83c4d3..cfafbaf23e4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala @@ -19,6 +19,7 @@ package org.apache.spark.examples.bagel import org.apache.spark._ import org.apache.spark.SparkContext._ +import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.bagel._ import org.apache.spark.bagel.Bagel._ diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala index b1f606e48e0..72b5c7b88e1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala @@ -37,7 +37,7 @@ object WikipediaPageRank { System.exit(-1) } - System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer") + System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") System.setProperty("spark.kryo.registrator", classOf[PRKryoRegistrator].getName) val inputFile = args(0) diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala index 3bfa48eaf3d..ddf6855325e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala @@ -17,19 +17,16 @@ package org.apache.spark.examples.bagel -import org.apache.spark._ -import serializer.{DeserializationStream, SerializationStream, SerializerInstance} -import org.apache.spark.SparkContext._ - -import org.apache.spark.bagel._ -import org.apache.spark.bagel.Bagel._ - -import scala.xml.{XML,NodeSeq} +import java.io.{InputStream, OutputStream, DataInputStream, DataOutputStream} +import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer +import scala.xml.{XML, NodeSeq} -import java.io.{InputStream, OutputStream, DataInputStream, DataOutputStream} -import java.nio.ByteBuffer +import org.apache.spark._ +import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.RDD object WikipediaPageRankStandalone { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala index 822da8c9b5c..fad512eebad 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.examples -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala index 2e3d9ccf004..0b45c30d20d 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala @@ -28,7 +28,7 @@ import org.apache.spark.streaming.util.RawTextHelper * lines have the word 'the' in them. This is useful for benchmarking purposes. This * will only work with spark.streaming.util.RawTextSender running on all worker nodes * and with Spark using Kryo serialization (set Java property "spark.serializer" to - * "org.apache.spark.KryoSerializer"). + * "org.apache.spark.serializer.KryoSerializer"). * Usage: RawNetworkGrep * is the Spark master URL * is the number rawNetworkStreams, which should be same as number diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala index 4f4a7f5296e..60cb44ce895 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala @@ -1,6 +1,6 @@ package org.apache.spark.mllib.classification -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD trait ClassificationModel extends Serializable { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 91bb50c829c..50aede9c07d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -19,7 +19,8 @@ package org.apache.spark.mllib.classification import scala.math.round -import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.MLUtils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index c92c7cc3f34..3511e24bce2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -19,7 +19,8 @@ package org.apache.spark.mllib.classification import scala.math.signum -import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.MLUtils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 2c3db099fa8..edbf77dbcc3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -20,8 +20,9 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable.ArrayBuffer import scala.util.Random -import org.apache.spark.{SparkContext, RDD} +import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.RDD import org.apache.spark.Logging import org.apache.spark.mllib.util.MLUtils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index d1fe5d138d3..cfc81c985aa 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.clustering -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.mllib.util.MLUtils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index b62c9b3340e..b77364e08dd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -17,7 +17,8 @@ package org.apache.spark.mllib.optimization -import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.jblas.DoubleMatrix @@ -29,8 +30,9 @@ import scala.collection.mutable.ArrayBuffer * @param gradient Gradient function to be used. * @param updater Updater to be used to update weights after every iteration. */ -class GradientDescent(var gradient: Gradient, var updater: Updater) extends Optimizer { - +class GradientDescent(var gradient: Gradient, var updater: Updater) + extends Optimizer with Logging +{ private var stepSize: Double = 1.0 private var numIterations: Int = 100 private var regParam: Double = 0.0 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala index 50059d385d2..94d30b56f21 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.optimization -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD trait Optimizer { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 218217acfe5..be002d02bcd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -21,9 +21,10 @@ import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.util.Random import scala.util.Sorting -import org.apache.spark.{HashPartitioner, Partitioner, SparkContext, RDD} +import org.apache.spark.{HashPartitioner, Partitioner, SparkContext} import org.apache.spark.storage.StorageLevel -import org.apache.spark.KryoRegistrator +import org.apache.spark.rdd.RDD +import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.SparkContext._ import com.esotericsoftware.kryo.Kryo @@ -432,7 +433,7 @@ object ALS { val (master, ratingsFile, rank, iters, outputDir) = (args(0), args(1), args(2).toInt, args(3).toInt, args(4)) val blocks = if (args.length == 6) args(5).toInt else -1 - System.setProperty("spark.serializer", "org.apache.spark.KryoSerializer") + System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") System.setProperty("spark.kryo.registrator", classOf[ALSRegistrator].getName) System.setProperty("spark.kryo.referenceTracking", "false") System.setProperty("spark.kryoserializer.buffer.mb", "8") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index ae9fe48aec3..af43d89c70f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.recommendation -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.jblas._ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 06015110ac0..f98b0b536de 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -17,7 +17,8 @@ package org.apache.spark.mllib.regression -import org.apache.spark.{Logging, RDD, SparkException} +import org.apache.spark.{Logging, SparkException} +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.jblas.DoubleMatrix diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index df3beb1959d..d9596953259 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -17,7 +17,8 @@ package org.apache.spark.mllib.regression -import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.util.MLUtils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 71f968471cd..ae95ea24fcd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -17,7 +17,8 @@ package org.apache.spark.mllib.regression -import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.util.MLUtils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala index 8dd325efc08..423afc32d66 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.regression -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD trait RegressionModel extends Serializable { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index 228ab9e4e8f..b29508d2b9f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -17,7 +17,8 @@ package org.apache.spark.mllib.regression -import org.apache.spark.{Logging, RDD, SparkContext} +import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.util.MLUtils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala index 7fd46230716..8b55bce7c4b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala @@ -17,7 +17,8 @@ package org.apache.spark.mllib.util -import org.apache.spark.{RDD, Logging} +import org.apache.spark.Logging +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala index 6500d47183e..9109189dff5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala @@ -19,7 +19,8 @@ package org.apache.spark.mllib.util import scala.util.Random -import org.apache.spark.{RDD, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD /** * Generate test data for KMeans. This class first chooses k cluster centers diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 4c49d484b4c..bc5045fb05d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -22,7 +22,8 @@ import scala.util.Random import org.jblas.DoubleMatrix -import org.apache.spark.{RDD, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.regression.LabeledPoint diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala index f553298fc5c..52c4a71d621 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala @@ -19,7 +19,8 @@ package org.apache.spark.mllib.util import scala.util.Random -import org.apache.spark.{RDD, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala index 7eb69ae81cb..5aec867257e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -21,7 +21,8 @@ import scala.util.Random import org.jblas.DoubleMatrix -import org.apache.spark.{RDD, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.util.MLUtils /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 0aeafbe23c7..d91b74c3ac2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -17,7 +17,8 @@ package org.apache.spark.mllib.util -import org.apache.spark.{RDD, SparkContext} +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.jblas.DoubleMatrix diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index d3f191b05b1..6e9f6676356 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -2,9 +2,10 @@ package org.apache.spark.mllib.util import scala.util.Random -import org.apache.spark.{RDD, SparkContext} - import org.jblas.DoubleMatrix + +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint /** diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 906e9221a1d..8fbf2965097 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -114,9 +114,9 @@ def __init__(self, master, jobName, sparkHome=None, pyFiles=None, self.addPyFile(path) # Create a temporary directory inside spark.local.dir: - local_dir = self._jvm.org.apache.spark.Utils.getLocalDir() + local_dir = self._jvm.org.apache.spark.util.Utils.getLocalDir() self._temp_dir = \ - self._jvm.org.apache.spark.Utils.createTempDir(local_dir).getAbsolutePath() + self._jvm.org.apache.spark.util.Utils.createTempDir(local_dir).getAbsolutePath() @property def defaultParallelism(self): diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala index 7e244e48a2a..e6e35c9b5df 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -28,7 +28,7 @@ import scala.reflect.NameTransformer import SparkIMain._ import org.apache.spark.HttpServer -import org.apache.spark.Utils +import org.apache.spark.util.Utils import org.apache.spark.SparkEnv /** An interpreter for Scala code. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala index 362247cc381..80da6bd30b4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala @@ -23,7 +23,8 @@ import org.apache.spark.util.MetadataCleaner //import Time._ -import org.apache.spark.{RDD, Logging} +import org.apache.spark.Logging +import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import scala.collection.mutable.ArrayBuffer diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala b/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala index 290ad378123..6bf275f5afc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming -import org.apache.spark.Utils +import org.apache.spark.util.Utils case class Duration (private val millis: Long) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala index d8a7381e87d..757bc98981c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala @@ -22,8 +22,9 @@ import org.apache.spark.streaming.dstream.{ReducedWindowedDStream, StateDStream} import org.apache.spark.streaming.dstream.{CoGroupedDStream, ShuffledDStream} import org.apache.spark.streaming.dstream.{MapValuedDStream, FlatMapValuedDStream} -import org.apache.spark.{Manifests, RDD, Partitioner, HashPartitioner} +import org.apache.spark.{Partitioner, HashPartitioner} import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.{Manifests, RDD, PairRDDFunctions} import org.apache.spark.storage.StorageLevel import scala.collection.mutable.ArrayBuffer @@ -101,8 +102,8 @@ extends Serializable { /** * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the - * combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.PairRDDFunctions]] for more - * information. + * combineByKey for RDDs. Please refer to combineByKey in + * [[org.apache.spark.rdd.PairRDDFunctions]] for more information. */ def combineByKey[C: ClassManifest]( createCombiner: V => C, @@ -379,7 +380,7 @@ extends Serializable { /** * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of each key. - * [[org.apache.spark.Paxrtitioner]] is used to control the partitioning of each RDD. + * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. * @param updateFunc State update function. If `this` function returns None, then * corresponding state key-value pair will be eliminated. Note, that * this function may generate a different a tuple with a different key diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 3852ac2daba..878725c705d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -24,6 +24,7 @@ import akka.zeromq.Subscribe import org.apache.spark.streaming.dstream._ import org.apache.spark._ +import org.apache.spark.rdd.RDD import org.apache.spark.streaming.receivers.ActorReceiver import org.apache.spark.streaming.receivers.ReceiverSupervisorStrategy import org.apache.spark.streaming.receivers.ZeroMQReceiver diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala index f8c8d8ece1e..d1932b6b05a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala @@ -21,7 +21,7 @@ import org.apache.spark.streaming.{Duration, Time, DStream} import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.api.java.JavaRDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD /** * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index 2e6fe9a9c4f..459695b7cab 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -26,7 +26,7 @@ import org.apache.spark.streaming._ import org.apache.spark.api.java.{JavaPairRDD, JavaRDDLike, JavaRDD} import org.apache.spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _} import java.util -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import JavaDStream._ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index c203dccd171..978fca33ad0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -25,14 +25,15 @@ import scala.collection.JavaConversions._ import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} -import org.apache.spark.{RDD, Partitioner} +import org.apache.spark.Partitioner import org.apache.hadoop.mapred.{JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.hadoop.conf.Configuration import org.apache.spark.api.java.{JavaUtils, JavaRDD, JavaPairRDD} import org.apache.spark.storage.StorageLevel import com.google.common.base.Optional -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.PairRDDFunctions class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( implicit val kManifiest: ClassManifest[K], @@ -147,7 +148,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Combine elements of each key in DStream's RDDs using custom function. This is similar to the - * combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.PairRDDFunctions]] for more + * combineByKey for RDDs. Please refer to combineByKey in [[PairRDDFunctions]] for more * information. */ def combineByKey[C](createCombiner: JFunction[V, C], diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index f10beb1db39..54ba3e60259 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -17,23 +17,26 @@ package org.apache.spark.streaming.api.java -import org.apache.spark.streaming._ -import receivers.{ActorReceiver, ReceiverSupervisorStrategy} -import org.apache.spark.streaming.dstream._ -import org.apache.spark.storage.StorageLevel -import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} -import org.apache.spark.api.java.{JavaSparkContext, JavaRDD} +import java.lang.{Long => JLong, Integer => JInt} +import java.io.InputStream +import java.util.{Map => JMap} + +import scala.collection.JavaConversions._ + 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 -import org.apache.spark.RDD + +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} +import org.apache.spark.api.java.{JavaSparkContext, JavaRDD} +import org.apache.spark.streaming._ +import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.receivers.{ActorReceiver, ReceiverSupervisorStrategy} /** * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala index 4a9d82211fc..4eddc755b97 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala @@ -17,7 +17,8 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.{RDD, Partitioner} +import org.apache.spark.Partitioner +import org.apache.spark.rdd.RDD import org.apache.spark.rdd.CoGroupedRDD import org.apache.spark.streaming.{Time, DStream, Duration} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala index 35cc4cb396b..a9a05c9981f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Time, StreamingContext} /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index 1c265ed972d..fea0573b770 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.rdd.UnionRDD import org.apache.spark.streaming.{DStreamCheckpointData, StreamingContext, Time} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala index 3166c687608..91ee2c1a36f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD private[streaming] class FilteredDStream[T: ClassManifest]( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala index 21950ad6ac9..ca7d7ca49ef 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ private[streaming] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala index 8377cfe60cd..b37966f9a79 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD private[streaming] class FlatMappedDStream[T: ClassManifest, U: ClassManifest]( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala index 3fb443143cd..18de7729468 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala @@ -17,10 +17,11 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.streaming.StreamingContext +import java.net.InetSocketAddress +import java.io.{ObjectInput, ObjectOutput, Externalizable} +import java.nio.ByteBuffer -import org.apache.spark.Utils -import org.apache.spark.storage.StorageLevel +import scala.collection.JavaConversions._ import org.apache.flume.source.avro.AvroSourceProtocol import org.apache.flume.source.avro.AvroFlumeEvent @@ -28,11 +29,9 @@ import org.apache.flume.source.avro.Status import org.apache.avro.ipc.specific.SpecificResponder import org.apache.avro.ipc.NettyServer -import scala.collection.JavaConversions._ - -import java.net.InetSocketAddress -import java.io.{ObjectInput, ObjectOutput, Externalizable} -import java.nio.ByteBuffer +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.util.Utils +import org.apache.spark.storage.StorageLevel private[streaming] class FlumeInputDStream[T: ClassManifest]( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala index c1f95650c8d..e21bac46025 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Duration, DStream, Job, Time} private[streaming] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala index 1e4c7e7fde8..4294b07d910 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD private[streaming] class GlommedDStream[T: ClassManifest](parent: DStream[T]) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala index 1d79d707bb4..5329601a6f9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD private[streaming] class MapPartitionedDStream[T: ClassManifest, U: ClassManifest]( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala index 312e0c0567d..8290df90a28 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ private[streaming] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala index af688dde5f3..b1682afea39 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD private[streaming] class MappedDStream[T: ClassManifest, U: ClassManifest] ( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala index 3d68da36a27..31f98915603 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala @@ -17,22 +17,21 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.streaming.{Time, StreamingContext, AddBlocks, RegisterReceiver, DeregisterReceiver} - -import org.apache.spark.{Logging, SparkEnv, RDD} -import org.apache.spark.rdd.BlockRDD -import org.apache.spark.storage.StorageLevel +import java.util.concurrent.ArrayBlockingQueue +import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import java.nio.ByteBuffer - import akka.actor.{Props, Actor} import akka.pattern.ask import akka.dispatch.Await import akka.util.duration._ + import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} -import java.util.concurrent.ArrayBlockingQueue +import org.apache.spark.streaming._ +import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.rdd.{RDD, BlockRDD} +import org.apache.spark.storage.StorageLevel /** * Abstract class for defining any InputDStream that has to start a receiver on worker diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala index b43ecaeebe0..7d9f3521b1c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.rdd.UnionRDD import scala.collection.mutable.Queue diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala index b6c672f899e..b88a4db9596 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -19,7 +19,7 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.StreamingContext._ -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.rdd.{CoGroupedRDD, MapPartitionsRDD} import org.apache.spark.Partitioner import org.apache.spark.SparkContext._ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala index 3a0bd2acd71..a95e66d7615 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala @@ -17,7 +17,8 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.{RDD, Partitioner} +import org.apache.spark.Partitioner +import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.streaming.{Duration, DStream, Time} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala index c1c9f808f08..362a6bf4cc4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.Partitioner import org.apache.spark.SparkContext._ import org.apache.spark.storage.StorageLevel diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala index edba2032b45..60485adef95 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Duration, DStream, Time} private[streaming] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala index 97eab97b2fa..c696bb70a8f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import collection.mutable.ArrayBuffer import org.apache.spark.rdd.UnionRDD diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala index dbbea39e812..3c572942690 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.RDD +import org.apache.spark.rdd.RDD import org.apache.spark.rdd.UnionRDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Duration, Interval, Time, DStream} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala index 50d72298e48..69779571266 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala @@ -17,7 +17,8 @@ package org.apache.spark.streaming.util -import org.apache.spark.{Logging, RDD} +import org.apache.spark.Logging +import org.apache.spark.rdd.RDD import org.apache.spark.streaming._ import org.apache.spark.streaming.dstream.ForEachDStream import StreamingContext._ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala index 249f6a22aeb..fc8655a0839 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala @@ -20,10 +20,11 @@ package org.apache.spark.streaming.util import java.nio.ByteBuffer import org.apache.spark.util.{RateLimitedOutputStream, IntParam} import java.net.ServerSocket -import org.apache.spark.{Logging, KryoSerializer} +import org.apache.spark.{Logging} import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream import scala.io.Source import java.io.IOException +import org.apache.spark.serializer.KryoSerializer /** * A helper program that sends blocks of Kryo-serialized text strings out on a socket at a diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 31c2fa0208e..37dd9c4cc61 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -20,8 +20,6 @@ package org.apache.spark.streaming import org.apache.spark.streaming.dstream.{InputDStream, ForEachDStream} import org.apache.spark.streaming.util.ManualClock -import org.apache.spark.{RDD, Logging} - import collection.mutable.ArrayBuffer import collection.mutable.SynchronizedBuffer @@ -29,6 +27,9 @@ import java.io.{ObjectInputStream, IOException} import org.scalatest.{BeforeAndAfter, FunSuite} +import org.apache.spark.Logging +import org.apache.spark.rdd.RDD + /** * This is a input stream just for the testsuites. This is equivalent to a checkpointable, * replayable, reliable message queue like Kafka. It requires a sequence as input, and diff --git a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala index 50335e5736b..f824c472aee 100644 --- a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala +++ b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark._ import org.apache.spark.api.java._ -import org.apache.spark.rdd.OrderedRDDFunctions +import org.apache.spark.rdd.{RDD, DoubleRDDFunctions, PairRDDFunctions, OrderedRDDFunctions} import org.apache.spark.streaming.{PairDStreamFunctions, DStream, StreamingContext} import org.apache.spark.streaming.api.java.{JavaPairDStream, JavaDStream, JavaStreamingContext} @@ -139,7 +139,7 @@ object JavaAPICompletenessChecker { scalaType match { case ParameterizedType(name, parameters, typebounds) => name match { - case "org.apache.spark.RDD" => + case "org.apache.spark.rdd.RDD" => if (parameters(0).name == classOf[Tuple2[_, _]].getName) { val tupleParams = parameters(0).asInstanceOf[ParameterizedType].parameters.map(applySubs) @@ -211,23 +211,23 @@ object JavaAPICompletenessChecker { // This list also includes a few methods that are only used by the web UI or other // internal Spark components. val excludedNames = Seq( - "org.apache.spark.RDD.origin", - "org.apache.spark.RDD.elementClassManifest", - "org.apache.spark.RDD.checkpointData", - "org.apache.spark.RDD.partitioner", - "org.apache.spark.RDD.partitions", - "org.apache.spark.RDD.firstParent", - "org.apache.spark.RDD.doCheckpoint", - "org.apache.spark.RDD.markCheckpointed", - "org.apache.spark.RDD.clearDependencies", - "org.apache.spark.RDD.getDependencies", - "org.apache.spark.RDD.getPartitions", - "org.apache.spark.RDD.dependencies", - "org.apache.spark.RDD.getPreferredLocations", - "org.apache.spark.RDD.collectPartitions", - "org.apache.spark.RDD.computeOrReadCheckpoint", - "org.apache.spark.PairRDDFunctions.getKeyClass", - "org.apache.spark.PairRDDFunctions.getValueClass", + "org.apache.spark.rdd.RDD.origin", + "org.apache.spark.rdd.RDD.elementClassManifest", + "org.apache.spark.rdd.RDD.checkpointData", + "org.apache.spark.rdd.RDD.partitioner", + "org.apache.spark.rdd.RDD.partitions", + "org.apache.spark.rdd.RDD.firstParent", + "org.apache.spark.rdd.RDD.doCheckpoint", + "org.apache.spark.rdd.RDD.markCheckpointed", + "org.apache.spark.rdd.RDD.clearDependencies", + "org.apache.spark.rdd.RDD.getDependencies", + "org.apache.spark.rdd.RDD.getPartitions", + "org.apache.spark.rdd.RDD.dependencies", + "org.apache.spark.rdd.RDD.getPreferredLocations", + "org.apache.spark.rdd.RDD.collectPartitions", + "org.apache.spark.rdd.RDD.computeOrReadCheckpoint", + "org.apache.spark.rdd.PairRDDFunctions.getKeyClass", + "org.apache.spark.rdd.PairRDDFunctions.getValueClass", "org.apache.spark.SparkContext.stringToText", "org.apache.spark.SparkContext.makeRDD", "org.apache.spark.SparkContext.runJob", From 3db404a43a90a9cca37090381857dc955496385a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 1 Sep 2013 23:45:57 +0000 Subject: [PATCH 0909/2521] Run script fixes for Windows after package & assembly change --- bin/compute-classpath.cmd | 46 ++++++-------- .../scala/org/apache/spark/SparkContext.scala | 28 +++++---- docs/spark-standalone.md | 4 +- run-example.cmd | 23 +++++++ run-example2.cmd | 61 +++++++++++++++++++ sbt/sbt.cmd | 2 +- spark-class | 2 +- run.cmd => spark-class.cmd | 5 +- run2.cmd => spark-class2.cmd | 56 ++++++----------- spark-shell.cmd | 4 +- 10 files changed, 148 insertions(+), 83 deletions(-) create mode 100644 run-example.cmd create mode 100644 run-example2.cmd rename run.cmd => spark-class.cmd (81%) rename run2.cmd => spark-class2.cmd (58%) diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 9178b852e6b..cf38188c4b2 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -28,30 +28,27 @@ 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 MLLIB_DIR=%FWDIR%mllib -set TOOLS_DIR=%FWDIR%tools -set YARN_DIR=%FWDIR%yarn -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 -set CLASSPATH=%CLASSPATH%;%MLLIB_DIR%\target\scala-%SCALA_VERSION%\classes -set CLASSPATH=%CLASSPATH%;%TOOLS_DIR%\target\scala-%SCALA_VERSION%\classes -set CLASSPATH=%CLASSPATH%;%YARN_DIR%\target\scala-%SCALA_VERSION%\classes +set CLASSPATH=%SPARK_CLASSPATH%;%FWDIR%conf +if exist "%FWDIR%RELEASE" ( + for %%d in ("%FWDIR%jars\spark-assembly*.jar") do ( + set ASSEMBLY_JAR=%%d + ) +) else ( + for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( + set ASSEMBLY_JAR=%%d + ) +) +set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR% + +if "x%SPARK_TESTING%"=="x1" ( + rem Add test clases to path + set CLASSPATH=%CLASSPATH%;%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes + set CLASSPATH=%CLASSPATH%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes + set CLASSPATH=%CLASSPATH%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes + set CLASSPATH=%CLASSPATH%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes + set CLASSPATH=%CLASSPATH%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-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 @@ -64,9 +61,6 @@ 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 "%DONT_PRINT_CLASSPATH%"=="1" goto exit diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index faf0c2362a2..f2641851cbe 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -631,20 +631,26 @@ class SparkContext( * filesystems), or an HTTP, HTTPS or FTP URI. */ def addJar(path: String) { - if (null == path) { + if (path == null) { logWarning("null specified as parameter to addJar", new SparkException("null specified as parameter to addJar")) } else { - val env = SparkEnv.get - val uri = new URI(path) - val key = uri.getScheme match { - case null | "file" => - if (env.hadoop.isYarnMode()) { - logWarning("local jar specified as parameter to addJar under Yarn mode") - return - } - env.httpFileServer.addJar(new File(uri.getPath)) - case _ => path + var key = "" + if (path.contains("\\")) { + // For local paths with backslashes on Windows, URI throws an exception + key = env.httpFileServer.addJar(new File(path)) + } else { + val uri = new URI(path) + key = uri.getScheme match { + case null | "file" => + if (env.hadoop.isYarnMode()) { + logWarning("local jar specified as parameter to addJar under Yarn mode") + return + } + env.httpFileServer.addJar(new File(uri.getPath)) + case _ => + path + } } addedJars(key) = System.currentTimeMillis logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 9ab6ba0830f..994a96f2c99 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -9,7 +9,7 @@ In addition to running on the Mesos or YARN cluster managers, Spark also provide You can start a standalone master server by executing: - ./spark-class spark.deploy.master.Master + ./spark-class org.apache.spark.deploy.master.Master Once started, the master will print out a `spark://HOST:PORT` URL for itself, which you can use to connect workers to it, or pass as the "master" argument to `SparkContext`. You can also find this URL on @@ -17,7 +17,7 @@ the master's web UI, which is [http://localhost:8080](http://localhost:8080) by Similarly, you can start one or more workers and connect them to the master via: - ./spark-class spark.deploy.worker.Worker spark://IP:PORT + ./spark-class org.apache.spark.deploy.worker.Worker spark://IP:PORT Once you have started a worker, look at the master's web UI ([http://localhost:8080](http://localhost:8080) by default). You should see the new node listed there, along with its number of CPUs and memory (minus one gigabyte left for the OS). diff --git a/run-example.cmd b/run-example.cmd new file mode 100644 index 00000000000..5b2d048d6ed --- /dev/null +++ b/run-example.cmd @@ -0,0 +1,23 @@ +@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 is the entry point for running a Spark example. To avoid polluting +rem the environment, it just launches a new cmd to do the real work. + +cmd /V /E /C %~dp0run-example2.cmd %* diff --git a/run-example2.cmd b/run-example2.cmd new file mode 100644 index 00000000000..dbb371ecfc2 --- /dev/null +++ b/run-example2.cmd @@ -0,0 +1,61 @@ +@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 +set FWDIR=%~dp0 + +rem Export this as SPARK_HOME +set SPARK_HOME=%FWDIR% + +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" + +rem Test that an argument was given +if not "x%1"=="x" goto arg_given + echo Usage: run-example ^ [^] + goto exit +:arg_given + +set EXAMPLES_DIR=%FWDIR%examples + +rem Figure out the JAR file that our examples were packaged into. +set SPARK_EXAMPLES_JAR= +for %%d in ("%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\spark-examples*assembly*.jar") do ( + set SPARK_EXAMPLES_JAR=%%d +) +if "x%SPARK_EXAMPLES_JAR%"=="x" ( + echo Failed to find Spark examples assembly JAR. + echo You need to build Spark with sbt\sbt assembly before running this program. + goto exit +) + +rem Compute Spark classpath using external script +set DONT_PRINT_CLASSPATH=1 +call "%FWDIR%bin\compute-classpath.cmd" +set DONT_PRINT_CLASSPATH=0 +set CLASSPATH=%SPARK_EXAMPLES_JAR%;%CLASSPATH% + +rem Figure out where java is. +set RUNNER=java +if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java + +"%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %* +:exit diff --git a/sbt/sbt.cmd b/sbt/sbt.cmd index 56a16d77f21..681fe00f921 100644 --- a/sbt/sbt.cmd +++ b/sbt/sbt.cmd @@ -22,4 +22,4 @@ if not "%MESOS_HOME%x"=="x" set EXTRA_ARGS=-Djava.library.path=%MESOS_HOME%\lib\ set SPARK_HOME=%~dp0.. -java -Xmx1200M -XX:MaxPermSize=200m %EXTRA_ARGS% -jar %SPARK_HOME%\sbt\sbt-launch-0.11.3-2.jar "%*" +java -Xmx1200M -XX:MaxPermSize=200m -XX:ReservedCodeCacheSize=256m %EXTRA_ARGS% -jar %SPARK_HOME%\sbt\sbt-launch-0.11.3-2.jar "%*" diff --git a/spark-class b/spark-class index 71dfb9c4e68..037abda3b71 100755 --- a/spark-class +++ b/spark-class @@ -31,7 +31,7 @@ if [ -e $FWDIR/conf/spark-env.sh ] ; then fi if [ -z "$1" ]; then - echo "Usage: run []" >&2 + echo "Usage: spark-class []" >&2 exit 1 fi diff --git a/run.cmd b/spark-class.cmd similarity index 81% rename from run.cmd rename to spark-class.cmd index c91764e6179..19850db9e1e 100644 --- a/run.cmd +++ b/spark-class.cmd @@ -17,4 +17,7 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem -cmd /V /E /C %~dp0run2.cmd %* +rem This is the entry point for running a Spark class. To avoid polluting +rem the environment, it just launches a new cmd to do the real work. + +cmd /V /E /C %~dp0spark-class2.cmd %* diff --git a/run2.cmd b/spark-class2.cmd similarity index 58% rename from run2.cmd rename to spark-class2.cmd index dc5e50931e3..d4d853e8ad9 100644 --- a/run2.cmd +++ b/spark-class2.cmd @@ -30,7 +30,7 @@ if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" rem Test that an argument was given if not "x%1"=="x" goto arg_given - echo Usage: run ^ [^] + echo Usage: spark-class ^ [^] goto exit :arg_given @@ -44,12 +44,6 @@ 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 - echo SCALA_HOME is not set - goto exit -:scala_exists - 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 @@ -58,43 +52,27 @@ 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 Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! -set CORE_DIR=%FWDIR%core -set EXAMPLES_DIR=%FWDIR%examples -set REPL_DIR=%FWDIR%repl +rem Test whether the user has built Spark +if exist "%FWDIR%RELEASE" goto skip_build_test +set FOUND_JAR=0 +for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( + set FOUND_JAR=1 +) +if "%FOUND_JAR%"=="0" ( + echo Failed to find Spark assembly JAR. + echo You need to build Spark with sbt\sbt assembly before running this program. + goto exit +) +:skip_build_test rem Compute classpath using external script set DONT_PRINT_CLASSPATH=1 call "%FWDIR%bin\compute-classpath.cmd" set DONT_PRINT_CLASSPATH=0 -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 ( - set SPARK_EXAMPLES_JAR=examples/target/scala-%SCALA_VERSION%/%%d -) -rem Then search in the build path from Maven: -for %%d in ("examples/target/spark-examples*hadoop*.jar") do ( - set SPARK_EXAMPLES_JAR=examples/target/%%d -) - -rem Figure out whether to run our class with java or with the scala launcher. -rem In most cases, we'd prefer to execute our process with java because scala -rem creates a shell script as the parent of its Java process, which makes it -rem hard to kill the child with stuff like Process.destroy(). However, for -rem the Spark shell, the wrapper is necessary to properly reset the terminal -rem when we exit, so we allow it to set a variable to launch with scala. -if "%SPARK_LAUNCH_WITH_SCALA%" NEQ 1 goto java_runner - set RUNNER=%SCALA_HOME%\bin\scala - # Java options will be passed to scala as JAVA_OPTS - set EXTRA_ARGS= - goto run_spark -:java_runner - set CLASSPATH=%CLASSPATH%;%SCALA_HOME%\lib\scala-library.jar;%SCALA_HOME%\lib\scala-compiler.jar;%SCALA_HOME%\lib\jline.jar - set RUNNER=java - if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java - rem The JVM doesn't read JAVA_OPTS by default so we need to pass it in - set EXTRA_ARGS=%JAVA_OPTS% -:run_spark +rem Figure out where java is. +set RUNNER=java +if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java -"%RUNNER%" -cp "%CLASSPATH%" %EXTRA_ARGS% %* +"%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %* :exit diff --git a/spark-shell.cmd b/spark-shell.cmd index b9b4d4bfb2b..3e52bf835e5 100644 --- a/spark-shell.cmd +++ b/spark-shell.cmd @@ -18,5 +18,5 @@ rem limitations under the License. rem set FWDIR=%~dp0 -set SPARK_LAUNCH_WITH_SCALA=1 -cmd /V /E /C %FWDIR%run2.cmd org.apache.spark.repl.Main %* + +cmd /V /E /C %FWDIR%spark-class2.cmd org.apache.spark.repl.Main %* From 6550e5e60c501cbce40f0e968fc674e499f21949 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 1 Sep 2013 18:06:15 -0700 Subject: [PATCH 0910/2521] Allow PySpark to launch worker.py directly on Windows --- .../api/python/PythonWorkerFactory.scala | 107 ++++++++++++++++-- python/pyspark/worker.py | 11 +- 2 files changed, 106 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 08e3f670f57..67d45723bad 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -17,8 +17,8 @@ package org.apache.spark.api.python -import java.io.{File, DataInputStream, IOException} -import java.net.{Socket, SocketException, InetAddress} +import java.io.{OutputStreamWriter, File, DataInputStream, IOException} +import java.net.{ServerSocket, Socket, SocketException, InetAddress} import scala.collection.JavaConversions._ @@ -26,11 +26,30 @@ import org.apache.spark._ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String, String]) extends Logging { + + // Because forking processes from Java is expensive, we prefer to launch a single Python daemon + // (pyspark/daemon.py) and tell it to fork new workers for our tasks. This daemon currently + // only works on UNIX-based systems now because it uses signals for child management, so we can + // also fall back to launching workers (pyspark/worker.py) directly. + val useDaemon = !System.getProperty("os.name").startsWith("Windows") + var daemon: Process = null val daemonHost = InetAddress.getByAddress(Array(127, 0, 0, 1)) var daemonPort: Int = 0 def create(): Socket = { + if (useDaemon) { + createThroughDaemon() + } else { + createSimpleWorker() + } + } + + /** + * Connect to a worker launched through pyspark/daemon.py, which forks python processes itself + * to avoid the high cost of forking from Java. This currently only works on UNIX-based systems. + */ + private def createThroughDaemon(): Socket = { synchronized { // Start the daemon if it hasn't been started startDaemon() @@ -50,6 +69,78 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } } + /** + * Launch a worker by executing worker.py directly and telling it to connect to us. + */ + private def createSimpleWorker(): Socket = { + var serverSocket: ServerSocket = null + try { + serverSocket = new ServerSocket(0, 1, InetAddress.getByAddress(Array(127, 0, 0, 1))) + + // Create and start the worker + val sparkHome = new ProcessBuilder().environment().get("SPARK_HOME") + val pb = new ProcessBuilder(Seq(pythonExec, sparkHome + "/python/pyspark/worker.py")) + val workerEnv = pb.environment() + workerEnv.putAll(envVars) + val pythonPath = sparkHome + "/python/" + File.pathSeparator + workerEnv.get("PYTHONPATH") + workerEnv.put("PYTHONPATH", pythonPath) + val worker = pb.start() + + // Redirect the worker's stderr to ours + new Thread("stderr reader for " + pythonExec) { + setDaemon(true) + override def run() { + scala.util.control.Exception.ignoring(classOf[IOException]) { + // FIXME: We copy the stream on the level of bytes to avoid encoding problems. + val in = worker.getErrorStream + val buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + System.err.write(buf, 0, len) + len = in.read(buf) + } + } + } + }.start() + + // Redirect worker's stdout to our stderr + new Thread("stdout reader for " + pythonExec) { + setDaemon(true) + override def run() { + scala.util.control.Exception.ignoring(classOf[IOException]) { + // FIXME: We copy the stream on the level of bytes to avoid encoding problems. + val in = worker.getInputStream + val buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + System.err.write(buf, 0, len) + len = in.read(buf) + } + } + } + }.start() + + // Tell the worker our port + val out = new OutputStreamWriter(worker.getOutputStream) + out.write(serverSocket.getLocalPort + "\n") + out.flush() + + // Wait for it to connect to our socket + serverSocket.setSoTimeout(10000) + try { + return serverSocket.accept() + } catch { + case e: Exception => + throw new SparkException("Python worker did not connect back in time", e) + } + } finally { + if (serverSocket != null) { + serverSocket.close() + } + } + null + } + def stop() { stopDaemon() } @@ -73,12 +164,12 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String // Redirect the stderr to ours new Thread("stderr reader for " + pythonExec) { + setDaemon(true) 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. + // FIXME: We copy the stream on the level of bytes to avoid encoding problems. val in = daemon.getErrorStream - var buf = new Array[Byte](1024) + val buf = new Array[Byte](1024) var len = in.read(buf) while (len != -1) { System.err.write(buf, 0, len) @@ -93,11 +184,11 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String // Redirect further stdout output to our stderr new Thread("stdout reader for " + pythonExec) { + setDaemon(true) 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) + // FIXME: We copy the stream on the level of bytes to avoid encoding problems. + val buf = new Array[Byte](1024) var len = in.read(buf) while (len != -1) { System.err.write(buf, 0, len) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 695f6dfb844..d63c2aaef77 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -21,6 +21,7 @@ import os import sys import time +import socket import traceback from base64 import standard_b64decode # CloudPickler needs to be imported so that depicklers are registered using the @@ -94,7 +95,9 @@ def main(infile, outfile): if __name__ == '__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) + # Read a local port to connect to from stdin + java_port = int(sys.stdin.readline()) + sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + sock.connect(("127.0.0.1", java_port)) + sock_file = sock.makefile("a+", 65536) + main(sock_file, sock_file) From 141f54279e538e36e0506eb37b51df90dfa27358 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 2 Sep 2013 01:19:29 +0000 Subject: [PATCH 0911/2521] Further fixes to get PySpark to work on Windows --- pyspark.cmd | 23 ++++++++++++++ pyspark2.cmd | 55 ++++++++++++++++++++++++++++++++++ python/pyspark/java_gateway.py | 17 +++++++---- 3 files changed, 90 insertions(+), 5 deletions(-) create mode 100644 pyspark.cmd create mode 100644 pyspark2.cmd diff --git a/pyspark.cmd b/pyspark.cmd new file mode 100644 index 00000000000..7c26fbbac28 --- /dev/null +++ b/pyspark.cmd @@ -0,0 +1,23 @@ +@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 is the entry point for running PySpark. To avoid polluting the +rem environment, it just launches a new cmd to do the real work. + +cmd /V /E /C %~dp0pyspark2.cmd %* diff --git a/pyspark2.cmd b/pyspark2.cmd new file mode 100644 index 00000000000..f58e3496439 --- /dev/null +++ b/pyspark2.cmd @@ -0,0 +1,55 @@ +@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 +set FWDIR=%~dp0 + +rem Export this as SPARK_HOME +set SPARK_HOME=%FWDIR% + +rem Test whether the user has built Spark +if exist "%FWDIR%RELEASE" goto skip_build_test +set FOUND_JAR=0 +for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( + set FOUND_JAR=1 +) +if "%FOUND_JAR%"=="0" ( + echo Failed to find Spark assembly JAR. + echo You need to build Spark with sbt\sbt assembly before running this program. + goto exit +) +:skip_build_test + +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" + +rem Figure out which Python to use. +if "x%PYSPARK_PYTHON%"=="x" set PYSPARK_PYTHON=python + +set PYTHONPATH=%FWDIR%python;%PYTHONPATH% + +set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% +set PYTHONSTARTUP=%FWDIR%python\pyspark\shell.py + +echo Running %PYSPARK_PYTHON% with PYTHONPATH=%PYTHONPATH% + +"%PYSPARK_PYTHON%" %* +:exit diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 26fbe0f0804..e615c1e9b63 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -18,6 +18,7 @@ import os import sys import signal +import platform from subprocess import Popen, PIPE from threading import Thread from py4j.java_gateway import java_import, JavaGateway, GatewayClient @@ -29,12 +30,18 @@ def launch_gateway(): # Launch the Py4j gateway using Spark's run command so that we pick up the # proper classpath and SPARK_MEM settings from spark-env.sh - command = [os.path.join(SPARK_HOME, "spark-class"), "py4j.GatewayServer", + on_windows = platform.system() == "Windows" + script = "spark-class.cmd" if on_windows else "spark-class" + command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer", "--die-on-broken-pipe", "0"] - # Don't send ctrl-c / SIGINT to the Java gateway: - def preexec_function(): - signal.signal(signal.SIGINT, signal.SIG_IGN) - proc = Popen(command, stdout=PIPE, stdin=PIPE, preexec_fn=preexec_function) + if not on_windows: + # Don't send ctrl-c / SIGINT to the Java gateway: + def preexec_func(): + signal.signal(signal.SIGINT, signal.SIG_IGN) + proc = Popen(command, stdout=PIPE, stdin=PIPE, preexec_fn=preexec_func) + else: + # preexec_fn not supported on Windows + proc = Popen(command, stdout=PIPE, stdin=PIPE) # Determine which ephemeral port the server started on: port = int(proc.stdout.readline()) # Create a thread to echo output from the GatewayServer, which is required From 9ee1e9db2ea4ae4794817ee6b4603ba869aaa7d4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 1 Sep 2013 22:12:03 -0700 Subject: [PATCH 0912/2521] Doc improvements --- docs/configuration.md | 4 +++- docs/index.md | 4 ++++ docs/python-programming-guide.md | 36 ++++++++++++++++---------------- 3 files changed, 25 insertions(+), 19 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 58e9434bdc0..aaf85ed4f4c 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -315,7 +315,7 @@ Apart from these, the following properties are also available, and may be useful # Environment Variables Certain Spark settings can also be configured through environment variables, which are read from the `conf/spark-env.sh` -script in the directory where Spark is installed. These variables are meant to be for machine-specific settings, such +script in the directory where Spark is installed (or `conf/spark-env.cmd` on Windows). These variables are meant to be for machine-specific settings, such as library search paths. While Java system properties can also be set here, for application settings, we recommend setting these properties within the application instead of in `spark-env.sh` so that different applications can use different settings. @@ -325,6 +325,8 @@ Note that `conf/spark-env.sh` does not exist by default when Spark is installed. The following variables can be set in `spark-env.sh`: +* `JAVA_HOME`, the location where Java is installed (if it's not on your default `PATH`) +* `PYSPARK_PYTHON`, the Python binary to use for PySpark * `SPARK_LOCAL_IP`, to configure which IP address of the machine to bind to. * `SPARK_LIBRARY_PATH`, to add search directories for native libraries. * `SPARK_CLASSPATH`, to add elements to Spark's classpath that you want to be present for _all_ applications. diff --git a/docs/index.md b/docs/index.md index 3cf9cc1c641..c7018d88468 100644 --- a/docs/index.md +++ b/docs/index.md @@ -11,6 +11,8 @@ Spark can run on the Apache Mesos cluster manager, Hadoop YARN, Amazon EC2, or w Get Spark by visiting the [downloads page](http://spark.incubator.apache.org/downloads.html) of the Apache Spark site. This documentation is for Spark version {{site.SPARK_VERSION}}. +Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS). All you need to run it is to have `java` to installed on your system `PATH`, or the `JAVA_HOME` environment variable pointing to a Java installation. + # Building Spark uses [Simple Build Tool](http://www.scala-sbt.org), which is bundled with it. To compile the code, go into the top-level Spark directory and run @@ -50,6 +52,8 @@ In addition, if you wish to run Spark on [YARN](running-on-yarn.md), set SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly +(Note that on Windows, you need to set the environment variables on separate lines, e.g., `set SPARK_HADOOP_VERSION=1.2.1`.) + # Where to Go from Here **Programming guides:** diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 8a539fe774f..8c33a953a44 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -4,7 +4,7 @@ title: Python Programming Guide --- -The Spark Python API (PySpark) exposes most of the Spark features available in the Scala version to Python. +The Spark Python API (PySpark) exposes the Spark programming model to Python. To learn the basics of Spark, we recommend reading through the [Scala programming guide](scala-programming-guide.html) first; it should be easy to follow even if you don't know Scala. @@ -15,12 +15,8 @@ This guide will show how to use the Spark features described there in Python. 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: - - `lookup` - - `sort` - - `persist` at storage levels other than `MEMORY_ONLY` - - Execution on Windows -- this is slated for a future release +* Python is dynamically typed, so RDDs can hold objects of multiple types. +* PySpark does not yet support a few API calls, such as `lookup`, `sort`, and `persist` at custom storage levels. See the [API docs](api/pyspark/index.html) for details. 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: @@ -30,7 +26,7 @@ logData = sc.textFile(logFile).cache() errors = logData.filter(lambda line: "ERROR" in line) {% endhighlight %} -You can also pass functions that are defined using the `def` keyword; this is useful for more complicated functions that cannot be expressed using `lambda`: +You can also pass functions that are defined with the `def` keyword; this is useful for longer functions that can't be expressed using `lambda`: {% highlight python %} def is_error(line): @@ -38,7 +34,7 @@ def is_error(line): errors = logData.filter(is_error) {% endhighlight %} -Functions can access objects in enclosing scopes, although modifications to those objects within RDD methods will not be propagated to other tasks: +Functions can access objects in enclosing scopes, although modifications to those objects within RDD methods will not be propagated back: {% highlight python %} error_keywords = ["Exception", "Error"] @@ -51,17 +47,20 @@ PySpark will automatically ship these functions to workers, along with any objec Instances of classes will be serialized and shipped to workers by PySpark, but classes themselves cannot be automatically distributed to workers. The [Standalone Use](#standalone-use) section describes how to ship code dependencies to workers. +In addition, PySpark fully supports interactive use---simply run `./pyspark` to launch an interactive shell. + # Installing and Configuring PySpark PySpark requires Python 2.6 or higher. -PySpark jobs are executed using a standard cPython interpreter in order to support Python modules that use C extensions. +PySpark jobs are executed using a standard CPython interpreter in order to support Python modules that use C extensions. We have not tested PySpark with Python 3 or with alternative Python interpreters, such as [PyPy](http://pypy.org/) or [Jython](http://www.jython.org/). -By default, PySpark's scripts will run programs using `python`; an alternate Python executable may be specified by setting the `PYSPARK_PYTHON` environment variable in `conf/spark-env.sh`. + +By default, PySpark requires `python` to be available on the system `PATH` and use it to run programs; an alternate Python executable may be specified by setting the `PYSPARK_PYTHON` environment variable in `conf/spark-env.sh` (or `.cmd` on Windows). All of PySpark's library dependencies, including [Py4J](http://py4j.sourceforge.net/), are bundled with PySpark and automatically imported. -Standalone PySpark jobs should be run using the `pyspark` script, which automatically configures the Java and Python environment using the settings in `conf/spark-env.sh`. +Standalone PySpark jobs should be run using the `pyspark` script, which automatically configures the Java and Python environment using the settings in `conf/spark-env.sh` or `.cmd`. The script automatically adds the `pyspark` package to the `PYTHONPATH`. @@ -101,7 +100,7 @@ $ MASTER=local[4] ./pyspark ## IPython It is also possible to launch PySpark in [IPython](http://ipython.org), the enhanced Python interpreter. -To do this, simply set the `IPYTHON` variable to `1` when running `pyspark`: +To do this, set the `IPYTHON` variable to `1` when running `pyspark`: {% highlight bash %} $ IPYTHON=1 ./pyspark @@ -132,15 +131,16 @@ sc = SparkContext("local", "Job Name", pyFiles=['MyFile.py', 'lib.zip', 'app.egg Files listed here will be added to the `PYTHONPATH` and shipped to remote worker machines. Code dependencies can be added to an existing SparkContext using its `addPyFile()` method. +# API Docs + +[API documentation](api/pyspark/index.html) for PySpark is available as Epydoc. +Many of the methods also contain [doctests](http://docs.python.org/2/library/doctest.html) that provide additional usage examples. # Where to Go from Here -PySpark includes several sample programs in the [`python/examples` folder](https://github.com/apache/incubator-spark/tree/master/python/examples). -You can run them by passing the files to the `pyspark` script; e.g.: +PySpark also includes several sample programs in the [`python/examples` folder](https://github.com/apache/incubator-spark/tree/master/python/examples). +You can run them by passing the files to `pyspark`; e.g.: ./pyspark python/examples/wordcount.py Each program prints usage help when run without arguments. - -We currently provide [API documentation](api/pyspark/index.html) for the Python API as Epydoc. -Many of the RDD method descriptions contain [doctests](http://docs.python.org/2/library/doctest.html) that provide additional usage examples. From 9329a7d4cd10a080dad451e4173c85f2a04809d6 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 2 Sep 2013 10:15:22 -0700 Subject: [PATCH 0913/2521] Fix spark.io.compression.codec and change default codec to LZF --- .../scala/org/apache/spark/io/CompressionCodec.scala | 12 ++++-------- docs/configuration.md | 2 +- 2 files changed, 5 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 90a0420cafb..570a979b568 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -39,17 +39,13 @@ trait CompressionCodec { private[spark] object CompressionCodec { def createCodec(): CompressionCodec = { - // Set the default codec to Snappy since the LZF implementation initializes a pretty large - // buffer for every stream, which results in a lot of memory overhead when the number of - // shuffle reduce buckets are large. - createCodec(classOf[SnappyCompressionCodec].getName) + createCodec(System.getProperty( + "spark.io.compression.codec", classOf[LZFCompressionCodec].getName)) } def createCodec(codecName: String): CompressionCodec = { - Class.forName( - System.getProperty("spark.io.compression.codec", codecName), - true, - Thread.currentThread.getContextClassLoader).newInstance().asInstanceOf[CompressionCodec] + Class.forName(codecName, true, Thread.currentThread.getContextClassLoader) + .newInstance().asInstanceOf[CompressionCodec] } } diff --git a/docs/configuration.md b/docs/configuration.md index aaf85ed4f4c..310e78a9ebc 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -147,7 +147,7 @@ Apart from these, the following properties are also available, and may be useful spark.io.compression.codec - org.apache.spark.io.
    SnappyCompressionCodec + org.apache.spark.io.
    LZFCompressionCodec The compression codec class to use for various compressions. By default, Spark provides two codecs: org.apache.spark.io.LZFCompressionCodec and org.apache.spark.io.SnappyCompressionCodec. From 246bf67f58fab00df736abc20a0f3740920e0083 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 2 Sep 2013 10:57:34 -0700 Subject: [PATCH 0914/2521] Fix test --- .../test/scala/org/apache/spark/io/CompressionCodecSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala index fd6f69041a9..ab81bfbe559 100644 --- a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala @@ -44,7 +44,7 @@ class CompressionCodecSuite extends FunSuite { test("default compression codec") { val codec = CompressionCodec.createCodec() - assert(codec.getClass === classOf[SnappyCompressionCodec]) + assert(codec.getClass === classOf[LZFCompressionCodec]) testCodec(codec) } From 2ba695292a01da4ce02e5581b6c33b7361bfd722 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 2 Sep 2013 12:22:52 -0700 Subject: [PATCH 0915/2521] Exclude some private modules in epydoc --- python/epydoc.conf | 1 + 1 file changed, 1 insertion(+) diff --git a/python/epydoc.conf b/python/epydoc.conf index d5d5aa5454b..1d0d002d366 100644 --- a/python/epydoc.conf +++ b/python/epydoc.conf @@ -34,3 +34,4 @@ private: no exclude: pyspark.cloudpickle pyspark.worker pyspark.join pyspark.serializers pyspark.java_gateway pyspark.examples pyspark.shell pyspark.test + pyspark.rddsampler pyspark.daemon From 12b2f1f9c9044a91018e5d5644dee33a65d9da95 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 2 Sep 2013 12:23:03 -0700 Subject: [PATCH 0916/2521] Add missing license headers found with RAT --- .../org/apache/spark/ui/static/webui.css | 17 +++++++++++++++++ .../deploy/master/ApplicationSource.scala | 17 +++++++++++++++++ .../spark/deploy/master/MasterSource.scala | 17 +++++++++++++++++ .../spark/deploy/worker/WorkerSource.scala | 17 +++++++++++++++++ .../spark/executor/ExecutorSource.scala | 17 +++++++++++++++++ .../main/scala/org/apache/spark/package.scala | 4 +--- .../spark/scheduler/DAGSchedulerSource.scala | 17 +++++++++++++++++ .../spark/storage/BlockManagerSource.scala | 17 +++++++++++++++++ .../apache/spark/ui/exec/ExecutorsUI.scala | 17 +++++++++++++++++ .../spark/ui/jobs/JobProgressListener.scala | 17 +++++++++++++++++ .../org/apache/spark/ui/jobs/PoolPage.scala | 17 +++++++++++++++++ .../org/apache/spark/ui/jobs/PoolTable.scala | 17 +++++++++++++++++ .../org/apache/spark/ui/jobs/StageTable.scala | 17 +++++++++++++++++ .../resources/test_metrics_config.properties | 17 +++++++++++++++++ .../resources/test_metrics_system.properties | 17 +++++++++++++++++ .../spark/PartitionPruningRDDSuite.scala | 17 +++++++++++++++++ .../apache/spark/examples/SparkPageRank.scala | 17 +++++++++++++++++ .../classification/ClassificationModel.scala | 17 +++++++++++++++++ .../spark/mllib/util/SVMDataGenerator.scala | 17 +++++++++++++++++ python/pyspark/__init__.py | 19 ++++++++++++++++++- yarn/pom.xml | 4 ++-- 21 files changed, 327 insertions(+), 6 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index a600c06c049..fe54c34ffb1 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + .navbar { height: 50px; font-size: 15px; diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala index 2d75ad5a2c9..5a24042e14b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.deploy.master import com.codahale.metrics.{Gauge, MetricRegistry} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala index 8dd0a42f717..23d1cb77da6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.deploy.master import com.codahale.metrics.{Gauge, MetricRegistry} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala index 6427c0178fa..df269fd0477 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.deploy.worker import com.codahale.metrics.{Gauge, MetricRegistry} diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index 17653cd5608..bf8fb4fd21f 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.executor import com.codahale.metrics.{Gauge, MetricRegistry} diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index c0ec527339b..f132e2b735e 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -1,5 +1,3 @@ -import org.apache.spark.rdd.{SequenceFileRDDFunctions, DoubleRDDFunctions, PairRDDFunctions} - /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -30,6 +28,6 @@ import org.apache.spark.rdd.{SequenceFileRDDFunctions, DoubleRDDFunctions, PairR * type (e.g. RDD[(Int, Int)] through implicit conversions when you * `import org.apache.spark.SparkContext._`. */ -package object spark { +package object spark { // For package docs only } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala index ce0dc9093d8..22e3723ac8e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.scheduler import com.codahale.metrics.{Gauge,MetricRegistry} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index 24190cdd675..3d709cfde49 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.storage import com.codahale.metrics.{Gauge,MetricRegistry} diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 6e56c22d04a..d1868dcf786 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.ui.exec import javax.servlet.http.HttpServletRequest diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 86e0af0399f..e2bcd985459 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.ui.jobs import scala.Seq diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index ce92b6932b3..89fffcb80d0 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index f31465e59da..5670c933bd2 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.ui.jobs import scala.collection.mutable.HashMap diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 71e58a977ef..a8911e46aef 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.ui.jobs import java.util.Date diff --git a/core/src/test/resources/test_metrics_config.properties b/core/src/test/resources/test_metrics_config.properties index 056a1584569..da2b29b9ce2 100644 --- a/core/src/test/resources/test_metrics_config.properties +++ b/core/src/test/resources/test_metrics_config.properties @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + *.sink.console.period = 10 *.sink.console.unit = seconds *.source.jvm.class = org.apache.spark.metrics.source.JvmSource diff --git a/core/src/test/resources/test_metrics_system.properties b/core/src/test/resources/test_metrics_system.properties index 6f5ecea93a3..35d0bd3b8d0 100644 --- a/core/src/test/resources/test_metrics_system.properties +++ b/core/src/test/resources/test_metrics_system.properties @@ -1,3 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + *.sink.console.period = 10 *.sink.console.unit = seconds test.sink.console.class = org.apache.spark.metrics.sink.ConsoleSink diff --git a/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala b/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala index 5a18dd13ff5..21f16ef2c6e 100644 --- a/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitionPruningRDDSuite.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark import org.scalatest.FunSuite diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index 2721caf08bd..a508c0df577 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.examples import org.apache.spark.SparkContext._ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala index 60cb44ce895..391f5b9b7a7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.mllib.classification import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index 6e9f6676356..07022093f30 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.spark.mllib.util import scala.util.Random diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index 3e8bca62f06..fd5972d381a 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -1,5 +1,22 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + """ -PySpark is a Python API for Spark. +PySpark is the Python API for Spark. Public classes: diff --git a/yarn/pom.xml b/yarn/pom.xml index fcacdac9eb8..654b5bcd2dc 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -1,5 +1,6 @@ - - 4.0.0 From 2615cad30b0dff32ffc0d070a4f763265c603514 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 2 Sep 2013 13:35:28 -0700 Subject: [PATCH 0917/2521] Some doc improvements - List higher-level projects that run on Spark - Tweak CSS --- docs/bagel-programming-guide.md | 2 +- docs/css/bootstrap.min.css | 2 +- docs/index.md | 12 +++++++++++- 3 files changed, 13 insertions(+), 3 deletions(-) diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md index 583684913d6..140190a38cf 100644 --- a/docs/bagel-programming-guide.md +++ b/docs/bagel-programming-guide.md @@ -3,7 +3,7 @@ layout: global title: Bagel Programming Guide --- -**Bagel** is a Spark implementation of Google's [Pregel](http://portal.acm.org/citation.cfm?id=1807184) graph processing framework. Bagel currently supports basic graph computation, combiners, and aggregators. +Bagel is a Spark implementation of Google's [Pregel](http://portal.acm.org/citation.cfm?id=1807184) graph processing framework. Bagel currently supports basic graph computation, combiners, and aggregators. In the Pregel programming model, jobs run as a sequence of iterations called _supersteps_. In each superstep, each vertex in the graph runs a user-specified function that can update state associated with the vertex and send messages to other vertices for use in the *next* iteration. diff --git a/docs/css/bootstrap.min.css b/docs/css/bootstrap.min.css index 3444ed79e24..ccb529eabab 100644 --- a/docs/css/bootstrap.min.css +++ b/docs/css/bootstrap.min.css @@ -6,4 +6,4 @@ * 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{height:auto;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;line-height:0;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:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-static-top .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;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-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.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:20px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:1;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1{font-size:36px;line-height:40px}h2{font-size:30px;line-height:40px}h3{font-size:24px;line-height:40px}h4{font-size:18px;line-height:20px}h5{font-size:14px;line-height:20px}h6{font-size:12px;line-height:20px}h1 small{font-size:24px}h2 small{font-size:18px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.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:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}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 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:25px}blockquote small{display:block;line-height:20px;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}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,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:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;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:20px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}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:20px;padding:4px 6px;margin-bottom:9px;font-size:14px;line-height:20px;color:#555;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}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-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;-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:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal;cursor:pointer}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;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}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-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)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.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[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .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}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"]{float:left}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}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;-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)}.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:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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;-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)}.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:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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;-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)}.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:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{margin-bottom:5px;font-size:0;white-space:nowrap}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;font-size:14px;vertical-align:top;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn{margin-left:-1px;vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .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 .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}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 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;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{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:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;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:10px;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:20px}.table th,.table td{padding:8px;line-height:20px;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:collapse;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,.table-bordered tfoot: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,.table-bordered tfoot: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-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-right-topleft:4px}.table-striped tbody tr:nth-child(odd) td,.table-striped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table-hover tbody tr:hover td,.table-hover tbody tr:hover th{background-color:#f5f5f5}table [class*=span],.row-fluid table [class*=span]{display:table-cell;float:none;margin-left:0}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}.table tbody tr.success td{background-color:#dff0d8}.table tbody tr.error td{background-color:#f2dede}.table tbody tr.info td{background-color:#d9edf7}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*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}.icon-white,.nav>.active>a>[class^="icon-"],.nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"]{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{width:16px;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{width:16px;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:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 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:6px;-moz-border-radius:6px;border-radius:6px;-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:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu li>a:focus,.dropdown-submenu:hover>a{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0088cc;background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0081c2;background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-repeat:repeat-x;outline:0;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .disabled>a,.dropdown-menu .disabled>a:hover{color:#999}.dropdown-menu .disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.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}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover .dropdown-menu{display:block}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.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 #e3e3e3;-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;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;overflow:visible \9;-webkit-transition:height .35s ease;-moz-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:20px;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 14px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;*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:-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(to bottom,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #bbb;*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:#a2a2a2;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',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]{color:#333;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;-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:16px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:2px}.btn-small{padding:3px 9px;font-size:12px;line-height:18px}.btn-small [class^="icon-"]{margin-top:0}.btn-mini{padding:2px 6px;font-size:11px;line-height:16px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}.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:#c5c5c5;border-color:rgba(0,0,0,0.15) rgba(0,0,0,0.15) rgba(0,0,0,0.25)}.btn-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-image:-moz-linear-gradient(top,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;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='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#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(to bottom,#fbb450,#f89406);background-image:-moz-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='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#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:linear-gradient(to bottom,#ee5f5b,#bd362f);background-image:-moz-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='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#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:linear-gradient(to bottom,#62c462,#51a351);background-image:-moz-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='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#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:linear-gradient(to bottom,#5bc0de,#2f96b4);background-image:-moz-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='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-image:-moz-linear-gradient(top,#444,#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='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}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-link,.btn-link:active{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover{color:#005580;text-decoration:underline;background-color:transparent}.btn-group{position:relative;*margin-left:.3em;font-size:0;white-space:nowrap}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-toolbar .btn+.btn,.btn-toolbar .btn-group+.btn,.btn-toolbar .btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu{font-size:14px}.btn-group>.btn-mini{font-size:11px}.btn-group>.btn-small{font-size:12px}.btn-group>.btn-large{font-size:16px}.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>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;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-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;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:#04c}.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:8px;margin-left:0}.btn-mini .caret,.btn-small .caret,.btn-large .caret{margin-top:6px}.btn-large .caret{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}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical .btn{display:block;float:none;width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical .btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical .btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical .btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical .btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical .btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;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 h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.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:20px;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-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;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:9px 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;line-height:0;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:20px;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-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft: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 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover{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;line-height:0;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}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible;color:#555}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#d6f2e2);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#d6f2e2));background-image:-webkit-linear-gradient(top,#fff,#d2e2f2);background-image:-o-linear-gradient(top,#fff,#d2e2f2);background-image:linear-gradient(to bottom,#fff,#d2e2f2);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffd2e2f2',GradientType=0);-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#555;text-shadow:0 1px 0 #fff}.navbar .brand:hover{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px}.navbar-link{color:#555}.navbar-link:hover{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:6px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;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,.navbar-form .btn{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:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;width:100%;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.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,.navbar-static-top .navbar-inner{border: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-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#555;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;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='#fff2f2f2',endColorstr='#ffe5e5e5',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]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#ccc \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 .nav>li>.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 .nav>li>.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 .nav>li>.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 .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse{color:#999}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover{color:#fff}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-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;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;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-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-image:-moz-linear-gradient(top,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #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='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#ccc}.breadcrumb .active{color:#999}.pagination{height:40px;margin:20px 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,.pagination span{float:left;padding:0 14px;line-height:38px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a,.pagination .active span{background-color:#f5f5f5}.pagination .active a,.pagination .active span{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,.pagination li:first-child span{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,.pagination li:last-child span{-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:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;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:2080}.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;-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-header h3{margin:0;line-height:30px}.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;line-height:0;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:1030;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:-3px}.tooltip.right{margin-left:3px}.tooltip.bottom{margin-top:3px}.tooltip.left{margin-left:-3px}.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;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;width:236px;padding:1px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-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}.popover.top{margin-bottom:10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-right:10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.popover .arrow,.popover .arrow:after{position:absolute;display:inline-block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow:after{z-index:-1;content:""}.popover.top .arrow{bottom:-10px;left:50%;margin-left:-10px;border-top-color:#fff;border-width:10px 10px 0}.popover.top .arrow:after{bottom:-1px;left:-11px;border-top-color:rgba(0,0,0,0.25);border-width:11px 11px 0}.popover.right .arrow{top:50%;left:-10px;margin-top:-10px;border-right-color:#fff;border-width:10px 10px 10px 0}.popover.right .arrow:after{bottom:-11px;left:-1px;border-right-color:rgba(0,0,0,0.25);border-width:11px 11px 11px 0}.popover.bottom .arrow{top:-10px;left:50%;margin-left:-10px;border-bottom-color:#fff;border-width:0 10px 10px}.popover.bottom .arrow:after{top:-1px;left:-11px;border-bottom-color:rgba(0,0,0,0.25);border-width:0 11px 11px}.popover.left .arrow{top:50%;right:-10px;margin-top:-10px;border-left-color:#fff;border-width:10px 0 10px 10px}.popover.left .arrow:after{right:-1px;bottom:-11px;border-left-color:rgba(0,0,0,0.25);border-width:11px 0 11px 11px}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}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;color:#555}.label,.badge{font-size:11.844px;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}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-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:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-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(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#fff9f9f9',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{float:left;width:0;height:100%;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(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',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;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;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:-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);-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,.progress .bar-danger{background-color:#dd514c;background-image:-moz-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(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{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:-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,.progress .bar-success{background-color:#5eb95e;background-image:-moz-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(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{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:-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,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-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(to bottom,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff339bb9',GradientType=0)}.progress-info.progress-striped .bar,.progress-striped .bar-info{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:-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,.progress .bar-warning{background-color:#faa732;background-image:-moz-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(to bottom,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0)}.progress-warning.progress-striped .bar,.progress-striped .bar-warning{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:-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:20px}.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:20px;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;-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:15px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{line-height:20px;color:#fff}.carousel-caption h4{margin:0 0 5px}.carousel-caption p{margin-bottom:0}.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:30px;color:inherit}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden}.affix{position:fixed} + */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{height:auto;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;line-height:0;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:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-static-top .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;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-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.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:20px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:1;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1{font-size:36px;line-height:40px}h2{font-size:30px;line-height:40px}h3{font-size:24px;line-height:40px}h4{font-size:18px;line-height:20px}h5{font-size:14px;line-height:20px}h6{font-size:12px;line-height:20px}h1 small{font-size:24px}h2 small{font-size:18px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.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:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}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 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:25px}blockquote small{display:block;line-height:20px;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}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,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:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;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:20px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}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:20px;padding:4px 6px;margin-bottom:9px;font-size:14px;line-height:20px;color:#555;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}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-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;-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:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal;cursor:pointer}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;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}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-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)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.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[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .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}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"]{float:left}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}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;-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)}.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:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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;-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)}.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:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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;-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)}.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:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),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:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{margin-bottom:5px;font-size:0;white-space:nowrap}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;font-size:14px;vertical-align:top;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn{margin-left:-1px;vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .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 .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}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 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;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{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:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;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:10px;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:20px}.table th,.table td{padding:8px;line-height:20px;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:collapse;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,.table-bordered tfoot: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,.table-bordered tfoot: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-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-right-topleft:4px}.table-striped tbody tr:nth-child(odd) td,.table-striped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table-hover tbody tr:hover td,.table-hover tbody tr:hover th{background-color:#f5f5f5}table [class*=span],.row-fluid table [class*=span]{display:table-cell;float:none;margin-left:0}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}.table tbody tr.success td{background-color:#dff0d8}.table tbody tr.error td{background-color:#f2dede}.table tbody tr.info td{background-color:#d9edf7}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*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}.icon-white,.nav>.active>a>[class^="icon-"],.nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"]{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{width:16px;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{width:16px;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:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 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:6px;-moz-border-radius:6px;border-radius:6px;-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:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu li>a:focus,.dropdown-submenu:hover>a{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0088cc;background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0081c2;background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-repeat:repeat-x;outline:0;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .disabled>a,.dropdown-menu .disabled>a:hover{color:#999}.dropdown-menu .disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.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}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover .dropdown-menu{display:block}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.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 #e3e3e3;-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;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;overflow:visible \9;-webkit-transition:height .35s ease;-moz-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:20px;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 14px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;*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:-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(to bottom,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #bbb;*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:#a2a2a2;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',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]{color:#333;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;-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:16px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:2px}.btn-small{padding:3px 9px;font-size:12px;line-height:18px}.btn-small [class^="icon-"]{margin-top:0}.btn-mini{padding:2px 6px;font-size:11px;line-height:16px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}.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:#c5c5c5;border-color:rgba(0,0,0,0.15) rgba(0,0,0,0.15) rgba(0,0,0,0.25)}.btn-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-image:-moz-linear-gradient(top,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;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='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#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(to bottom,#fbb450,#f89406);background-image:-moz-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='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#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:linear-gradient(to bottom,#ee5f5b,#bd362f);background-image:-moz-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='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#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:linear-gradient(to bottom,#62c462,#51a351);background-image:-moz-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='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#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:linear-gradient(to bottom,#5bc0de,#2f96b4);background-image:-moz-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='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-image:-moz-linear-gradient(top,#444,#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='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}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-link,.btn-link:active{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover{color:#005580;text-decoration:underline;background-color:transparent}.btn-group{position:relative;*margin-left:.3em;font-size:0;white-space:nowrap}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-toolbar .btn+.btn,.btn-toolbar .btn-group+.btn,.btn-toolbar .btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu{font-size:14px}.btn-group>.btn-mini{font-size:11px}.btn-group>.btn-small{font-size:12px}.btn-group>.btn-large{font-size:16px}.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>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;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-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;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:#04c}.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:8px;margin-left:0}.btn-mini .caret,.btn-small .caret,.btn-large .caret{margin-top:6px}.btn-large .caret{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}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical .btn{display:block;float:none;width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical .btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical .btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical .btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical .btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical .btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;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 h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.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:20px;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-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;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:9px 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;line-height:0;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:20px;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-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft: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 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover{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;line-height:0;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}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible;color:#555}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#d4e4f4);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#d4e4f4));background-image:-webkit-linear-gradient(top,#fff,#d4e4f4);background-image:-o-linear-gradient(top,#fff,#d4e4f4);background-image:linear-gradient(to bottom,#fff,#d4e4f4);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffd4e4f4',GradientType=0);-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#555;text-shadow:0 1px 0 #fff}.navbar .brand:hover{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px}.navbar-link{color:#555}.navbar-link:hover{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:6px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;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,.navbar-form .btn{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:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;width:100%;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.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,.navbar-static-top .navbar-inner{border: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-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#555;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;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='#fff2f2f2',endColorstr='#ffe5e5e5',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]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#ccc \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 .nav>li>.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 .nav>li>.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 .nav>li>.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 .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse{color:#999}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover{color:#fff}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-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;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;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-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-image:-moz-linear-gradient(top,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #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='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#ccc}.breadcrumb .active{color:#999}.pagination{height:40px;margin:20px 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,.pagination span{float:left;padding:0 14px;line-height:38px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a,.pagination .active span{background-color:#f5f5f5}.pagination .active a,.pagination .active span{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,.pagination li:first-child span{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,.pagination li:last-child span{-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:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;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:2080}.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;-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-header h3{margin:0;line-height:30px}.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;line-height:0;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:1030;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:-3px}.tooltip.right{margin-left:3px}.tooltip.bottom{margin-top:3px}.tooltip.left{margin-left:-3px}.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;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;width:236px;padding:1px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-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}.popover.top{margin-bottom:10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-right:10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.popover .arrow,.popover .arrow:after{position:absolute;display:inline-block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow:after{z-index:-1;content:""}.popover.top .arrow{bottom:-10px;left:50%;margin-left:-10px;border-top-color:#fff;border-width:10px 10px 0}.popover.top .arrow:after{bottom:-1px;left:-11px;border-top-color:rgba(0,0,0,0.25);border-width:11px 11px 0}.popover.right .arrow{top:50%;left:-10px;margin-top:-10px;border-right-color:#fff;border-width:10px 10px 10px 0}.popover.right .arrow:after{bottom:-11px;left:-1px;border-right-color:rgba(0,0,0,0.25);border-width:11px 11px 11px 0}.popover.bottom .arrow{top:-10px;left:50%;margin-left:-10px;border-bottom-color:#fff;border-width:0 10px 10px}.popover.bottom .arrow:after{top:-1px;left:-11px;border-bottom-color:rgba(0,0,0,0.25);border-width:0 11px 11px}.popover.left .arrow{top:50%;right:-10px;margin-top:-10px;border-left-color:#fff;border-width:10px 0 10px 10px}.popover.left .arrow:after{right:-1px;bottom:-11px;border-left-color:rgba(0,0,0,0.25);border-width:11px 0 11px 11px}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}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;color:#555}.label,.badge{font-size:11.844px;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}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-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:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-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(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#fff9f9f9',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{float:left;width:0;height:100%;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(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',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;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;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:-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);-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,.progress .bar-danger{background-color:#dd514c;background-image:-moz-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(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{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:-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,.progress .bar-success{background-color:#5eb95e;background-image:-moz-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(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{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:-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,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-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(to bottom,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff339bb9',GradientType=0)}.progress-info.progress-striped .bar,.progress-striped .bar-info{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:-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,.progress .bar-warning{background-color:#faa732;background-image:-moz-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(to bottom,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0)}.progress-warning.progress-striped .bar,.progress-striped .bar-warning{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:-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:20px}.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:20px;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;-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:15px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{line-height:20px;color:#fff}.carousel-caption h4{margin:0 0 5px}.carousel-caption p{margin-bottom:0}.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:30px;color:inherit}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden}.affix{position:fixed} diff --git a/docs/index.md b/docs/index.md index c7018d88468..7d739299409 100644 --- a/docs/index.md +++ b/docs/index.md @@ -5,7 +5,7 @@ title: Spark Overview Apache Spark is a fast and general-purpose cluster computing system. It provides high-level APIs in [Scala](scala-programming-guide.html), [Java](java-programming-guide.html), and [Python](python-programming-guide.html) that make parallel jobs easy to write, and an optimized engine that supports general computation graphs. -Spark can run on the Apache Mesos cluster manager, Hadoop YARN, Amazon EC2, or without an independent resource manager ("standalone mode"). +It also supports a rich set of higher-level tools including [Shark](http://shark.cs.berkeley.edu) (Hive on Spark), [MLlib](mllib-guide.html) for machine learning, [Bagel](bagel-programming-guide.html) for graph processing, and [Spark Streaming](streaming-programming-guide.html). # Downloading @@ -37,6 +37,15 @@ or `local` to run locally with one thread, or `local[N]` to run locally with N t Finally, Spark can be used interactively through modified versions of the Scala shell (`./spark-shell`) or Python interpreter (`./pyspark`). These are a great way to learn Spark. +# Running on a Cluster + +Spark supports several options for deployment: + +* [Amazon EC2](ec2-scripts.html): our scripts let you launch a cluster in about 5 minutes +* [Standalone Deploy Mode](spark-standalone.html): simplest way to deploy Spark on a private cluster +* [Apache Mesos](running-on-mesos.html) +* [Hadoop YARN](running-on-yarn.html) + # A Note About Hadoop Versions Spark uses the Hadoop-client library to talk to HDFS and other Hadoop-supported @@ -94,6 +103,7 @@ In addition, if you wish to run Spark on [YARN](running-on-yarn.md), set **External resources:** * [Spark Homepage](http://spark.incubator.apache.org) +* [Shark](http://shark.cs.berkeley.edu): Apache Hive over Spark * [Mailing Lists](http://spark.incubator.apache.org/mailing-lists.html): ask questions about Spark here * [AMP Camps](http://ampcamp.berkeley.edu/): a series of training camps at UC Berkeley that featured talks and exercises about Spark, Shark, Mesos, and more. [Videos](http://ampcamp.berkeley.edu/agenda-2012), From 4cc27d962b2cb13bed8588808d8169ec3132f39f Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 2 Sep 2013 14:34:09 -0700 Subject: [PATCH 0918/2521] Add Apache incubator notice to README --- README.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/README.md b/README.md index c4170650f79..ca1d66d80df 100644 --- a/README.md +++ b/README.md @@ -89,6 +89,17 @@ Please refer to the [Configuration guide](http://spark.incubator.apache.org/docs in the online documentation for an overview on how to configure Spark. +## Apache Incubator Notice + +Apache Spark is an effort undergoing incubation at The Apache Software +Foundation (ASF), sponsored by the Apache Incubator. Incubation is required of +all newly accepted projects until a further review indicates that the +infrastructure, communications, and decision making process have stabilized in +a manner consistent with other successful ASF projects. While incubation status +is not necessarily a reflection of the completeness or stability of the code, +it does indicate that the project has yet to be fully endorsed by the ASF. + + ## Contributing to Spark Contributions via GitHub pull requests are gladly accepted from their original From f586c8ef38be44847fc8d720825654a067d19c84 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 2 Sep 2013 14:56:49 -0700 Subject: [PATCH 0919/2521] Updated LICENSE with third-party licenses --- LICENSE | 196 ++++++++++++++++++ .../org/apache/spark/repl/SparkHelper.scala | 17 ++ 2 files changed, 213 insertions(+) diff --git a/LICENSE b/LICENSE index d6456956733..1c166d13336 100644 --- a/LICENSE +++ b/LICENSE @@ -200,3 +200,199 @@ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. + + +======================================================================= +Apache Spark Subcomponents: + +The Apache Spark project contains subcomponents with separate copyright +notices and license terms. Your use of the source code for the these +subcomponents is subject to the terms and conditions of the following +licenses. + + +======================================================================= +For the Boto EC2 library (ec2/third_party/boto*.zip): +======================================================================= + +Copyright (c) 2006-2008 Mitch Garnaat http://garnaat.org/ + +Permission is hereby granted, free of charge, to any person obtaining a +copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, dis- +tribute, sublicense, and/or sell copies of the Software, and to permit +persons to whom the Software is furnished to do so, subject to the fol- +lowing conditions: + +The above copyright notice and this permission notice shall be included +in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS +OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABIL- +ITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT +SHALL THE AUTHOR BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, +WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS +IN THE SOFTWARE. + + +======================================================================== +For CloudPickle (pyspark/cloudpickle.py): +======================================================================== + +Copyright (c) 2012, Regents of the University of California. +Copyright (c) 2009 `PiCloud, Inc. `_. +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. + + +======================================================================== +For Py4J (python/lib/py4j0.7.egg and files in assembly/lib/net/sf/py4j): +======================================================================== + +Copyright (c) 2009-2011, Barthelemy Dagenais 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. + +- The name of the author may not 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. + + +======================================================================== +For DPark join code (python/pyspark/join.py): +======================================================================== + +Copyright (c) 2011, Douban Inc. +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 Douban Inc. 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 +OWNER 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. + + +======================================================================== +For sorttable (core/src/main/resources/org/apache/spark/ui/static/sorttable.js): +======================================================================== + +Copyright (c) 1997-2007 Stuart Langridge + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. + + +======================================================================== +For Scala Interpreter classes (all .scala files in repl/src/main/scala +except for Main.Scala, SparkHelper.scala and ExecutorClassLoader.scala): +======================================================================== + +Copyright (c) 2002-2013 EPFL +Copyright (c) 2011-2013 Typesafe, Inc. + +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 EPFL 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 OWNER 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. diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkHelper.scala b/repl/src/main/scala/org/apache/spark/repl/SparkHelper.scala index d8fb7191b44..5340951d913 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkHelper.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkHelper.scala @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package scala.tools.nsc object SparkHelper { From cf7b115496d6847100fee82ce5a1435b52109538 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Mon, 2 Sep 2013 18:21:21 -0700 Subject: [PATCH 0920/2521] Enabling getting the actual WEBUI port --- .../org/apache/spark/deploy/DeployMessage.scala | 4 ++++ .../apache/spark/deploy/LocalSparkCluster.scala | 2 +- .../org/apache/spark/deploy/master/Master.scala | 17 ++++++++++++++--- 3 files changed, 19 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index c31619db279..14a453699d3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -127,4 +127,8 @@ private[deploy] object DeployMessages { case object CheckForWorkerTimeOut + case object RequestWebUIPort + + case class WebUIPortResponse(boundedPort: Int) {} + } diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index 78e3747ad81..10161c82041 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -43,7 +43,7 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I logInfo("Starting a local Spark cluster with " + numWorkers + " workers.") /* Start the Master */ - val (masterSystem, masterPort) = Master.startSystemAndActor(localHostname, 0, 0) + val (masterSystem, masterPort, _) = Master.startSystemAndActor(localHostname, 0, 0) masterActorSystems += masterSystem val masterUrl = "spark://" + localHostname + ":" + masterPort diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 7cf0a7754f7..50d5900ecd4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -26,6 +26,8 @@ import akka.actor._ import akka.actor.Terminated import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown} import akka.util.duration._ +import akka.dispatch.Await +import akka.pattern.ask import org.apache.spark.{Logging, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} @@ -33,6 +35,8 @@ import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.{Utils, AkkaUtils} +import akka.util.Timeout +import java.util.concurrent.TimeUnit private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { @@ -180,6 +184,10 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act case CheckForWorkerTimeOut => { timeOutDeadWorkers() } + + case RequestWebUIPort => { + sender ! WebUIPortResponse(webUi.boundPort.getOrElse(-1)) + } } /** @@ -364,7 +372,7 @@ private[spark] object Master { def main(argStrings: Array[String]) { val args = new MasterArguments(argStrings) - val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort) + val (actorSystem, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort) actorSystem.awaitTermination() } @@ -378,9 +386,12 @@ private[spark] object Master { } } - def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int) = { + def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int, Int) = { val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port) val actor = actorSystem.actorOf(Props(new Master(host, boundPort, webUiPort)), name = actorName) - (actorSystem, boundPort) + implicit val timeout = Timeout(1 seconds) + val respFuture = actor ? RequestWebUIPort + val resp = Await.result(respFuture, timeout.duration).asInstanceOf[WebUIPortResponse] + (actorSystem, boundPort, resp.boundedPort) } } From 59218bdd4996a13116009e3669b1b875be23a694 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 2 Sep 2013 18:34:03 -0700 Subject: [PATCH 0921/2521] Add Apache parent POM --- pom.xml | 5 +++++ project/SparkBuild.scala | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/pom.xml b/pom.xml index 6a73e81da0b..c561b099ab7 100644 --- a/pom.xml +++ b/pom.xml @@ -18,6 +18,11 @@ 4.0.0 + + org.apache + apache + 11 + org.apache.spark spark-parent 0.8.0-SNAPSHOT diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b1b99b37c4f..d038a4f4796 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -103,6 +103,11 @@ object SparkBuild extends Build { //useGpg in Global := true, pomExtra := ( + + org.apache + apache + 13 + http://spark.incubator.apache.org/ From e452bd6d77bf1824a128599de2c3ec3663808ab6 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Mon, 2 Sep 2013 19:04:08 -0700 Subject: [PATCH 0922/2521] Brushing the code up slightly --- .../main/scala/org/apache/spark/deploy/DeployMessage.scala | 2 +- .../main/scala/org/apache/spark/deploy/master/Master.scala | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index 14a453699d3..b8856e6e001 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -129,6 +129,6 @@ private[deploy] object DeployMessages { case object RequestWebUIPort - case class WebUIPortResponse(boundedPort: Int) {} + case class WebUIPortResponse(webUIBoundPort: Int) {} } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 50d5900ecd4..dd7dcae6194 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -389,9 +389,9 @@ private[spark] object Master { def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int, Int) = { val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port) val actor = actorSystem.actorOf(Props(new Master(host, boundPort, webUiPort)), name = actorName) - implicit val timeout = Timeout(1 seconds) - val respFuture = actor ? RequestWebUIPort + implicit val timeout = Timeout(5 seconds) + val respFuture = actor ? RequestWebUIPort // ask pattern val resp = Await.result(respFuture, timeout.duration).asInstanceOf[WebUIPortResponse] - (actorSystem, boundPort, resp.boundedPort) + (actorSystem, boundPort, resp.webUIBoundPort) } } From 36d8fca2cca7cb3f8c490631b0f6699a119e1230 Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Mon, 2 Sep 2013 19:31:09 -0700 Subject: [PATCH 0923/2521] Reynold's comment fixed --- core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index b8856e6e001..1cfff5e565d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -129,6 +129,6 @@ private[deploy] object DeployMessages { case object RequestWebUIPort - case class WebUIPortResponse(webUIBoundPort: Int) {} + case class WebUIPortResponse(webUIBoundPort: Int) } From cbfef9b3ffadbdb8e39d092a97090de5ffd7c1dd Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Mon, 2 Sep 2013 19:33:55 -0700 Subject: [PATCH 0924/2521] Sort order of imports to match project guidelines --- .../scala/org/apache/spark/deploy/master/Master.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index dd7dcae6194..de4d39bbeed 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -17,17 +17,19 @@ package org.apache.spark.deploy.master -import java.text.SimpleDateFormat +import java.util.concurrent.TimeUnit import java.util.Date +import java.text.SimpleDateFormat import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import akka.actor._ import akka.actor.Terminated -import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown} -import akka.util.duration._ import akka.dispatch.Await import akka.pattern.ask +import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown} +import akka.util.duration._ +import akka.util.Timeout import org.apache.spark.{Logging, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} @@ -35,8 +37,6 @@ import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.{Utils, AkkaUtils} -import akka.util.Timeout -import java.util.concurrent.TimeUnit private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { From bd0788505f4ebac37a3cdf4753589deb58455c1b Mon Sep 17 00:00:00 2001 From: Ali Ghodsi Date: Tue, 3 Sep 2013 00:50:35 -0700 Subject: [PATCH 0925/2521] Using configured akka timeouts --- .../scala/org/apache/spark/deploy/master/Master.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index dd7dcae6194..e918a33d09d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -35,7 +35,7 @@ import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.{Utils, AkkaUtils} -import akka.util.Timeout +import akka.util.{Duration, Timeout} import java.util.concurrent.TimeUnit @@ -389,9 +389,11 @@ private[spark] object Master { def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int, Int) = { val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port) val actor = actorSystem.actorOf(Props(new Master(host, boundPort, webUiPort)), name = actorName) - implicit val timeout = Timeout(5 seconds) + val timeoutDuration = Duration.create( + System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + implicit val timeout = Timeout(timeoutDuration) val respFuture = actor ? RequestWebUIPort // ask pattern - val resp = Await.result(respFuture, timeout.duration).asInstanceOf[WebUIPortResponse] + val resp = Await.result(respFuture, timeoutDuration).asInstanceOf[WebUIPortResponse] (actorSystem, boundPort, resp.webUIBoundPort) } } From c8cc27611002f4310bc386df858059700a3fec9d Mon Sep 17 00:00:00 2001 From: "Y.CORP.YAHOO.COM\\tgraves" Date: Tue, 3 Sep 2013 10:50:21 -0500 Subject: [PATCH 0926/2521] Review comment changes and update to org.apache packaging --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../scala/org/apache/spark/SparkEnv.scala | 2 +- .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../scala/org/apache/spark/ui/SparkUI.scala | 1 - .../scala/org/apache/spark/ui/UIUtils.scala | 43 +++++++------------ .../scala/org/apache/spark/util/Utils.scala | 2 +- docs/running-on-yarn.md | 6 +-- .../spark/deploy/yarn/ApplicationMaster.scala | 17 ++++---- .../yarn/ApplicationMasterArguments.scala | 2 +- .../org/apache/spark/deploy/yarn/Client.scala | 5 ++- .../spark/deploy/yarn/ClientArguments.scala | 2 +- .../spark/deploy/yarn/WorkerRunnable.scala | 3 +- .../deploy/yarn/YarnAllocationHandler.scala | 3 +- .../cluster/YarnClusterScheduler.scala | 1 + 14 files changed, 41 insertions(+), 50 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index f2641851cbe..89318712a57 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -196,7 +196,7 @@ class SparkContext( case "yarn-standalone" => val scheduler = try { - val clazz = Class.forName("spark.scheduler.cluster.YarnClusterScheduler") + val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClusterScheduler") val cons = clazz.getConstructor(classOf[SparkContext]) cons.newInstance(this).asInstanceOf[ClusterScheduler] } catch { diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 478e5a0aaf2..29968c273c3 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -62,7 +62,7 @@ class SparkEnv ( val yarnMode = java.lang.Boolean.valueOf(System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))) if(yarnMode) { try { - Class.forName("spark.deploy.yarn.YarnSparkHadoopUtil").newInstance.asInstanceOf[SparkHadoopUtil] + Class.forName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil").newInstance.asInstanceOf[SparkHadoopUtil] } catch { case th: Throwable => throw new SparkException("Unable to load YARN support", th) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 3f8a96fc472..9a2cf20de79 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -50,7 +50,7 @@ private[spark] class SparkDeploySchedulerBackend( "org.apache.spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome().getOrElse(null) val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome, - sc.ui.appHttpUIAddress) + "http://" + sc.ui.appUIAddress) client = new Client(sc.env.actorSystem, master, appDesc, this) client.start() diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 5d54420e722..48eb0960635 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -79,7 +79,6 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { server.foreach(_.stop()) } - private[spark] def appHttpUIAddress = "http://" + appUIAddress private[spark] def appUIAddress = host + ":" + boundPort.getOrElse("-1") } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 9190b8ac2c1..d3af75630ee 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -33,49 +33,38 @@ private[spark] object UIUtils { return uiRoot + resource } - private[spark] val storageStr = addBaseUri("/storage") - private[spark] val stagesStr = addBaseUri("/stages") - private[spark] val envStr = addBaseUri("/environment") - private[spark] val executorsStr = addBaseUri("/executors") - private[spark] val bootstrapMinCssStr = addBaseUri("/static/bootstrap.min.css") - private[spark] val webuiCssStr = addBaseUri("/static/webui.css") - private[spark] val sortTableStr = addBaseUri("/static/sorttable.js") - private[spark] val sparkLogoHdStr = addBaseUri("/static/spark-logo-77x50px-hd.png") - private[spark] val sparkLogoStr = addBaseUri("/static/spark_logo.png") - - /** Returns a spark page with correctly formatted headers */ def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String, page: Page.Value) : Seq[Node] = { val jobs = page match { - case Stages =>
  • Stages
  • - case _ =>
  • Stages
  • + case Stages =>
  • Stages
  • + case _ =>
  • Stages
  • } val storage = page match { - case Storage =>
  • Storage
  • - case _ =>
  • Storage
  • + case Storage =>
  • Storage
  • + case _ =>
  • Storage
  • } val environment = page match { - case Environment =>
  • Environment
  • - case _ =>
  • Environment
  • + case Environment =>
  • Environment
  • + case _ =>
  • Environment
  • } val executors = page match { - case Executors =>
  • Executors
  • - case _ =>
  • Executors
  • + case Executors =>
  • Executors
  • + case _ =>
  • Executors
  • } - - - + + + {sc.appName} - {title}